diff --git a/nodestream/cli/operations/run_pipeline.py b/nodestream/cli/operations/run_pipeline.py
index 367425f00..9ad2bd792 100644
--- a/nodestream/cli/operations/run_pipeline.py
+++ b/nodestream/cli/operations/run_pipeline.py
@@ -122,6 +122,7 @@ def create_progress_reporter(
callback=indicator.progress_callback,
on_start_callback=indicator.on_start,
on_finish_callback=indicator.on_finish,
+ on_fatal_error_callback=indicator.on_fatal_error,
)
@@ -139,8 +140,18 @@ def progress_callback(self, _, __):
def on_finish(self, context: PipelineContext):
pass
+ def on_fatal_error(self, exception: Exception):
+ self.command.line(
+ "Encountered a fatal error while running pipeline"
+ )
+ self.command.line(f"{exception}")
+
class SpinnerProgressIndicator(ProgressIndicator):
+ def __init__(self, command: NodestreamCommand, pipeline_name: str) -> None:
+ super().__init__(command, pipeline_name)
+ self.exception = None
+
def on_start(self):
self.progress = self.command.progress_indicator()
self.progress.start(f"Running pipeline: '{self.pipeline_name}'")
@@ -156,3 +167,12 @@ def on_finish(self, context: PipelineContext):
stats = ((k, str(v)) for k, v in context.stats.items())
table = self.command.table(STATS_TABLE_COLS, stats)
table.render()
+
+ if self.exception:
+ raise self.exception
+
+ def on_fatal_error(self, exception: Exception):
+ self.progress.set_message(
+ "Encountered a fatal error while running pipeline"
+ )
+ self.exception = exception
diff --git a/nodestream/databases/null.py b/nodestream/databases/null.py
index eaad1510f..cc88d66d9 100644
--- a/nodestream/databases/null.py
+++ b/nodestream/databases/null.py
@@ -1,7 +1,6 @@
from typing import AsyncGenerator, Iterable
from ..model import IngestionHook, Node, RelationshipWithNodes, TimeToLiveConfiguration
-from ..pipeline.pipeline import empty_async_generator
from ..schema.migrations import Migrator
from ..schema.migrations.operations import Operation
from .copy import TypeRetriever
@@ -13,6 +12,11 @@
)
+async def empty_async_generator():
+ for i in []:
+ yield i # pragma: no cover
+
+
class NullMigrator(Migrator):
async def execute_operation(self, _: Operation):
pass
diff --git a/nodestream/pipeline/channel.py b/nodestream/pipeline/channel.py
new file mode 100644
index 000000000..03ad03a0b
--- /dev/null
+++ b/nodestream/pipeline/channel.py
@@ -0,0 +1,163 @@
+import asyncio
+from asyncio import Queue, wait_for
+from typing import Optional, Tuple
+
+
+# WARNING: DoneObject refernces should not exist beyond this module.
+class DoneObject:
+ """A `DoneObject` is a marker object that indicates that a step is done.
+
+ When a step is done processing records, it should emit a `DoneObject` to
+ indicate that it is finished. This allows downstream steps to know when a
+ step is done and they can stop processing records.
+
+ The `DoneObject` is a special object that is used to signal the end of a
+ stream of records. It is not a record itself and should not be processed as
+ such.
+ """
+
+ pass
+
+
+CHANNEL_TIMEOUT = 0.25
+
+
+class Channel:
+ """`Channel` is a communication channel between steps in a pipeline.
+
+ A `Channel` is used to pass records between steps in a pipeline. It is a
+ simple queue that can be used to pass records from one step to another. The
+ `Channel` is asynchronous and can be used to pass records between steps in
+ an asynchronous context.
+
+ A `Channel` has a fixed size and will block if the queue is full. This
+ allows the pipeline to control the flow of records between steps and
+ prevent one step from overwhelming another step with too many records.
+ """
+
+ __slots__ = ("queue", "input_dropped")
+
+ def __init__(self, size: int) -> None:
+ self.queue = Queue(maxsize=size)
+ self.input_dropped = False
+
+ async def get(self):
+ """Get an object from the channel.
+
+ This method is used to get an object from the channel. It will block
+ until an object is available in the channel. If the channel is empty,
+ it will block until an object is available.
+
+ Returns:
+ object: The object that was retrieved from the channel.
+ """
+ return await self.queue.get()
+
+ async def put(self, obj) -> bool:
+ """Put an object in the channel.
+
+ This method is used to put an object in the channel. It will block
+ until the object is put in the channel. If the channel is full, it
+ will block until there is space in the channel.
+
+ Returns:
+ bool: True if the object was successfully put in the channel, False
+ if the channel is full and the object was not put in the channel.
+ """
+ try:
+ await wait_for(self.queue.put(obj), timeout=CHANNEL_TIMEOUT)
+ return True
+ except (TimeoutError, asyncio.TimeoutError):
+ return False
+
+
+class StepOutput:
+ """`StepOutput` is an output channel for a step in a pipeline.
+
+ A `StepOutput` is used to pass records from a step to the next step in a
+ pipeline. It is a simple wrapper around a `Channel` that provides a more
+ convenient interface for putting records in the channel.
+ """
+
+ __slots__ = ("channel",)
+
+ def __init__(self, channel: Channel) -> None:
+ self.channel = channel
+
+ async def done(self):
+ """Mark the output channel as done.
+
+ This method is used to mark the output channel as done. It will put a
+ `DoneObject` in the channel to indicate that the step is finished
+ processing records. This allows downstream steps to know when the step
+ is done and they can stop processing records.
+ """
+ await self.put(DoneObject)
+
+ async def put(self, obj) -> bool:
+ """Put an object in the output channel.
+
+ This method is used to put an object in the output channel. It will
+ block until the object is put in the channel. If the channel is full,
+ it will block until there is space in the channel unless the channel is
+ closed on the other end.
+
+ Returns:
+ bool: True if the object was successfully put in the channel, False
+ if the channel is closed on the other end and the object was not
+ put in the channel.
+ """
+ successfully_put = False
+ while not successfully_put and not self.channel.input_dropped:
+ successfully_put = await self.channel.put(obj)
+
+ return successfully_put
+
+
+class StepInput:
+ """`StepInput` is an input channel for a step in a pipeline.
+
+ A `StepInput` is used to get records from the previous step in a pipeline.
+ It is a simple wrapper around a `Channel` that provides a more convenient
+ interface for getting records from the channel.
+
+ The `StepInput` is an asynchronous generator that can be used to get
+ records from the channel in an asynchronous context.
+ """
+
+ __slots__ = ("channel",)
+
+ def __init__(self, channel: Channel) -> None:
+ self.channel = channel
+
+ async def get(self) -> Optional[object]:
+ """Get an object from the input channel.
+
+ This method is used to get an object from the input channel. It will
+ block until an object is available in the channel. Once the channel is
+ closed (i.e. a `DoneObject` is received), this method will return None.
+
+ Returns:
+ object: The object that was retrieved from the channel.
+ """
+ if (object := await self.channel.get()) is DoneObject:
+ return None
+
+ return object
+
+ def done(self):
+ """Mark the input channel as done.
+
+ This will close the channel.
+ """
+ self.channel.input_dropped = True
+
+
+def channel(size: int) -> Tuple[StepInput, StepOutput]:
+ """Create a new input and output channel.
+
+ Args:
+ size: The size of the channel.
+ """
+ channel = Channel(size)
+ return StepInput(channel), StepOutput(channel)
diff --git a/nodestream/pipeline/extractors/extractor.py b/nodestream/pipeline/extractors/extractor.py
index d7f76b6e3..259113da2 100644
--- a/nodestream/pipeline/extractors/extractor.py
+++ b/nodestream/pipeline/extractors/extractor.py
@@ -7,13 +7,12 @@
class Extractor(Step):
"""Extractors represent the source of a set of records.
- They are like any other step. However, they ignore the incoming record stream and instead produce their own
- stream of records. For this reason they generally should only be set at the beginning of a pipeline.
+ They are like any other step. However, they ignore the incoming record '
+ stream and instead produce their own stream of records. For this reason
+ they generally should only be set at the beginning of a pipeline.
"""
- def handle_async_record_stream(
- self, _: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
+ def emit_outstanding_records(self):
return self.extract_records()
@abstractmethod
diff --git a/nodestream/pipeline/filters.py b/nodestream/pipeline/filters.py
index 5c1cfa2c3..e32c2bb65 100644
--- a/nodestream/pipeline/filters.py
+++ b/nodestream/pipeline/filters.py
@@ -10,16 +10,13 @@
class Filter(Step):
"""A `Filter` takes a given record and evaluates whether or not it should continue downstream.
- `Filter` steps generally make up the middle of an ETL pipeline and are responsible
- for ensuring only relevant records make it through.
+ `Filter` steps generally make up the middle of an ETL pipeline and are
+ responsible for ensuring only relevant records make it through.
"""
- async def handle_async_record_stream(
- self, record_stream: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
- async for record in record_stream:
- if record is Flush or not await self.filter_record(record):
- yield record
+ async def process_record(self, record, _) -> AsyncGenerator[object, None]:
+ if record is Flush or not await self.filter_record(record):
+ yield record
@abstractmethod
async def filter_record(self, record: Any) -> bool:
diff --git a/nodestream/pipeline/pipeline.py b/nodestream/pipeline/pipeline.py
index 855949aeb..ec25dbbf4 100644
--- a/nodestream/pipeline/pipeline.py
+++ b/nodestream/pipeline/pipeline.py
@@ -1,280 +1,188 @@
-import asyncio
-import traceback
-from logging import getLogger
-from typing import Any, AsyncGenerator, Iterable, List, Optional
+from asyncio import create_task, gather
+from typing import Iterable, List, Tuple
from ..schema import ExpandsSchema, ExpandsSchemaFromChildren
+from .channel import StepInput, StepOutput, channel
from .meta import get_context
from .progress_reporter import PipelineProgressReporter
-from .step import Step
+from .step import Step, StepContext
-async def empty_async_generator():
- for item in []:
- yield item
-
-
-async def enumerate_async(iterable):
- count = 0
-
- async for item in iterable:
- yield count, item
- count += 1
-
-
-class DoneObject:
- pass
-
-
-START_EXCEPTION = "Exception in Start Process:"
-WORK_BODY_EXCEPTION = "Exception in Work Body:"
-STOP_EXCEPTION = "Exception in Stop Process:"
-OUTBOX_POLL_TIME = 0.1
-PRECHECK_MESSAGE = "Detected fatal error in pipeline while attempting to process record, this step can no longer continue."
-TIMEOUT_MESSAGE = "Unable to place record into outbox because the pipeline has failed, this step can no longer continue."
-
-
-class StepException(Exception):
- """
- Exception Format:
- Exceptions in StepExecutor {exec_num} ({step_name})
- Exception in Start
- Stack
- Exception in Work Body
- Stack
- Exception in Stop
- Stack
- """
-
- def __init__(self, errors: dict[str, Exception], identifier: Optional[str]):
- self.exceptions = errors
- self.identifier = identifier
- super().__init__(self.build_message())
-
- def build_message(self):
- message = ""
- for section, exception in self.exceptions.items():
- sub_message = "".join(traceback.format_tb(exception.__traceback__))
- # Break it apart, add a tab, bring it back together.
- message += f"\t{section}\n"
- for sentence in [
- "\t\t" + sentence + "\n" for sentence in sub_message.split("\n")
- ]:
- message += sentence
- return f"Exceptions in {self.identifier}:\n" + message
-
-
-class ForwardProgressHalted(Exception):
- pass
-
+class StepExecutor:
+ """`StepExecutor` is a utility that is used to run a step in a pipeline.
-class PipelineException(Exception):
- """
- Exception in Pipeline:
- Output from StepException 0
- Output from StepException 1
+ The `StepExecutor` is responsible for starting, stopping, and running a
+ step in a pipeline. It is used to execute a step by passing records
+ between the input and output channels of the step.
"""
- def __init__(self, errors: list[Exception]):
- self.errors = errors
- super().__init__(self.build_message())
-
- def build_message(self):
- message = ""
- sub_message = ""
- for error in self.errors:
- sub_message += error.build_message()
-
- # Break it apart, add a tab, bring it back together.
- for sentence in [
- "\t" + sentence + "\n" for sentence in sub_message.split("\n")
- ]:
- message += sentence
-
- return "Exceptions in Pipeline:\n" + message
+ __slots__ = ("step", "input", "output", "context")
-
-class PipelineState:
- def __init__(self) -> None:
- self.has_fatal_error = False
-
- def signal_fatal_error(self):
- self.has_fatal_error = True
-
-
-class StepExecutor:
def __init__(
self,
- pipeline_state: PipelineState,
- upstream: Optional["StepExecutor"],
step: Step,
- outbox_size: int = 0,
- step_index: Optional[int] = 0,
- progress_reporter: Optional[PipelineProgressReporter] = None,
+ input: StepInput,
+ output: StepOutput,
+ context: StepContext,
) -> None:
- self.pipeline_state = pipeline_state
- self.outbox = asyncio.Queue(maxsize=outbox_size)
- self.upstream = upstream
- self.done = False
self.step = step
- self.step_index = step_index
- self.progress_reporter = progress_reporter
- self.end_of_line = False
- self.identifier = f"{self.__class__.__name__} {self.step_index} ({self.step.__class__.__name__})"
- self.logger = getLogger(name=self.identifier)
- self.exceptions = {}
-
- async def outbox_generator(self):
- while not self.done or not self.outbox.empty():
- if (value := await self.outbox.get()) is not DoneObject:
- yield value
- self.outbox.task_done()
-
- def start(self):
- if self.progress_reporter:
- self.progress_reporter.on_start_callback()
-
- def set_end_of_line(self, progress_reporter: Optional[PipelineProgressReporter]):
- self.progress_reporter = progress_reporter
- self.end_of_line = True
-
- def pipeline_has_died(self) -> bool:
- return self.pipeline_state.has_fatal_error
-
- async def submit_object_or_die_trying(self, obj):
- should_continue = True
- while should_continue:
- if self.pipeline_has_died() and not self.done:
- raise ForwardProgressHalted(PRECHECK_MESSAGE)
- try:
- await asyncio.wait_for(self.outbox.put(obj), timeout=OUTBOX_POLL_TIME)
- should_continue = False
- except asyncio.TimeoutError:
- # We timed out, now we need to check if the pipeline has died and if so, raise an exception.
- if self.pipeline_has_died():
- raise ForwardProgressHalted(TIMEOUT_MESSAGE)
-
- async def stop(self):
- self.done = True
- await self.submit_object_or_die_trying(DoneObject)
- await self.step.finish()
-
- if self.progress_reporter:
- self.progress_reporter.on_finish_callback(get_context())
-
- async def work_body(self):
- if self.upstream is None:
- upstream = empty_async_generator()
- else:
- upstream = self.upstream.outbox_generator()
- results = self.step.handle_async_record_stream(upstream)
- async for index, record in enumerate_async(results):
- if not self.end_of_line:
- await self.submit_object_or_die_trying(record)
- if self.progress_reporter:
- self.progress_reporter.report(index, record)
-
- def try_start(self):
- # Start only calls some callbacks on the reporter.
- # If this fails, we can just log the error and move on.
+ self.input = input
+ self.output = output
+ self.context = context
+
+ async def start_step(self):
try:
- self.start()
- except Exception as start_exception:
- self.logger.exception(
- f"Exception during start for step {self.step.__class__.__name__}. This will not be considered fatal.",
- stack_info=True,
- extra={"step": self.step.__class__.__name__},
- )
- self.exceptions[START_EXCEPTION] = start_exception
+ await self.step.start(self.context)
+ except Exception as e:
+ self.context.report_error("Error starting step", e)
- async def try_work_body(self):
- # During the main exection of the code, we want to catch any
- # exceptions that happen and log them.
- # Since this step can no longer do any work, we will raise the
- # exception so that the pipeline can safely come to a stop.
+ async def stop_step(self):
try:
- await self.work_body()
- except Exception as work_body_exception:
- self.logger.exception(
- f"Exception during the work body for step {self.step.__class__.__name__}. This will not be considered fatal.",
- stack_info=True,
- extra={"step": self.step.__class__.__name__},
+ await self.step.finish(self.context)
+ except Exception as e:
+ self.context.report_error("Error stopping step", e)
+
+ async def emit_record(self, record):
+ can_continue = await self.output.put(record)
+ if not can_continue:
+ self.context.debug(
+ "Downstream is not accepting more records. Gracefully stopping."
)
- self.exceptions[WORK_BODY_EXCEPTION] = work_body_exception
- async def try_stop(self):
- # When we're done, we need to try to call stop on the step.
- # This is because the step may have some cleanup to do.
- # If this fails, we are near the end of the exection of the pipeline
- # so we can just log the error and move on.
+ return can_continue
+
+ async def drive_step(self):
try:
- await self.stop()
- except Exception as stop_exception:
- self.logger.exception(
- f"Exception during stop for step {self.step.__class__.__name__}. This will not be considered fatal.",
- stack_info=True,
- extra={"step": self.step.__class__.__name__},
- )
- self.exceptions[STOP_EXCEPTION] = stop_exception
+ while (next_record := await self.input.get()) is not None:
+ results = self.step.process_record(next_record, self.context)
+ async for record in results:
+ if not await self.emit_record(record):
+ return
+
+ async for record in self.step.emit_outstanding_records():
+ if not await self.emit_record(record):
+ return
+
+ self.context.debug("Step finished emitting")
+ except Exception as e:
+ self.context.report_error("Error running step", e, fatal=True)
+
+ async def run(self):
+ self.context.debug("Starting step")
+ await self.start_step()
+ await self.drive_step()
+ await self.output.done()
+ self.input.done()
+ await self.stop_step()
+ self.context.debug("Finished step")
+
+
+class PipelineOutput:
+ """`PipelineOutput` is an output channel for a pipeline.
+
+ A `PipelineOutput` is used to consume records from the last step in a
+ pipeline and report the progress of the pipeline.
+ """
- def raise_encountered_exceptions(self):
- if self.exceptions:
- self.pipeline_state.signal_fatal_error()
- raise StepException(errors=self.exceptions, identifier=self.identifier)
+ __slots__ = ("input", "reporter")
- async def work_loop(self):
- self.try_start()
- await self.try_work_body()
- await self.try_stop()
- self.raise_encountered_exceptions()
+ def __init__(self, input: StepInput, reporter: PipelineProgressReporter):
+ self.input = input
+ self.reporter = reporter
+
+ async def call_handling_errors(self, f, *args):
+ try:
+ f(*args)
+ except Exception:
+ self.reporter.logger.exception(f"Error running {f.__name__}")
+
+ async def run(self):
+ """Run the pipeline output.
+
+ This method is used to run the pipeline output. It will consume records
+ from the last step in the pipeline and report the progress of the
+ pipeline using the `PipelineProgressReporter`. The pipeline output will
+ block until all records have been consumed from the last step in the
+ pipeline.
+ """
+ await self.call_handling_errors(self.reporter.on_start_callback)
+
+ index = 0
+ while (obj := await self.input.get()) is not None:
+ if index % self.reporter.reporting_frequency == 0:
+ await self.call_handling_errors(self.reporter.callback, index, obj)
+ index += 1
+
+ await self.call_handling_errors(self.reporter.on_finish_callback, get_context())
class Pipeline(ExpandsSchemaFromChildren):
- """A pipeline is a series of steps that are executed in order."""
+ """`Pipeline` is a collection of steps that are executed in sequence.
+
+ A `Pipeline` is a collection of steps that are executed in sequence. Each
+ step processes records and emits new records that are passed to the next
+ step in the pipeline. The pipeline is responsible for starting, stopping,
+ and running the steps in the pipeline.
+ """
- __slots__ = ("steps",)
+ __slots__ = ("steps", "step_outbox_size")
- def __init__(self, steps: List[Step], step_outbox_size: int) -> None:
+ def __init__(self, steps: Tuple[Step, ...], step_outbox_size: int) -> None:
self.steps = steps
self.step_outbox_size = step_outbox_size
- self.logger = getLogger(self.__class__.__name__)
- self.errors = []
-
- def build_steps_into_tasks(
- self, progress_reporter: Optional[PipelineProgressReporter] = None
- ):
- current_executor = None
- tasks = []
- state = PipelineState()
- for step_index, step in enumerate(self.steps):
- current_executor = StepExecutor(
- pipeline_state=state,
- upstream=current_executor,
- step=step,
- outbox_size=self.step_outbox_size,
- step_index=step_index,
- )
- tasks.append(asyncio.create_task(current_executor.work_loop()))
- current_executor.set_end_of_line(progress_reporter)
- return tasks
-
- def propogate_errors_from_return_states(self, return_states):
- for return_state in return_states:
- if return_state:
- self.errors.append(return_state)
-
- # Raise the error and log it.
- if self.errors:
- raise PipelineException(self.errors)
-
- async def run(
- self, progress_reporter: Optional[PipelineProgressReporter] = None
- ) -> AsyncGenerator[Any, Any]:
- self.logger.info("Starting Pipeline")
- tasks = self.build_steps_into_tasks(progress_reporter)
- return_states = await asyncio.gather(*tasks, return_exceptions=True)
- self.propogate_errors_from_return_states(return_states)
- self.logger.info("Pipeline Completed")
def get_child_expanders(self) -> Iterable[ExpandsSchema]:
return (s for s in self.steps if isinstance(s, ExpandsSchema))
+
+ async def run(self, reporter: PipelineProgressReporter):
+ """Run the pipeline.
+
+ This method is used to run the pipeline. It will start, stop, and run
+ the steps in the pipeline in sequence. It will pass records between the
+ steps in the pipeline using channels. The pipeline will report on the
+ progress of the pipeline using the `PipelineProgressReporter`. The
+ pipeline will run in an asynchronous context. The pipeline will block
+ until all steps in the pipeline are finished.
+
+ This method does not return anything. If an error occurs during the
+ processing of the pipeline, it will be reported using the
+ `PipelineProgressReporter`.
+
+ Args:
+ channel_size: The size of the channels used to pass records between
+ steps in the pipeline.
+ reporter: The `PipelineProgressReporter` used to report on the
+ progress of the pipeline.
+ """
+ # Create the input and output channels for the pipeline. The input
+ # channel is used to pass records from the previous step to the current
+ # step. The output channel is used to pass records from the current
+ # step to the next step. The channels are used to pass records between
+ # the steps in the pipeline. The channels have a fixed size to control
+ # the flow of records between the steps.
+ executors: List[StepExecutor] = []
+ current_input, current_output = channel(self.step_outbox_size)
+ pipeline_output = PipelineOutput(current_input, reporter)
+
+ # Create the executors for the steps in the pipeline. The executors
+ # will be used to run the steps concurrently. The steps are created in
+ # reverse order so that the output of each step is connected to the
+ # input of the next step.
+ for reversed_index, step in reversed(list(enumerate(self.steps))):
+ index = len(self.steps) - reversed_index - 1
+ context = StepContext(step.__class__.__name__, index, reporter)
+ current_input, next_output = channel(self.step_outbox_size)
+ exec = StepExecutor(step, current_input, current_output, context)
+ current_output = next_output
+ executors.append(exec)
+
+ # There is a "leftover" input channel that is not connected to any
+ # step. This channel is connected to the first step in the pipeline
+ # so we can mark it as done since we are not going to produce anything
+ # onto it.
+ await current_output.done()
+
+ # Run the pipeline by running all the steps and the pipeline output
+ # concurrently. This will block until all steps are finished.
+ running_steps = (create_task(executor.run()) for executor in executors)
+ await gather(*running_steps, create_task(pipeline_output.run()))
diff --git a/nodestream/pipeline/progress_reporter.py b/nodestream/pipeline/progress_reporter.py
index 7f63e8f4e..636ac8408 100644
--- a/nodestream/pipeline/progress_reporter.py
+++ b/nodestream/pipeline/progress_reporter.py
@@ -32,13 +32,15 @@ class PipelineProgressReporter:
callback: Callable[[int, Any], None] = field(default=no_op)
on_start_callback: Callable[[], None] = field(default=no_op)
on_finish_callback: Callable[[PipelineContext], None] = field(default=no_op)
+ on_fatal_error_callback: Callable[[Exception], None] = field(default=no_op)
@classmethod
def for_testing(cls, results_list: list) -> "PipelineProgressReporter":
"""Create a `PipelineProgressReporter` for testing.
- This method is intended to be used for testing purposes only. It will create a
- `PipelineProgressReporter` with the default values for testing.
+ This method is intended to be used for testing purposes only. It will
+ create a `PipelineProgressReporter` with the default values for
+ testing.
Args:
results_list: The list to append results to.
diff --git a/nodestream/pipeline/step.py b/nodestream/pipeline/step.py
index 3cbae7796..ba935b8c7 100644
--- a/nodestream/pipeline/step.py
+++ b/nodestream/pipeline/step.py
@@ -1,33 +1,139 @@
-from abc import ABC, abstractmethod
-from typing import Any, AsyncGenerator
+from typing import AsyncGenerator, Optional
+from .progress_reporter import PipelineProgressReporter
-class Step(ABC):
- """A `Step` represents a phase of an ETl pipeline."""
- @classmethod
- def from_file_data(cls, **kwargs):
- return cls(**kwargs)
+class StepContext:
+ """`StepContext` is a context object that is passed to steps in a pipeline.
- @abstractmethod
- async def handle_async_record_stream(
- self, record_stream: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
- raise NotImplementedError
+ The `StepContext` provides a way for steps to interact with the pipeline
+ and report and perist information about the state of the pipeline.
+ """
- async def finish(self):
+ __slots__ = ("reporter", "index", "name")
+
+ def __init__(
+ self, name: str, index: int, reporter: PipelineProgressReporter
+ ) -> None:
+ self.name = name
+ self.reporter = reporter
+ self.index = index
+
+ def report_error(
+ self,
+ message: str,
+ exception: Optional[Exception] = None,
+ fatal: bool = False,
+ **extras,
+ ):
+ """Report an error.
+
+ This method is used to report an error that occurred during the
+ processing of a step. It can be used to log the error and take
+ appropriate action based on the error.
+
+ Args:
+ message: The error message.
+ exception: The exception that caused the error.
+ fatal: Whether the error is fatal.
+ """
+ self.reporter.logger.error(
+ message,
+ extra=dict(index=self.index, fatal=fatal, step_name=self.name, **extras),
+ exc_info=exception,
+ stack_info=True,
+ )
+ if fatal:
+ self.reporter.on_fatal_error_callback(exception)
+
+ def debug(self, message: str, **extras):
+ """Log a debug message.
+
+ This method is used to log a debug message. It can be used to log
+ information about the state of the pipeline and the steps in the
+ pipeline.
+ """
+ self.reporter.logger.debug(
+ message, extra=dict(index=self.index, step_name=self.name, **extras)
+ )
+
+ def info(self, message: str, **extras):
+ """Log an info message.
+
+ This method is used to log an info message. It can be used to log
+ information about the state of the pipeline and the steps in the
+ pipeline.
+ """
+ self.reporter.logger.info(
+ message, extra=dict(index=self.index, step_name=self.name, **extras)
+ )
+
+ def warning(self, message: str, **extras):
+ """Log a warning message.
+
+ This method is used to log a warning message. It can be used to log
+ information about the state of the pipeline and the steps in the
+ pipeline.
+ """
+ self.reporter.logger.warning(
+ message, extra=dict(index=self.index, step_name=self.name, **extras)
+ )
+
+
+class Step:
+ """A `Step` is a unit of work that can be executed in a pipeline.
+
+ Steps are the building blocks of a pipeline. They are responsible for
+ processing records and emitting new records. Steps can be chained together
+ to form a pipeline.
+
+ Steps are asynchronous and can be started, stopped, and run in an
+ asynchronous context. They can process records and emit new records.
+ """
+
+ async def start(self, context: StepContext):
+ """Start the step.
+
+ This method is called when the step is started. It is responsible for
+ setting up the step and preparing it for processing records. This
+ method is called once before any records are processed.
+ """
+ pass
+
+ async def process_record(
+ self, record, context: StepContext
+ ) -> AsyncGenerator[object, None]:
+ """Process a record.
+
+ This method is called for each record that is passed to the step. It
+ should process the record and emit new records.
+ """
+ yield record
+
+ async def emit_outstanding_records(self):
+ """Emit any outstanding records.
+
+ This method is called after all records have been processed. It is
+ responsible for emitting any outstanding records that were not emitted
+ during the processing of records.
+ """
+ for record in ():
+ yield record # pragma: no cover
+
+ async def finish(self, context: StepContext):
+ """Finish the step.
+
+ This method is called when the step is finished. It is responsible for
+ cleaning up the step and releasing any resources that were acquired
+ during the processing of records. This method is called once after all
+ records have been processed.
+ """
pass
class PassStep(Step):
- """A `PassStep` is a step that does nothing."""
+ """A `PassStep` passes records through."""
def __init__(self, **kwargs) -> None:
for key, value in kwargs.items():
setattr(self, key, value)
-
- async def handle_async_record_stream(
- self, record_stream: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
- async for record in record_stream:
- yield record
diff --git a/nodestream/pipeline/transformers/transformer.py b/nodestream/pipeline/transformers/transformer.py
index 90da7448b..31efa5bc9 100644
--- a/nodestream/pipeline/transformers/transformer.py
+++ b/nodestream/pipeline/transformers/transformer.py
@@ -6,30 +6,28 @@
from ..class_loader import ClassLoader
from ..flush import Flush
-from ..step import Step
+from ..step import Step, StepContext
from ..value_providers import ProviderContext, StaticValueOrValueProvider, ValueProvider
class Transformer(Step):
"""A `Transformer` takes a given record and mutates into a new record.
- `Transformer` steps generally make up the middle of an ETL pipeline and are responsible
- for reshaping an object so its more ingestible by the downstream sink.
+ `Transformer` steps generally make up the middle of an ETL pipeline and
+ are responsible for reshaping an object so its more ingestible by the
+ downstream sink.
"""
- async def handle_async_record_stream(
- self, record_stream: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
- async for record in record_stream:
- if record is Flush:
- yield record
+ async def process_record(self, record, _) -> AsyncGenerator[object, None]:
+ if record is Flush:
+ yield record
+ else:
+ val_or_gen = self.transform_record(record)
+ if isinstance(val_or_gen, AsyncGenerator):
+ async for result in val_or_gen:
+ yield result
else:
- val_or_gen = self.transform_record(record)
- if isinstance(val_or_gen, AsyncGenerator):
- async for result in val_or_gen:
- yield result
- else:
- yield await val_or_gen
+ yield await val_or_gen
@abstractmethod
async def transform_record(self, record: Any) -> Any:
@@ -37,15 +35,20 @@ async def transform_record(self, record: Any) -> Any:
class ConcurrentTransformer(Transformer):
- """A `ConcurrentTransformer` takes a given record and mutates into a new record while managing a pool of concurrent workers.
-
- `ConcurrentTransformer`s are useful when the transformation work is IO bound and can be parallelized across multiple threads.
- `ConcurrentTransformer`s are not useful when the transformation work is CPU bound as the GIL will prevent useful parallelization.
- Additionally, `ConcurrentTransformer`s do not guarantee the order of the output stream will match the input stream.
+ """A `ConcurrentTransformer` can process multiple records concurrently.
+
+ `ConcurrentTransformer`s are useful when the transformation work is IO
+ bound and can be parallelized across multiple threads.
+ `ConcurrentTransformer`s are not useful when the transformation work is
+ CPU bound as the GIL will prevent useful parallelization. Additionally,
+ `ConcurrentTransformer`s do not guarantee the order of the output stream
+ will match the input stream.
"""
def __init__(
- self, thread_pool_size: Optional[int] = None, maximum_pending_tasks: int = 1000
+ self,
+ thread_pool_size: Optional[int] = None,
+ maximum_pending_tasks: int = 1000,
) -> None:
self.logger = getLogger(self.__class__.__name__)
self.maximum_pending_tasks = maximum_pending_tasks
@@ -54,83 +57,65 @@ def __init__(
thread_name_prefix=self.__class__.__name__,
initializer=self.on_worker_thread_start,
)
+ self.pending_tasks = []
def on_worker_thread_start(self):
pass
- async def handle_async_record_stream(
- self, record_stream: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
- pending_tasks = []
-
- def drain_completed_tasks():
- tasks_drained = 0
- remaining_pending_tasks = []
- for task in pending_tasks:
- if task.done():
- yield task.result()
- tasks_drained += 1
- else:
- remaining_pending_tasks.append(task)
-
- pending_tasks[:] = remaining_pending_tasks
- if tasks_drained:
- self.logger.debug(
- "Drained %s completed tasks, %s pending tasks remain",
- tasks_drained,
- len(pending_tasks),
- )
-
- async def drain_completed_tasks_async():
- for result in drain_completed_tasks():
- yield result
- await self.yield_processor()
-
- async for record in record_stream:
- if record is Flush:
- # Flush the pending tasks, then yield the flush.
- # In order to fully respect the Flush,
- # we need to wait for all pending tasks to complete.
- # If we're still waiting we can yield the processor.
- while pending_tasks:
- async for result in drain_completed_tasks_async():
- yield result
- yield record
+ async def drain_completed_tasks(self):
+ tasks_drained = 0
+ remaining_pending_tasks = []
+ for task in self.pending_tasks:
+ if task.done():
+ yield task.result()
+ tasks_drained += 1
else:
- # Submit the work to the thread pool (only if we have capacity)
- # If we don't have capacity, yield completed tasks.
- # Once there is nothing remaining to yield from these tasks, yield the processor until capacity is available.
- submitted = False
- while not submitted:
- if len(pending_tasks) < self.maximum_pending_tasks:
- task = self.thread_pool.submit(self.do_work_on_record, record)
- pending_tasks.append(task)
- submitted = True
- async for result in drain_completed_tasks_async():
- yield result
-
- # After we've finished enqueuing records, we need to drain all tasks.
- # Items yielded from this loop are the final results of the step.
- self.logger.debug("Finished enqueuing records, draining pending tasks")
- while pending_tasks:
- async for result in drain_completed_tasks_async():
- yield result
+ remaining_pending_tasks.append(task)
+
+ await self.yield_processor()
+
+ self.pending_tasks[:] = remaining_pending_tasks
+ if tasks_drained:
+ self.logger.debug(
+ "Drained %s completed tasks, %s pending tasks remain",
+ tasks_drained,
+ len(self.pending_tasks),
+ )
+
+ async def process_record(self, record: Any, _) -> Any:
+ if record is Flush:
+ async for outstanding in self.drain_completed_tasks():
+ yield outstanding
+ yield record
+ else:
+ # Submit the work to the thread pool (only if we have capacity)
+ # If we don't have capacity, yield completed tasks.
+ # Once there is nothing remaining to yield from these tasks,
+ # yield the processor until capacity is available.
+ submitted = False
+ while not submitted:
+ if len(self.pending_tasks) < self.maximum_pending_tasks:
+ task = self.thread_pool.submit(self.do_work_on_record, record)
+ self.pending_tasks.append(task)
+ submitted = True
+ async for result in self.drain_completed_tasks():
+ yield result
- async def finish(self):
- self.thread_pool.shutdown(wait=True)
+ def do_work_on_record(self, record: Any) -> Any:
+ return asyncio.run(self.transform_record(record))
async def yield_processor(self):
await asyncio.sleep(0)
- def do_work_on_record(self, record: Any) -> Any:
- # Handles the work nessary to transform a single record.
- # This method wraps the `transform_record` method in an asyncio loop.
- # This is necessary because the thread pool executor is not async.
- return asyncio.run(self.transform_record(record))
+ async def emit_outstanding_records(self):
+ while self.pending_tasks:
+ async for result in self.drain_completed_tasks():
+ yield result
- @abstractmethod
- async def transform_record(self, record: Any) -> Any:
- raise NotImplementedError
+ async def finish(self, _: StepContext):
+ await asyncio.get_event_loop().run_in_executor(
+ None, self.thread_pool.shutdown, True
+ )
class PassTransformer(Transformer):
@@ -172,6 +157,7 @@ async def transform_record(self, record: Any):
context = ProviderContext.fresh(record)
key = self.switch_on.normalize_single_value(context, self.normalization)
transformer = self.transformers.get(key, self.default)
- # If we don't have a transformer to handle the record we just yield the record with PassTransformer
+ # If we don't have a transformer to handle the record we just yield
+ # the record with PassTransformer
async for result in transformer.transform_record(record):
yield result
diff --git a/nodestream/pipeline/writers.py b/nodestream/pipeline/writers.py
index 70703f519..72112a98f 100644
--- a/nodestream/pipeline/writers.py
+++ b/nodestream/pipeline/writers.py
@@ -7,22 +7,19 @@
class Writer(Step):
- """A `Writer` takes a given record and commits it to a downstream data store.
+ """A `Writer` takes a given record and commits it to a downstream store.
- `Writer` steps generally make up the end of an ETL pipeline and are responsible
- for ensuring that the newly transformed data is persisted. After writing the record,
- the record is passed downstream.
+ `Writer` steps generally make up the end of an ETL pipeline and are
+ responsible for ensuring that the newly transformed data is persisted.
+ After writing the record, the record is passed downstream.
"""
- async def handle_async_record_stream(
- self, record_stream: AsyncGenerator[Any, Any]
- ) -> AsyncGenerator[Any, Any]:
- async for record in record_stream:
- if record is Flush:
- await self.flush()
- else:
- await self.write_record(record)
- yield record
+ async def process_record(self, record, _) -> AsyncGenerator[object, None]:
+ if record is Flush:
+ await self.flush()
+ else:
+ await self.write_record(record)
+ yield record
@abstractmethod
async def write_record(self, record: Any):
@@ -31,7 +28,7 @@ async def write_record(self, record: Any):
async def flush(self):
pass
- async def finish(self):
+ async def finish(self, _):
await self.flush()
diff --git a/poetry.lock b/poetry.lock
index c77b5e441..15cbaf67b 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -128,17 +128,17 @@ uvloop = ["uvloop (>=0.15.2)"]
[[package]]
name = "boto3"
-version = "1.34.144"
+version = "1.34.150"
description = "The AWS SDK for Python"
optional = false
python-versions = ">=3.8"
files = [
- {file = "boto3-1.34.144-py3-none-any.whl", hash = "sha256:b8433d481d50b68a0162c0379c0dd4aabfc3d1ad901800beb5b87815997511c1"},
- {file = "boto3-1.34.144.tar.gz", hash = "sha256:2f3e88b10b8fcc5f6100a9d74cd28230edc9d4fa226d99dd40a3ab38ac213673"},
+ {file = "boto3-1.34.150-py3-none-any.whl", hash = "sha256:ad648c89a4935590a69341e5430fc42a021489a22de171ee3fd7bb204f9ef0fa"},
+ {file = "boto3-1.34.150.tar.gz", hash = "sha256:894b222f7850b870a7ac63d7e378ac36c5c34375da24ddc30e131d9fafe369dc"},
]
[package.dependencies]
-botocore = ">=1.34.144,<1.35.0"
+botocore = ">=1.34.150,<1.35.0"
jmespath = ">=0.7.1,<2.0.0"
s3transfer = ">=0.10.0,<0.11.0"
@@ -147,13 +147,13 @@ crt = ["botocore[crt] (>=1.21.0,<2.0a0)"]
[[package]]
name = "botocore"
-version = "1.34.144"
+version = "1.34.150"
description = "Low-level, data-driven core of boto 3."
optional = false
python-versions = ">=3.8"
files = [
- {file = "botocore-1.34.144-py3-none-any.whl", hash = "sha256:a2cf26e1bf10d5917a2285e50257bc44e94a1d16574f282f3274f7a5d8d1f08b"},
- {file = "botocore-1.34.144.tar.gz", hash = "sha256:4215db28d25309d59c99507f1f77df9089e5bebbad35f6e19c7c44ec5383a3e8"},
+ {file = "botocore-1.34.150-py3-none-any.whl", hash = "sha256:b988d47f4d502df85befce11a48002421e4e6ea4289997b5e0261bac5fa76ce6"},
+ {file = "botocore-1.34.150.tar.gz", hash = "sha256:4d23387e0f076d87b637a2a35c0ff2b8daca16eace36b63ce27f65630c6b375a"},
]
[package.dependencies]
@@ -541,43 +541,38 @@ files = [
[[package]]
name = "cryptography"
-version = "42.0.8"
+version = "43.0.0"
description = "cryptography is a package which provides cryptographic recipes and primitives to Python developers."
optional = false
python-versions = ">=3.7"
files = [
- {file = "cryptography-42.0.8-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:81d8a521705787afe7a18d5bfb47ea9d9cc068206270aad0b96a725022e18d2e"},
- {file = "cryptography-42.0.8-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:961e61cefdcb06e0c6d7e3a1b22ebe8b996eb2bf50614e89384be54c48c6b63d"},
- {file = "cryptography-42.0.8-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e3ec3672626e1b9e55afd0df6d774ff0e953452886e06e0f1eb7eb0c832e8902"},
- {file = "cryptography-42.0.8-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e599b53fd95357d92304510fb7bda8523ed1f79ca98dce2f43c115950aa78801"},
- {file = "cryptography-42.0.8-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:5226d5d21ab681f432a9c1cf8b658c0cb02533eece706b155e5fbd8a0cdd3949"},
- {file = "cryptography-42.0.8-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:6b7c4f03ce01afd3b76cf69a5455caa9cfa3de8c8f493e0d3ab7d20611c8dae9"},
- {file = "cryptography-42.0.8-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:2346b911eb349ab547076f47f2e035fc8ff2c02380a7cbbf8d87114fa0f1c583"},
- {file = "cryptography-42.0.8-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:ad803773e9df0b92e0a817d22fd8a3675493f690b96130a5e24f1b8fabbea9c7"},
- {file = "cryptography-42.0.8-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:2f66d9cd9147ee495a8374a45ca445819f8929a3efcd2e3df6428e46c3cbb10b"},
- {file = "cryptography-42.0.8-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:d45b940883a03e19e944456a558b67a41160e367a719833c53de6911cabba2b7"},
- {file = "cryptography-42.0.8-cp37-abi3-win32.whl", hash = "sha256:a0c5b2b0585b6af82d7e385f55a8bc568abff8923af147ee3c07bd8b42cda8b2"},
- {file = "cryptography-42.0.8-cp37-abi3-win_amd64.whl", hash = "sha256:57080dee41209e556a9a4ce60d229244f7a66ef52750f813bfbe18959770cfba"},
- {file = "cryptography-42.0.8-cp39-abi3-macosx_10_12_universal2.whl", hash = "sha256:dea567d1b0e8bc5764b9443858b673b734100c2871dc93163f58c46a97a83d28"},
- {file = "cryptography-42.0.8-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c4783183f7cb757b73b2ae9aed6599b96338eb957233c58ca8f49a49cc32fd5e"},
- {file = "cryptography-42.0.8-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a0608251135d0e03111152e41f0cc2392d1e74e35703960d4190b2e0f4ca9c70"},
- {file = "cryptography-42.0.8-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:dc0fdf6787f37b1c6b08e6dfc892d9d068b5bdb671198c72072828b80bd5fe4c"},
- {file = "cryptography-42.0.8-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:9c0c1716c8447ee7dbf08d6db2e5c41c688544c61074b54fc4564196f55c25a7"},
- {file = "cryptography-42.0.8-cp39-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:fff12c88a672ab9c9c1cf7b0c80e3ad9e2ebd9d828d955c126be4fd3e5578c9e"},
- {file = "cryptography-42.0.8-cp39-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:cafb92b2bc622cd1aa6a1dce4b93307792633f4c5fe1f46c6b97cf67073ec961"},
- {file = "cryptography-42.0.8-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:31f721658a29331f895a5a54e7e82075554ccfb8b163a18719d342f5ffe5ecb1"},
- {file = "cryptography-42.0.8-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:b297f90c5723d04bcc8265fc2a0f86d4ea2e0f7ab4b6994459548d3a6b992a14"},
- {file = "cryptography-42.0.8-cp39-abi3-win32.whl", hash = "sha256:2f88d197e66c65be5e42cd72e5c18afbfae3f741742070e3019ac8f4ac57262c"},
- {file = "cryptography-42.0.8-cp39-abi3-win_amd64.whl", hash = "sha256:fa76fbb7596cc5839320000cdd5d0955313696d9511debab7ee7278fc8b5c84a"},
- {file = "cryptography-42.0.8-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:ba4f0a211697362e89ad822e667d8d340b4d8d55fae72cdd619389fb5912eefe"},
- {file = "cryptography-42.0.8-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:81884c4d096c272f00aeb1f11cf62ccd39763581645b0812e99a91505fa48e0c"},
- {file = "cryptography-42.0.8-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:c9bb2ae11bfbab395bdd072985abde58ea9860ed84e59dbc0463a5d0159f5b71"},
- {file = "cryptography-42.0.8-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:7016f837e15b0a1c119d27ecd89b3515f01f90a8615ed5e9427e30d9cdbfed3d"},
- {file = "cryptography-42.0.8-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:5a94eccb2a81a309806027e1670a358b99b8fe8bfe9f8d329f27d72c094dde8c"},
- {file = "cryptography-42.0.8-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:dec9b018df185f08483f294cae6ccac29e7a6e0678996587363dc352dc65c842"},
- {file = "cryptography-42.0.8-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:343728aac38decfdeecf55ecab3264b015be68fc2816ca800db649607aeee648"},
- {file = "cryptography-42.0.8-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:013629ae70b40af70c9a7a5db40abe5d9054e6f4380e50ce769947b73bf3caad"},
- {file = "cryptography-42.0.8.tar.gz", hash = "sha256:8d09d05439ce7baa8e9e95b07ec5b6c886f548deb7e0f69ef25f64b3bce842f2"},
+ {file = "cryptography-43.0.0-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:64c3f16e2a4fc51c0d06af28441881f98c5d91009b8caaff40cf3548089e9c74"},
+ {file = "cryptography-43.0.0-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3dcdedae5c7710b9f97ac6bba7e1052b95c7083c9d0e9df96e02a1932e777895"},
+ {file = "cryptography-43.0.0-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3d9a1eca329405219b605fac09ecfc09ac09e595d6def650a437523fcd08dd22"},
+ {file = "cryptography-43.0.0-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:ea9e57f8ea880eeea38ab5abf9fbe39f923544d7884228ec67d666abd60f5a47"},
+ {file = "cryptography-43.0.0-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:9a8d6802e0825767476f62aafed40532bd435e8a5f7d23bd8b4f5fd04cc80ecf"},
+ {file = "cryptography-43.0.0-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:cc70b4b581f28d0a254d006f26949245e3657d40d8857066c2ae22a61222ef55"},
+ {file = "cryptography-43.0.0-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:4a997df8c1c2aae1e1e5ac49c2e4f610ad037fc5a3aadc7b64e39dea42249431"},
+ {file = "cryptography-43.0.0-cp37-abi3-win32.whl", hash = "sha256:6e2b11c55d260d03a8cf29ac9b5e0608d35f08077d8c087be96287f43af3ccdc"},
+ {file = "cryptography-43.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:31e44a986ceccec3d0498e16f3d27b2ee5fdf69ce2ab89b52eaad1d2f33d8778"},
+ {file = "cryptography-43.0.0-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:7b3f5fe74a5ca32d4d0f302ffe6680fcc5c28f8ef0dc0ae8f40c0f3a1b4fca66"},
+ {file = "cryptography-43.0.0-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ac1955ce000cb29ab40def14fd1bbfa7af2017cca696ee696925615cafd0dce5"},
+ {file = "cryptography-43.0.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:299d3da8e00b7e2b54bb02ef58d73cd5f55fb31f33ebbf33bd00d9aa6807df7e"},
+ {file = "cryptography-43.0.0-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:ee0c405832ade84d4de74b9029bedb7b31200600fa524d218fc29bfa371e97f5"},
+ {file = "cryptography-43.0.0-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:cb013933d4c127349b3948aa8aaf2f12c0353ad0eccd715ca789c8a0f671646f"},
+ {file = "cryptography-43.0.0-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:fdcb265de28585de5b859ae13e3846a8e805268a823a12a4da2597f1f5afc9f0"},
+ {file = "cryptography-43.0.0-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:2905ccf93a8a2a416f3ec01b1a7911c3fe4073ef35640e7ee5296754e30b762b"},
+ {file = "cryptography-43.0.0-cp39-abi3-win32.whl", hash = "sha256:47ca71115e545954e6c1d207dd13461ab81f4eccfcb1345eac874828b5e3eaaf"},
+ {file = "cryptography-43.0.0-cp39-abi3-win_amd64.whl", hash = "sha256:0663585d02f76929792470451a5ba64424acc3cd5227b03921dab0e2f27b1709"},
+ {file = "cryptography-43.0.0-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:2c6d112bf61c5ef44042c253e4859b3cbbb50df2f78fa8fae6747a7814484a70"},
+ {file = "cryptography-43.0.0-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:844b6d608374e7d08f4f6e6f9f7b951f9256db41421917dfb2d003dde4cd6b66"},
+ {file = "cryptography-43.0.0-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:51956cf8730665e2bdf8ddb8da0056f699c1a5715648c1b0144670c1ba00b48f"},
+ {file = "cryptography-43.0.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:aae4d918f6b180a8ab8bf6511a419473d107df4dbb4225c7b48c5c9602c38c7f"},
+ {file = "cryptography-43.0.0-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:232ce02943a579095a339ac4b390fbbe97f5b5d5d107f8a08260ea2768be8cc2"},
+ {file = "cryptography-43.0.0-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:5bcb8a5620008a8034d39bce21dc3e23735dfdb6a33a06974739bfa04f853947"},
+ {file = "cryptography-43.0.0-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:08a24a7070b2b6804c1940ff0f910ff728932a9d0e80e7814234269f9d46d069"},
+ {file = "cryptography-43.0.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:e9c5266c432a1e23738d178e51c2c7a5e2ddf790f248be939448c0ba2021f9d1"},
+ {file = "cryptography-43.0.0.tar.gz", hash = "sha256:b88075ada2d51aa9f18283532c9f60e72170041bba88d7f37e49cbb10275299e"},
]
[package.dependencies]
@@ -590,7 +585,7 @@ nox = ["nox"]
pep8test = ["check-sdist", "click", "mypy", "ruff"]
sdist = ["build"]
ssh = ["bcrypt (>=3.1.5)"]
-test = ["certifi", "pretend", "pytest (>=6.2.0)", "pytest-benchmark", "pytest-cov", "pytest-xdist"]
+test = ["certifi", "cryptography-vectors (==43.0.0)", "pretend", "pytest (>=6.2.0)", "pytest-benchmark", "pytest-cov", "pytest-xdist"]
test-randomorder = ["pytest-randomly"]
[[package]]
@@ -1001,13 +996,13 @@ pyyaml = ">=5.1"
[[package]]
name = "mkdocs-material"
-version = "9.5.29"
+version = "9.5.30"
description = "Documentation that simply works"
optional = false
python-versions = ">=3.8"
files = [
- {file = "mkdocs_material-9.5.29-py3-none-any.whl", hash = "sha256:afc1f508e2662ded95f0a35a329e8a5acd73ee88ca07ba73836eb6fcdae5d8b4"},
- {file = "mkdocs_material-9.5.29.tar.gz", hash = "sha256:3e977598ec15a4ddad5c4dfc9e08edab6023edb51e88f0729bd27be77e3d322a"},
+ {file = "mkdocs_material-9.5.30-py3-none-any.whl", hash = "sha256:fc070689c5250a180e9b9d79d8491ef9a3a7acb240db0728728d6c31eeb131d4"},
+ {file = "mkdocs_material-9.5.30.tar.gz", hash = "sha256:3fd417dd42d679e3ba08b9e2d72cd8b8af142cc4a3969676ad6b00993dd182ec"},
]
[package.dependencies]
@@ -1451,13 +1446,13 @@ tests-numpy = ["numpy", "pyhamcrest[tests]"]
[[package]]
name = "pymdown-extensions"
-version = "10.8.1"
+version = "10.9"
description = "Extension pack for Python Markdown."
optional = false
python-versions = ">=3.8"
files = [
- {file = "pymdown_extensions-10.8.1-py3-none-any.whl", hash = "sha256:f938326115884f48c6059c67377c46cf631c733ef3629b6eed1349989d1b30cb"},
- {file = "pymdown_extensions-10.8.1.tar.gz", hash = "sha256:3ab1db5c9e21728dabf75192d71471f8e50f216627e9a1fa9535ecb0231b9940"},
+ {file = "pymdown_extensions-10.9-py3-none-any.whl", hash = "sha256:d323f7e90d83c86113ee78f3fe62fc9dee5f56b54d912660703ea1816fed5626"},
+ {file = "pymdown_extensions-10.9.tar.gz", hash = "sha256:6ff740bcd99ec4172a938970d42b96128bdc9d4b9bcad72494f29921dc69b753"},
]
[package.dependencies]
@@ -1703,104 +1698,104 @@ pyyaml = "*"
[[package]]
name = "rapidfuzz"
-version = "3.9.4"
+version = "3.9.5"
description = "rapid fuzzy string matching"
optional = false
python-versions = ">=3.8"
files = [
- {file = "rapidfuzz-3.9.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c9b9793c19bdf38656c8eaefbcf4549d798572dadd70581379e666035c9df781"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:015b5080b999404fe06ec2cb4f40b0be62f0710c926ab41e82dfbc28e80675b4"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:acc5ceca9c1e1663f3e6c23fb89a311f69b7615a40ddd7645e3435bf3082688a"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1424e238bc3f20e1759db1e0afb48a988a9ece183724bef91ea2a291c0b92a95"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ed01378f605aa1f449bee82cd9c83772883120d6483e90aa6c5a4ce95dc5c3aa"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:eb26d412271e5a76cdee1c2d6bf9881310665d3fe43b882d0ed24edfcb891a84"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f37e9e1f17be193c41a31c864ad4cd3ebd2b40780db11cd5c04abf2bcf4201b"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:d070ec5cf96b927c4dc5133c598c7ff6db3b833b363b2919b13417f1002560bc"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:10e61bb7bc807968cef09a0e32ce253711a2d450a4dce7841d21d45330ffdb24"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:31a2fc60bb2c7face4140010a7aeeafed18b4f9cdfa495cc644a68a8c60d1ff7"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:fbebf1791a71a2e89f5c12b78abddc018354d5859e305ec3372fdae14f80a826"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:aee9fc9e3bb488d040afc590c0a7904597bf4ccd50d1491c3f4a5e7e67e6cd2c"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-win32.whl", hash = "sha256:005a02688a51c7d2451a2d41c79d737aa326ff54167211b78a383fc2aace2c2c"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-win_amd64.whl", hash = "sha256:3a2e75e41ee3274754d3b2163cc6c82cd95b892a85ab031f57112e09da36455f"},
- {file = "rapidfuzz-3.9.4-cp310-cp310-win_arm64.whl", hash = "sha256:2c99d355f37f2b289e978e761f2f8efeedc2b14f4751d9ff7ee344a9a5ca98d9"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:07141aa6099e39d48637ce72a25b893fc1e433c50b3e837c75d8edf99e0c63e1"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:db1664eaff5d7d0f2542dd9c25d272478deaf2c8412e4ad93770e2e2d828e175"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bc01a223f6605737bec3202e94dcb1a449b6c76d46082cfc4aa980f2a60fd40e"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1869c42e73e2a8910b479be204fa736418741b63ea2325f9cc583c30f2ded41a"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:62ea7007941fb2795fff305ac858f3521ec694c829d5126e8f52a3e92ae75526"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:698e992436bf7f0afc750690c301215a36ff952a6dcd62882ec13b9a1ebf7a39"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b76f611935f15a209d3730c360c56b6df8911a9e81e6a38022efbfb96e433bab"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:129627d730db2e11f76169344a032f4e3883d34f20829419916df31d6d1338b1"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:90a82143c14e9a14b723a118c9ef8d1bbc0c5a16b1ac622a1e6c916caff44dd8"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:ded58612fe3b0e0d06e935eaeaf5a9fd27da8ba9ed3e2596307f40351923bf72"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:f16f5d1c4f02fab18366f2d703391fcdbd87c944ea10736ca1dc3d70d8bd2d8b"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:26aa7eece23e0df55fb75fbc2a8fb678322e07c77d1fd0e9540496e6e2b5f03e"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-win32.whl", hash = "sha256:f187a9c3b940ce1ee324710626daf72c05599946bd6748abe9e289f1daa9a077"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-win_amd64.whl", hash = "sha256:d8e9130fe5d7c9182990b366ad78fd632f744097e753e08ace573877d67c32f8"},
- {file = "rapidfuzz-3.9.4-cp311-cp311-win_arm64.whl", hash = "sha256:40419e98b10cd6a00ce26e4837a67362f658fc3cd7a71bd8bd25c99f7ee8fea5"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:b5d5072b548db1b313a07d62d88fe0b037bd2783c16607c647e01b070f6cf9e5"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:cf5bcf22e1f0fd273354462631d443ef78d677f7d2fc292de2aec72ae1473e66"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0c8fc973adde8ed52810f590410e03fb6f0b541bbaeb04c38d77e63442b2df4c"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f2464bb120f135293e9a712e342c43695d3d83168907df05f8c4ead1612310c7"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8d9d58689aca22057cf1a5851677b8a3ccc9b535ca008c7ed06dc6e1899f7844"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:167e745f98baa0f3034c13583e6302fb69249a01239f1483d68c27abb841e0a1"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:db0bf0663b4b6da1507869722420ea9356b6195aa907228d6201303e69837af9"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:cd6ac61b74fdb9e23f04d5f068e6cf554f47e77228ca28aa2347a6ca8903972f"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:60ff67c690acecf381759c16cb06c878328fe2361ddf77b25d0e434ea48a29da"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:cb934363380c60f3a57d14af94325125cd8cded9822611a9f78220444034e36e"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:fe833493fb5cc5682c823ea3e2f7066b07612ee8f61ecdf03e1268f262106cdd"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:2797fb847d89e04040d281cb1902cbeffbc4b5131a5c53fc0db490fd76b2a547"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-win32.whl", hash = "sha256:52e3d89377744dae68ed7c84ad0ddd3f5e891c82d48d26423b9e066fc835cc7c"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-win_amd64.whl", hash = "sha256:c76da20481c906e08400ee9be230f9e611d5931a33707d9df40337c2655c84b5"},
- {file = "rapidfuzz-3.9.4-cp312-cp312-win_arm64.whl", hash = "sha256:f2d2846f3980445864c7e8b8818a29707fcaff2f0261159ef6b7bd27ba139296"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:355fc4a268ffa07bab88d9adee173783ec8d20136059e028d2a9135c623c44e6"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:4d81a78f90269190b568a8353d4ea86015289c36d7e525cd4d43176c88eff429"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9e618625ffc4660b26dc8e56225f8b966d5842fa190e70c60db6cd393e25b86e"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b712336ad6f2bacdbc9f1452556e8942269ef71f60a9e6883ef1726b52d9228a"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:84fc1ee19fdad05770c897e793836c002344524301501d71ef2e832847425707"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1950f8597890c0c707cb7e0416c62a1cf03dcdb0384bc0b2dbda7e05efe738ec"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4a6c35f272ec9c430568dc8c1c30cb873f6bc96be2c79795e0bce6db4e0e101d"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:1df0f9e9239132a231c86ae4f545ec2b55409fa44470692fcfb36b1bd00157ad"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:d2c51955329bfccf99ae26f63d5928bf5be9fcfcd9f458f6847fd4b7e2b8986c"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:3c522f462d9fc504f2ea8d82e44aa580e60566acc754422c829ad75c752fbf8d"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:d8a52fc50ded60d81117d7647f262c529659fb21d23e14ebfd0b35efa4f1b83d"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:04dbdfb0f0bfd3f99cf1e9e24fadc6ded2736d7933f32f1151b0f2abb38f9a25"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-win32.whl", hash = "sha256:4968c8bd1df84b42f382549e6226710ad3476f976389839168db3e68fd373298"},
- {file = "rapidfuzz-3.9.4-cp38-cp38-win_amd64.whl", hash = "sha256:3fe4545f89f8d6c27b6bbbabfe40839624873c08bd6700f63ac36970a179f8f5"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9f256c8fb8f3125574c8c0c919ab0a1f75d7cba4d053dda2e762dcc36357969d"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f5fdc09cf6e9d8eac3ce48a4615b3a3ee332ea84ac9657dbbefef913b13e632f"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d395d46b80063d3b5d13c0af43d2c2cedf3ab48c6a0c2aeec715aa5455b0c632"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7fa714fb96ce9e70c37e64c83b62fe8307030081a0bfae74a76fac7ba0f91715"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1bc1a0f29f9119be7a8d3c720f1d2068317ae532e39e4f7f948607c3a6de8396"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6022674aa1747d6300f699cd7c54d7dae89bfe1f84556de699c4ac5df0838082"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dcb72e5f9762fd469701a7e12e94b924af9004954f8c739f925cb19c00862e38"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:ad04ae301129f0eb5b350a333accd375ce155a0c1cec85ab0ec01f770214e2e4"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:f46a22506f17c0433e349f2d1dc11907c393d9b3601b91d4e334fa9a439a6a4d"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:01b42a8728c36011718da409aa86b84984396bf0ca3bfb6e62624f2014f6022c"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:e590d5d5443cf56f83a51d3c4867bd1f6be8ef8cfcc44279522bcef3845b2a51"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:4c72078b5fdce34ba5753f9299ae304e282420e6455e043ad08e4488ca13a2b0"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-win32.whl", hash = "sha256:f75639277304e9b75e6a7b3c07042d2264e16740a11e449645689ed28e9c2124"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-win_amd64.whl", hash = "sha256:e81e27e8c32a1e1278a4bb1ce31401bfaa8c2cc697a053b985a6f8d013df83ec"},
- {file = "rapidfuzz-3.9.4-cp39-cp39-win_arm64.whl", hash = "sha256:15bc397ee9a3ed1210b629b9f5f1da809244adc51ce620c504138c6e7095b7bd"},
- {file = "rapidfuzz-3.9.4-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:20488ade4e1ddba3cfad04f400da7a9c1b91eff5b7bd3d1c50b385d78b587f4f"},
- {file = "rapidfuzz-3.9.4-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:e61b03509b1a6eb31bc5582694f6df837d340535da7eba7bedb8ae42a2fcd0b9"},
- {file = "rapidfuzz-3.9.4-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:098d231d4e51644d421a641f4a5f2f151f856f53c252b03516e01389b2bfef99"},
- {file = "rapidfuzz-3.9.4-pp310-pypy310_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:17ab8b7d10fde8dd763ad428aa961c0f30a1b44426e675186af8903b5d134fb0"},
- {file = "rapidfuzz-3.9.4-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e272df61bee0a056a3daf99f9b1bd82cf73ace7d668894788139c868fdf37d6f"},
- {file = "rapidfuzz-3.9.4-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:d6481e099ff8c4edda85b8b9b5174c200540fd23c8f38120016c765a86fa01f5"},
- {file = "rapidfuzz-3.9.4-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:ad61676e9bdae677d577fe80ec1c2cea1d150c86be647e652551dcfe505b1113"},
- {file = "rapidfuzz-3.9.4-pp38-pypy38_pp73-macosx_11_0_arm64.whl", hash = "sha256:af65020c0dd48d0d8ae405e7e69b9d8ae306eb9b6249ca8bf511a13f465fad85"},
- {file = "rapidfuzz-3.9.4-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4d38b4e026fcd580e0bda6c0ae941e0e9a52c6bc66cdce0b8b0da61e1959f5f8"},
- {file = "rapidfuzz-3.9.4-pp38-pypy38_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f74ed072c2b9dc6743fb19994319d443a4330b0e64aeba0aa9105406c7c5b9c2"},
- {file = "rapidfuzz-3.9.4-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:aee5f6b8321f90615c184bd8a4c676e9becda69b8e4e451a90923db719d6857c"},
- {file = "rapidfuzz-3.9.4-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:3a555e3c841d6efa350f862204bb0a3fea0c006b8acc9b152b374fa36518a1c6"},
- {file = "rapidfuzz-3.9.4-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:0772150d37bf018110351c01d032bf9ab25127b966a29830faa8ad69b7e2f651"},
- {file = "rapidfuzz-3.9.4-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:addcdd3c3deef1bd54075bd7aba0a6ea9f1d01764a08620074b7a7b1e5447cb9"},
- {file = "rapidfuzz-3.9.4-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3fe86b82b776554add8f900b6af202b74eb5efe8f25acdb8680a5c977608727f"},
- {file = "rapidfuzz-3.9.4-pp39-pypy39_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b0fc91ac59f4414d8542454dfd6287a154b8e6f1256718c898f695bdbb993467"},
- {file = "rapidfuzz-3.9.4-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a944e546a296a5fdcaabb537b01459f1b14d66f74e584cb2a91448bffadc3c1"},
- {file = "rapidfuzz-3.9.4-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:4fb96ba96d58c668a17a06b5b5e8340fedc26188e87b0d229d38104556f30cd8"},
- {file = "rapidfuzz-3.9.4.tar.gz", hash = "sha256:366bf8947b84e37f2f4cf31aaf5f37c39f620d8c0eddb8b633e6ba0129ca4a0a"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:7659058863d84a2c36c5a76c28bc8713d33eab03e677e67260d9e1cca43fc3bb"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:802a018776bd3cb7c5d23ba38ebbb1663a9f742be1d58e73b62d8c7cace6e607"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da71e8fdb0d1a21f4b58b2c84bcbc2b89a472c073c5f7bdb9339f4cb3122c0e3"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f9433cb12731167b358fbcff9828d2294429986a03222031f6d14308eb643c77"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3e33e1d185206730b916b3e7d9bce1941c65b2a1488cdd0457ae21be385a7912"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:758719e9613c47a274768f1926460955223fe0a03e7eda264f2b78b1b97a4743"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7981cc6240d01d4480795d758ea2ee748257771f68127d630045e58fe1b5545a"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:b6cdca86120c3f9aa069f8d4e1c5422e92f833d705d719a2ba7082412f4c933b"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:ffa533acb1a9dcb6e26c4467fdc1347995fb168ec9f794b97545f6b72dee733c"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:13eeaeb0d5fe00fa99336f73fb5ab65c46109c7121cf87659b9601908b8b6178"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:d7b1922b1403ccb3583218e8cd931b08e04c5442ca03dbaf6ea4fcf574ee2b24"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:b0189f691cea4dc9fe074ea6b97da30a91d0882fa69724b4b34b51d2c1983473"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-win32.whl", hash = "sha256:72e466e5de12a327a09ed6d0116f024759b5146b335645c32241da84134a7f34"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-win_amd64.whl", hash = "sha256:345011cfcafaa3674c46673baad67d2394eb58285530d8333e65c3c9a143b4f4"},
+ {file = "rapidfuzz-3.9.5-cp310-cp310-win_arm64.whl", hash = "sha256:5dc19c8222475e4f7f528b94d2fa28e7979355c5cf7c6e73902d2abb2be96522"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6c741972d64031535cfd76d89cf47259e590e822353be57ec2f5d56758c98296"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a7452d079800cf70a7314f73044f03cbcbd90a651d9dec39443d2a8a2b63ab53"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f06f163a0341bad162e972590b73e17f9cea2ed8ee27b193875ccbc3dd6eca2f"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:529e2cf441746bd492f6c36a38bf9fa6a418df95b9c003f8e92a55d8a979bd9c"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9811a741aa1350ad36689d675ded8b34e423e68b396bd30bff751a9c582f586e"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9e36c4640a789b8c922b69a548968939d1c0433fa7aac83cb08e1334d4e5d7de"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:53fb2f32f14c921d2f673c5b7cd58d4cc626c574a28c0791f283880d8e57022c"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:031806eb035a6f09f4ff23b9d971d50b30b5e93aa3ee620c920bee1dc32827e7"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:f6dbe1df0b9334e3cf07445d810c81734ae23d137b5efc69e1d676ff55691351"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:24345826b50aafcea26e2e4be5c103d96fe9d7fc549ac9190641300290958f3b"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:bfd3b66ee1f0ebb40c672a7a7e5bda00fb763fa9bca082058084175151f8e685"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:a6f1df5b0e602e94199cccb5e241bbc2319644003e34f077741ebf48aea7ed1a"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-win32.whl", hash = "sha256:f080d6709f51a8335e73826b96af9b4e3657631eca6c69e1ac501868dcc84b7f"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-win_amd64.whl", hash = "sha256:bf9ed6988da6a2c1f8df367cb5d6be26a3d8543646c8eba79741ac9e764fbc59"},
+ {file = "rapidfuzz-3.9.5-cp311-cp311-win_arm64.whl", hash = "sha256:599714790dfac0a23a473134e6677d0a103690a4e21ba189cfc826e322cdc8d5"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:9729852038fb2de096e249899f8a9bee90fb1f92e10b6ccc539d5bb798c703bc"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:9dc39435476fb3b3b3c24ab2c08c726056b2b487aa7ee450aee698b808c808ac"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d6ceea632b0eb97dac54411c29feb190054e91fd0571f585b56e4a9159c55ab0"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cadd66e6ef9901909dc1b11db91048f1bf4613ba7d773386f922e28b1e1df4da"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:63e34fb3586431589a5e1cd7fc61c6f057576c6c6804c1c673bac3de0516dee7"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:181073256faec68e6b8ab3329a36cfa1360f7906aa70d9aee4a39cb70889f73f"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8419c18bbbd67058ca1312f35acda2e4e4592650f105cfd166569a2ebccd01f1"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:191d1057cca56641f7b919fe712cb7e48cd226342e097a78136127f8bde32caa"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:fe5a11eefd0ae90d32d9ff706a894498b4efb4b0c263ad9d1e6401050863504d"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:e1b024d9d69bb83e125adee4162991f2764f16acc3fb1ed0f0fc1ad5aeb7e394"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:7d5a34b8388ae99bdbd5a3646f45ac318f4c870105bdbe42a2f4c85e5b347761"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:0e09abc0d397019bba61c8e6dfe2ec863d4dfb1762f51c9197ce0af5d5fd9adb"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-win32.whl", hash = "sha256:e3c4be3057472c79ba6f4eab35daa9f12908cb697c472d05fbbd47949a87aec6"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-win_amd64.whl", hash = "sha256:0d9fdb74df87018dd4146f3d00df9fca2c27f060936a9e8d3015e7bfb9cb69e4"},
+ {file = "rapidfuzz-3.9.5-cp312-cp312-win_arm64.whl", hash = "sha256:491d3d425b5fe3f61f3b9a70abfd498ce9139d94956db7a8551e537e017c0e57"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:518dec750a30f115ba1299ef2547cf468a69f310581a030c8a875257de747c5f"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:252dc3d1c3d613b8db1b59d13381937e420c99f8a351ffa0e78c2f54746e107f"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ebd17688b75b6fa983e8586cad30f36eb9736b860946cc8b633b9442c9481831"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e8032492021b0aa55a623d6f6e739a5d4aaabc32af379c2a5656bf1e9e178bf1"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:73362eb1c3d02f32e4c7f0d77eb284e9a13f278cff224f71e8f60e2aff5b6a5d"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a42d1f7b8988f50013e703ed27b5e216ef8a725b2f4ac53754ad0476020b26f4"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d4f2e985172bb76c9179e11fb67d9c9ecbee4933740eca2977797094df02498d"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:8e943c5cbd10e15369be1f371ef303cb413c1008f64d93bd13762ea06ca84d59"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:0d34b0e8e29f80cb2ac8afe8fb7b01a542b136ffbf7e2b9983d11bce49398f68"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:62b8f9f58e9dffaa86fef84db2705457a58e191a962124f2b815026ba79d9aba"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:ebf682bdb0f01b6b1f9a9ffe918aa3ac84fbdadb998ffbfcd5f9b12bd280170f"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:3ed0c17e5b6fdd2ac3230bdefa908579971377c36aa4a2f132700fa8145040db"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-win32.whl", hash = "sha256:ac460d89b9759e37eef23fed05184179654882a241f6b2363df194f8940cc55f"},
+ {file = "rapidfuzz-3.9.5-cp38-cp38-win_amd64.whl", hash = "sha256:cf9aceb4227fd09f9a20e505f78487b2089d6420ce232d288522ea0a78b986b9"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:14587df847d0d50bd10cde0a198b5d64eedb7484c72b825f5c2ead6e6ff16eee"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:fd94d952299ec73ea63a0fa4b699a2750785b6bb82aa56fd886d9023b86f90ab"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:733bf3d7876bf6d8167e6436f99d6ea16a218ec2c8eb9da6048f20b9cc8733e2"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fb28f2b7173ed3678b4630b0c8b21503087d1cd082bae200dc2519ca38b26686"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80a4c8a2c5ae4b133fec6b5db1af9a4126ffa6eca18a558fe5b6ab8e330d3d78"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5feb75e905281e5c669e21c98d594acc3b222a8694d9342f17df988766d83748"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d047b01637a31d9bf776b66438f574fd1db856ad14cf296c1f48bb6bef8a5aff"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:d9e0a656274ac75ec24499a06c0bc5eee67bcd8276c6061da7c05d549f1b1a61"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:16c982dd3cdd33cf4aac91027a263a081d1a8050dc33a27470367a391a8d1576"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:9a0c878d0980508e90e973a9cbfb591acc370085f2301c6aacadbd8362d52a36"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:1d9bcfec5efd55b6268328cccd12956d833582d8da6385231a5c6c6201a1156a"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:8171fc6e4645e636161a9ef5b44b20605adbefe23cd990b68d72cae0b9c12509"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-win32.whl", hash = "sha256:35088e759b083398ab3c4154517476e116653b7403604677af9a894179f1042f"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-win_amd64.whl", hash = "sha256:6d8cc7e6e5c6fbcacdfe3cf7a86b60dcaf216216d86e6879ff52d488e5b11e27"},
+ {file = "rapidfuzz-3.9.5-cp39-cp39-win_arm64.whl", hash = "sha256:506547889f18db0acca787ffb9f287757cbfe9f0fadddd4e07c64ce0bd924e13"},
+ {file = "rapidfuzz-3.9.5-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:f4e0122603af2119579e9f94e172c6e460860fdcdb713164332c1951c13df999"},
+ {file = "rapidfuzz-3.9.5-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:e46cd486289d1d8e3dab779c725f5dde77b286185d32e7b874bfc3d161e3a927"},
+ {file = "rapidfuzz-3.9.5-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7e2c0c8bbe4f4525009e3ad9b94a39cdff5d6378233e754d0b13c29cdfaa75fc"},
+ {file = "rapidfuzz-3.9.5-pp310-pypy310_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:bfb47513a17c935f6ee606dcae0ea9d20a3fb0fe9ca597758472ea08be62dc54"},
+ {file = "rapidfuzz-3.9.5-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:976ed1105a76935b6a4d2bbc7d577be1b97b43997bcec2f29a0ab48ff6f5d6b1"},
+ {file = "rapidfuzz-3.9.5-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:9cf2028edb9ccd21d1d5aaacef2fe3e14bee4343df1c2c0f7373ef6e81013bef"},
+ {file = "rapidfuzz-3.9.5-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:926701c8e61319ee2e4888619143f58ddcc0e3e886668269b8e053f2d68c1e92"},
+ {file = "rapidfuzz-3.9.5-pp38-pypy38_pp73-macosx_11_0_arm64.whl", hash = "sha256:99eaa8dd8a44664813e0bef014775993ef75a134a863bc54cd855a60622203fd"},
+ {file = "rapidfuzz-3.9.5-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7508ef727ef4891141dd3ac7a39a2327384ece070521ac9c58f06c27d57c72d5"},
+ {file = "rapidfuzz-3.9.5-pp38-pypy38_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f9f33d05db5bba1d076446c51347a6d93ff24d8f9d01b0b8b15ca8ec8b1ef382"},
+ {file = "rapidfuzz-3.9.5-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7252666b85c931d51a59d5308bb6827a67434917ef510747d3ce7e88ec17e7f2"},
+ {file = "rapidfuzz-3.9.5-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:d26f7299e2872d18fb7df1bc043e53aa94fc5a4a2a6a9537ad8707579fcb1668"},
+ {file = "rapidfuzz-3.9.5-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:2b17ecc17322b659962234799e90054e420911b8ca510a7869c2f4419f9f3ecb"},
+ {file = "rapidfuzz-3.9.5-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:f3e037b9ec621dec0157d81566e7d47a91405e379335cf8f4ed3c20d61db91d8"},
+ {file = "rapidfuzz-3.9.5-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:42c4d1ba2647c8d2a82313c4dde332de750c936b94f016308339e762c2e5e53d"},
+ {file = "rapidfuzz-3.9.5-pp39-pypy39_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:876e663b11d9067e1096ea76a2de87227c7b513aff2b60667b20417da74183e4"},
+ {file = "rapidfuzz-3.9.5-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:adee55488490375c1604b878fbc1eb1a51fe5e6f5bd05047df2f8c6505a48728"},
+ {file = "rapidfuzz-3.9.5-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:abb1ac683671000bd4ec215a494aba687d75a198db72188408154a19ea313ff4"},
+ {file = "rapidfuzz-3.9.5.tar.gz", hash = "sha256:257f2406a671371bafd99a2a2c57f991783446bc2176b93a83d1d833e35d36df"},
]
[package.extras]
@@ -1808,90 +1803,90 @@ full = ["numpy"]
[[package]]
name = "regex"
-version = "2024.5.15"
+version = "2024.7.24"
description = "Alternative regular expression module, to replace re."
optional = false
python-versions = ">=3.8"
files = [
- {file = "regex-2024.5.15-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:a81e3cfbae20378d75185171587cbf756015ccb14840702944f014e0d93ea09f"},
- {file = "regex-2024.5.15-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:7b59138b219ffa8979013be7bc85bb60c6f7b7575df3d56dc1e403a438c7a3f6"},
- {file = "regex-2024.5.15-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a0bd000c6e266927cb7a1bc39d55be95c4b4f65c5be53e659537537e019232b1"},
- {file = "regex-2024.5.15-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5eaa7ddaf517aa095fa8da0b5015c44d03da83f5bd49c87961e3c997daed0de7"},
- {file = "regex-2024.5.15-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ba68168daedb2c0bab7fd7e00ced5ba90aebf91024dea3c88ad5063c2a562cca"},
- {file = "regex-2024.5.15-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6e8d717bca3a6e2064fc3a08df5cbe366369f4b052dcd21b7416e6d71620dca1"},
- {file = "regex-2024.5.15-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1337b7dbef9b2f71121cdbf1e97e40de33ff114801263b275aafd75303bd62b5"},
- {file = "regex-2024.5.15-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f9ebd0a36102fcad2f03696e8af4ae682793a5d30b46c647eaf280d6cfb32796"},
- {file = "regex-2024.5.15-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:9efa1a32ad3a3ea112224897cdaeb6aa00381627f567179c0314f7b65d354c62"},
- {file = "regex-2024.5.15-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:1595f2d10dff3d805e054ebdc41c124753631b6a471b976963c7b28543cf13b0"},
- {file = "regex-2024.5.15-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b802512f3e1f480f41ab5f2cfc0e2f761f08a1f41092d6718868082fc0d27143"},
- {file = "regex-2024.5.15-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:a0981022dccabca811e8171f913de05720590c915b033b7e601f35ce4ea7019f"},
- {file = "regex-2024.5.15-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:19068a6a79cf99a19ccefa44610491e9ca02c2be3305c7760d3831d38a467a6f"},
- {file = "regex-2024.5.15-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:1b5269484f6126eee5e687785e83c6b60aad7663dafe842b34691157e5083e53"},
- {file = "regex-2024.5.15-cp310-cp310-win32.whl", hash = "sha256:ada150c5adfa8fbcbf321c30c751dc67d2f12f15bd183ffe4ec7cde351d945b3"},
- {file = "regex-2024.5.15-cp310-cp310-win_amd64.whl", hash = "sha256:ac394ff680fc46b97487941f5e6ae49a9f30ea41c6c6804832063f14b2a5a145"},
- {file = "regex-2024.5.15-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:f5b1dff3ad008dccf18e652283f5e5339d70bf8ba7c98bf848ac33db10f7bc7a"},
- {file = "regex-2024.5.15-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c6a2b494a76983df8e3d3feea9b9ffdd558b247e60b92f877f93a1ff43d26656"},
- {file = "regex-2024.5.15-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a32b96f15c8ab2e7d27655969a23895eb799de3665fa94349f3b2fbfd547236f"},
- {file = "regex-2024.5.15-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:10002e86e6068d9e1c91eae8295ef690f02f913c57db120b58fdd35a6bb1af35"},
- {file = "regex-2024.5.15-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ec54d5afa89c19c6dd8541a133be51ee1017a38b412b1321ccb8d6ddbeb4cf7d"},
- {file = "regex-2024.5.15-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:10e4ce0dca9ae7a66e6089bb29355d4432caed736acae36fef0fdd7879f0b0cb"},
- {file = "regex-2024.5.15-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3e507ff1e74373c4d3038195fdd2af30d297b4f0950eeda6f515ae3d84a1770f"},
- {file = "regex-2024.5.15-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d1f059a4d795e646e1c37665b9d06062c62d0e8cc3c511fe01315973a6542e40"},
- {file = "regex-2024.5.15-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:0721931ad5fe0dda45d07f9820b90b2148ccdd8e45bb9e9b42a146cb4f695649"},
- {file = "regex-2024.5.15-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:833616ddc75ad595dee848ad984d067f2f31be645d603e4d158bba656bbf516c"},
- {file = "regex-2024.5.15-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:287eb7f54fc81546346207c533ad3c2c51a8d61075127d7f6d79aaf96cdee890"},
- {file = "regex-2024.5.15-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:19dfb1c504781a136a80ecd1fff9f16dddf5bb43cec6871778c8a907a085bb3d"},
- {file = "regex-2024.5.15-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:119af6e56dce35e8dfb5222573b50c89e5508d94d55713c75126b753f834de68"},
- {file = "regex-2024.5.15-cp311-cp311-win32.whl", hash = "sha256:1c1c174d6ec38d6c8a7504087358ce9213d4332f6293a94fbf5249992ba54efa"},
- {file = "regex-2024.5.15-cp311-cp311-win_amd64.whl", hash = "sha256:9e717956dcfd656f5055cc70996ee2cc82ac5149517fc8e1b60261b907740201"},
- {file = "regex-2024.5.15-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:632b01153e5248c134007209b5c6348a544ce96c46005d8456de1d552455b014"},
- {file = "regex-2024.5.15-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:e64198f6b856d48192bf921421fdd8ad8eb35e179086e99e99f711957ffedd6e"},
- {file = "regex-2024.5.15-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:68811ab14087b2f6e0fc0c2bae9ad689ea3584cad6917fc57be6a48bbd012c49"},
- {file = "regex-2024.5.15-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f8ec0c2fea1e886a19c3bee0cd19d862b3aa75dcdfb42ebe8ed30708df64687a"},
- {file = "regex-2024.5.15-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d0c0c0003c10f54a591d220997dd27d953cd9ccc1a7294b40a4be5312be8797b"},
- {file = "regex-2024.5.15-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2431b9e263af1953c55abbd3e2efca67ca80a3de8a0437cb58e2421f8184717a"},
- {file = "regex-2024.5.15-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4a605586358893b483976cffc1723fb0f83e526e8f14c6e6614e75919d9862cf"},
- {file = "regex-2024.5.15-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:391d7f7f1e409d192dba8bcd42d3e4cf9e598f3979cdaed6ab11288da88cb9f2"},
- {file = "regex-2024.5.15-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9ff11639a8d98969c863d4617595eb5425fd12f7c5ef6621a4b74b71ed8726d5"},
- {file = "regex-2024.5.15-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:4eee78a04e6c67e8391edd4dad3279828dd66ac4b79570ec998e2155d2e59fd5"},
- {file = "regex-2024.5.15-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:8fe45aa3f4aa57faabbc9cb46a93363edd6197cbc43523daea044e9ff2fea83e"},
- {file = "regex-2024.5.15-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:d0a3d8d6acf0c78a1fff0e210d224b821081330b8524e3e2bc5a68ef6ab5803d"},
- {file = "regex-2024.5.15-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c486b4106066d502495b3025a0a7251bf37ea9540433940a23419461ab9f2a80"},
- {file = "regex-2024.5.15-cp312-cp312-win32.whl", hash = "sha256:c49e15eac7c149f3670b3e27f1f28a2c1ddeccd3a2812cba953e01be2ab9b5fe"},
- {file = "regex-2024.5.15-cp312-cp312-win_amd64.whl", hash = "sha256:673b5a6da4557b975c6c90198588181029c60793835ce02f497ea817ff647cb2"},
- {file = "regex-2024.5.15-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:87e2a9c29e672fc65523fb47a90d429b70ef72b901b4e4b1bd42387caf0d6835"},
- {file = "regex-2024.5.15-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:c3bea0ba8b73b71b37ac833a7f3fd53825924165da6a924aec78c13032f20850"},
- {file = "regex-2024.5.15-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:bfc4f82cabe54f1e7f206fd3d30fda143f84a63fe7d64a81558d6e5f2e5aaba9"},
- {file = "regex-2024.5.15-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e5bb9425fe881d578aeca0b2b4b3d314ec88738706f66f219c194d67179337cb"},
- {file = "regex-2024.5.15-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:64c65783e96e563103d641760664125e91bd85d8e49566ee560ded4da0d3e704"},
- {file = "regex-2024.5.15-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cf2430df4148b08fb4324b848672514b1385ae3807651f3567871f130a728cc3"},
- {file = "regex-2024.5.15-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5397de3219a8b08ae9540c48f602996aa6b0b65d5a61683e233af8605c42b0f2"},
- {file = "regex-2024.5.15-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:455705d34b4154a80ead722f4f185b04c4237e8e8e33f265cd0798d0e44825fa"},
- {file = "regex-2024.5.15-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:b2b6f1b3bb6f640c1a92be3bbfbcb18657b125b99ecf141fb3310b5282c7d4ed"},
- {file = "regex-2024.5.15-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:3ad070b823ca5890cab606c940522d05d3d22395d432f4aaaf9d5b1653e47ced"},
- {file = "regex-2024.5.15-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:5b5467acbfc153847d5adb21e21e29847bcb5870e65c94c9206d20eb4e99a384"},
- {file = "regex-2024.5.15-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:e6662686aeb633ad65be2a42b4cb00178b3fbf7b91878f9446075c404ada552f"},
- {file = "regex-2024.5.15-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:2b4c884767504c0e2401babe8b5b7aea9148680d2e157fa28f01529d1f7fcf67"},
- {file = "regex-2024.5.15-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:3cd7874d57f13bf70078f1ff02b8b0aa48d5b9ed25fc48547516c6aba36f5741"},
- {file = "regex-2024.5.15-cp38-cp38-win32.whl", hash = "sha256:e4682f5ba31f475d58884045c1a97a860a007d44938c4c0895f41d64481edbc9"},
- {file = "regex-2024.5.15-cp38-cp38-win_amd64.whl", hash = "sha256:d99ceffa25ac45d150e30bd9ed14ec6039f2aad0ffa6bb87a5936f5782fc1569"},
- {file = "regex-2024.5.15-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:13cdaf31bed30a1e1c2453ef6015aa0983e1366fad2667657dbcac7b02f67133"},
- {file = "regex-2024.5.15-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:cac27dcaa821ca271855a32188aa61d12decb6fe45ffe3e722401fe61e323cd1"},
- {file = "regex-2024.5.15-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7dbe2467273b875ea2de38ded4eba86cbcbc9a1a6d0aa11dcf7bd2e67859c435"},
- {file = "regex-2024.5.15-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:64f18a9a3513a99c4bef0e3efd4c4a5b11228b48aa80743be822b71e132ae4f5"},
- {file = "regex-2024.5.15-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d347a741ea871c2e278fde6c48f85136c96b8659b632fb57a7d1ce1872547600"},
- {file = "regex-2024.5.15-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1878b8301ed011704aea4c806a3cadbd76f84dece1ec09cc9e4dc934cfa5d4da"},
- {file = "regex-2024.5.15-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4babf07ad476aaf7830d77000874d7611704a7fcf68c9c2ad151f5d94ae4bfc4"},
- {file = "regex-2024.5.15-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:35cb514e137cb3488bce23352af3e12fb0dbedd1ee6e60da053c69fb1b29cc6c"},
- {file = "regex-2024.5.15-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:cdd09d47c0b2efee9378679f8510ee6955d329424c659ab3c5e3a6edea696294"},
- {file = "regex-2024.5.15-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:72d7a99cd6b8f958e85fc6ca5b37c4303294954eac1376535b03c2a43eb72629"},
- {file = "regex-2024.5.15-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:a094801d379ab20c2135529948cb84d417a2169b9bdceda2a36f5f10977ebc16"},
- {file = "regex-2024.5.15-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:c0c18345010870e58238790a6779a1219b4d97bd2e77e1140e8ee5d14df071aa"},
- {file = "regex-2024.5.15-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:16093f563098448ff6b1fa68170e4acbef94e6b6a4e25e10eae8598bb1694b5d"},
- {file = "regex-2024.5.15-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:e38a7d4e8f633a33b4c7350fbd8bad3b70bf81439ac67ac38916c4a86b465456"},
- {file = "regex-2024.5.15-cp39-cp39-win32.whl", hash = "sha256:71a455a3c584a88f654b64feccc1e25876066c4f5ef26cd6dd711308aa538694"},
- {file = "regex-2024.5.15-cp39-cp39-win_amd64.whl", hash = "sha256:cab12877a9bdafde5500206d1020a584355a97884dfd388af3699e9137bf7388"},
- {file = "regex-2024.5.15.tar.gz", hash = "sha256:d3ee02d9e5f482cc8309134a91eeaacbdd2261ba111b0fef3748eeb4913e6a2c"},
+ {file = "regex-2024.7.24-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:228b0d3f567fafa0633aee87f08b9276c7062da9616931382993c03808bb68ce"},
+ {file = "regex-2024.7.24-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3426de3b91d1bc73249042742f45c2148803c111d1175b283270177fdf669024"},
+ {file = "regex-2024.7.24-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f273674b445bcb6e4409bf8d1be67bc4b58e8b46fd0d560055d515b8830063cd"},
+ {file = "regex-2024.7.24-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:23acc72f0f4e1a9e6e9843d6328177ae3074b4182167e34119ec7233dfeccf53"},
+ {file = "regex-2024.7.24-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:65fd3d2e228cae024c411c5ccdffae4c315271eee4a8b839291f84f796b34eca"},
+ {file = "regex-2024.7.24-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c414cbda77dbf13c3bc88b073a1a9f375c7b0cb5e115e15d4b73ec3a2fbc6f59"},
+ {file = "regex-2024.7.24-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf7a89eef64b5455835f5ed30254ec19bf41f7541cd94f266ab7cbd463f00c41"},
+ {file = "regex-2024.7.24-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:19c65b00d42804e3fbea9708f0937d157e53429a39b7c61253ff15670ff62cb5"},
+ {file = "regex-2024.7.24-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:7a5486ca56c8869070a966321d5ab416ff0f83f30e0e2da1ab48815c8d165d46"},
+ {file = "regex-2024.7.24-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6f51f9556785e5a203713f5efd9c085b4a45aecd2a42573e2b5041881b588d1f"},
+ {file = "regex-2024.7.24-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:a4997716674d36a82eab3e86f8fa77080a5d8d96a389a61ea1d0e3a94a582cf7"},
+ {file = "regex-2024.7.24-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:c0abb5e4e8ce71a61d9446040c1e86d4e6d23f9097275c5bd49ed978755ff0fe"},
+ {file = "regex-2024.7.24-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:18300a1d78cf1290fa583cd8b7cde26ecb73e9f5916690cf9d42de569c89b1ce"},
+ {file = "regex-2024.7.24-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:416c0e4f56308f34cdb18c3f59849479dde5b19febdcd6e6fa4d04b6c31c9faa"},
+ {file = "regex-2024.7.24-cp310-cp310-win32.whl", hash = "sha256:fb168b5924bef397b5ba13aabd8cf5df7d3d93f10218d7b925e360d436863f66"},
+ {file = "regex-2024.7.24-cp310-cp310-win_amd64.whl", hash = "sha256:6b9fc7e9cc983e75e2518496ba1afc524227c163e43d706688a6bb9eca41617e"},
+ {file = "regex-2024.7.24-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:382281306e3adaaa7b8b9ebbb3ffb43358a7bbf585fa93821300a418bb975281"},
+ {file = "regex-2024.7.24-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4fdd1384619f406ad9037fe6b6eaa3de2749e2e12084abc80169e8e075377d3b"},
+ {file = "regex-2024.7.24-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:3d974d24edb231446f708c455fd08f94c41c1ff4f04bcf06e5f36df5ef50b95a"},
+ {file = "regex-2024.7.24-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a2ec4419a3fe6cf8a4795752596dfe0adb4aea40d3683a132bae9c30b81e8d73"},
+ {file = "regex-2024.7.24-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:eb563dd3aea54c797adf513eeec819c4213d7dbfc311874eb4fd28d10f2ff0f2"},
+ {file = "regex-2024.7.24-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:45104baae8b9f67569f0f1dca5e1f1ed77a54ae1cd8b0b07aba89272710db61e"},
+ {file = "regex-2024.7.24-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:994448ee01864501912abf2bad9203bffc34158e80fe8bfb5b031f4f8e16da51"},
+ {file = "regex-2024.7.24-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3fac296f99283ac232d8125be932c5cd7644084a30748fda013028c815ba3364"},
+ {file = "regex-2024.7.24-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:7e37e809b9303ec3a179085415cb5f418ecf65ec98cdfe34f6a078b46ef823ee"},
+ {file = "regex-2024.7.24-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:01b689e887f612610c869421241e075c02f2e3d1ae93a037cb14f88ab6a8934c"},
+ {file = "regex-2024.7.24-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:f6442f0f0ff81775eaa5b05af8a0ffa1dda36e9cf6ec1e0d3d245e8564b684ce"},
+ {file = "regex-2024.7.24-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:871e3ab2838fbcb4e0865a6e01233975df3a15e6fce93b6f99d75cacbd9862d1"},
+ {file = "regex-2024.7.24-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:c918b7a1e26b4ab40409820ddccc5d49871a82329640f5005f73572d5eaa9b5e"},
+ {file = "regex-2024.7.24-cp311-cp311-win32.whl", hash = "sha256:2dfbb8baf8ba2c2b9aa2807f44ed272f0913eeeba002478c4577b8d29cde215c"},
+ {file = "regex-2024.7.24-cp311-cp311-win_amd64.whl", hash = "sha256:538d30cd96ed7d1416d3956f94d54e426a8daf7c14527f6e0d6d425fcb4cca52"},
+ {file = "regex-2024.7.24-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:fe4ebef608553aff8deb845c7f4f1d0740ff76fa672c011cc0bacb2a00fbde86"},
+ {file = "regex-2024.7.24-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:74007a5b25b7a678459f06559504f1eec2f0f17bca218c9d56f6a0a12bfffdad"},
+ {file = "regex-2024.7.24-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:7df9ea48641da022c2a3c9c641650cd09f0cd15e8908bf931ad538f5ca7919c9"},
+ {file = "regex-2024.7.24-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6a1141a1dcc32904c47f6846b040275c6e5de0bf73f17d7a409035d55b76f289"},
+ {file = "regex-2024.7.24-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80c811cfcb5c331237d9bad3bea2c391114588cf4131707e84d9493064d267f9"},
+ {file = "regex-2024.7.24-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7214477bf9bd195894cf24005b1e7b496f46833337b5dedb7b2a6e33f66d962c"},
+ {file = "regex-2024.7.24-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d55588cba7553f0b6ec33130bc3e114b355570b45785cebdc9daed8c637dd440"},
+ {file = "regex-2024.7.24-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:558a57cfc32adcf19d3f791f62b5ff564922942e389e3cfdb538a23d65a6b610"},
+ {file = "regex-2024.7.24-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:a512eed9dfd4117110b1881ba9a59b31433caed0c4101b361f768e7bcbaf93c5"},
+ {file = "regex-2024.7.24-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:86b17ba823ea76256b1885652e3a141a99a5c4422f4a869189db328321b73799"},
+ {file = "regex-2024.7.24-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:5eefee9bfe23f6df09ffb6dfb23809f4d74a78acef004aa904dc7c88b9944b05"},
+ {file = "regex-2024.7.24-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:731fcd76bbdbf225e2eb85b7c38da9633ad3073822f5ab32379381e8c3c12e94"},
+ {file = "regex-2024.7.24-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:eaef80eac3b4cfbdd6de53c6e108b4c534c21ae055d1dbea2de6b3b8ff3def38"},
+ {file = "regex-2024.7.24-cp312-cp312-win32.whl", hash = "sha256:185e029368d6f89f36e526764cf12bf8d6f0e3a2a7737da625a76f594bdfcbfc"},
+ {file = "regex-2024.7.24-cp312-cp312-win_amd64.whl", hash = "sha256:2f1baff13cc2521bea83ab2528e7a80cbe0ebb2c6f0bfad15be7da3aed443908"},
+ {file = "regex-2024.7.24-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:66b4c0731a5c81921e938dcf1a88e978264e26e6ac4ec96a4d21ae0354581ae0"},
+ {file = "regex-2024.7.24-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:88ecc3afd7e776967fa16c80f974cb79399ee8dc6c96423321d6f7d4b881c92b"},
+ {file = "regex-2024.7.24-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:64bd50cf16bcc54b274e20235bf8edbb64184a30e1e53873ff8d444e7ac656b2"},
+ {file = "regex-2024.7.24-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:eb462f0e346fcf41a901a126b50f8781e9a474d3927930f3490f38a6e73b6950"},
+ {file = "regex-2024.7.24-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a82465ebbc9b1c5c50738536fdfa7cab639a261a99b469c9d4c7dcbb2b3f1e57"},
+ {file = "regex-2024.7.24-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:68a8f8c046c6466ac61a36b65bb2395c74451df2ffb8458492ef49900efed293"},
+ {file = "regex-2024.7.24-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dac8e84fff5d27420f3c1e879ce9929108e873667ec87e0c8eeb413a5311adfe"},
+ {file = "regex-2024.7.24-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ba2537ef2163db9e6ccdbeb6f6424282ae4dea43177402152c67ef869cf3978b"},
+ {file = "regex-2024.7.24-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:43affe33137fcd679bdae93fb25924979517e011f9dea99163f80b82eadc7e53"},
+ {file = "regex-2024.7.24-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:c9bb87fdf2ab2370f21e4d5636e5317775e5d51ff32ebff2cf389f71b9b13750"},
+ {file = "regex-2024.7.24-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:945352286a541406f99b2655c973852da7911b3f4264e010218bbc1cc73168f2"},
+ {file = "regex-2024.7.24-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:8bc593dcce679206b60a538c302d03c29b18e3d862609317cb560e18b66d10cf"},
+ {file = "regex-2024.7.24-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:3f3b6ca8eae6d6c75a6cff525c8530c60e909a71a15e1b731723233331de4169"},
+ {file = "regex-2024.7.24-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:c51edc3541e11fbe83f0c4d9412ef6c79f664a3745fab261457e84465ec9d5a8"},
+ {file = "regex-2024.7.24-cp38-cp38-win32.whl", hash = "sha256:d0a07763776188b4db4c9c7fb1b8c494049f84659bb387b71c73bbc07f189e96"},
+ {file = "regex-2024.7.24-cp38-cp38-win_amd64.whl", hash = "sha256:8fd5afd101dcf86a270d254364e0e8dddedebe6bd1ab9d5f732f274fa00499a5"},
+ {file = "regex-2024.7.24-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:0ffe3f9d430cd37d8fa5632ff6fb36d5b24818c5c986893063b4e5bdb84cdf24"},
+ {file = "regex-2024.7.24-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:25419b70ba00a16abc90ee5fce061228206173231f004437730b67ac77323f0d"},
+ {file = "regex-2024.7.24-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:33e2614a7ce627f0cdf2ad104797d1f68342d967de3695678c0cb84f530709f8"},
+ {file = "regex-2024.7.24-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d33a0021893ede5969876052796165bab6006559ab845fd7b515a30abdd990dc"},
+ {file = "regex-2024.7.24-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:04ce29e2c5fedf296b1a1b0acc1724ba93a36fb14031f3abfb7abda2806c1535"},
+ {file = "regex-2024.7.24-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b16582783f44fbca6fcf46f61347340c787d7530d88b4d590a397a47583f31dd"},
+ {file = "regex-2024.7.24-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:836d3cc225b3e8a943d0b02633fb2f28a66e281290302a79df0e1eaa984ff7c1"},
+ {file = "regex-2024.7.24-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:438d9f0f4bc64e8dea78274caa5af971ceff0f8771e1a2333620969936ba10be"},
+ {file = "regex-2024.7.24-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:973335b1624859cb0e52f96062a28aa18f3a5fc77a96e4a3d6d76e29811a0e6e"},
+ {file = "regex-2024.7.24-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:c5e69fd3eb0b409432b537fe3c6f44ac089c458ab6b78dcec14478422879ec5f"},
+ {file = "regex-2024.7.24-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:fbf8c2f00904eaf63ff37718eb13acf8e178cb940520e47b2f05027f5bb34ce3"},
+ {file = "regex-2024.7.24-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:ae2757ace61bc4061b69af19e4689fa4416e1a04840f33b441034202b5cd02d4"},
+ {file = "regex-2024.7.24-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:44fc61b99035fd9b3b9453f1713234e5a7c92a04f3577252b45feefe1b327759"},
+ {file = "regex-2024.7.24-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:84c312cdf839e8b579f504afcd7b65f35d60b6285d892b19adea16355e8343c9"},
+ {file = "regex-2024.7.24-cp39-cp39-win32.whl", hash = "sha256:ca5b2028c2f7af4e13fb9fc29b28d0ce767c38c7facdf64f6c2cd040413055f1"},
+ {file = "regex-2024.7.24-cp39-cp39-win_amd64.whl", hash = "sha256:7c479f5ae937ec9985ecaf42e2e10631551d909f203e31308c12d703922742f9"},
+ {file = "regex-2024.7.24.tar.gz", hash = "sha256:9cfd009eed1a46b27c14039ad5bbc5e71b6367c5b2e6d5f5da0ea91600817506"},
]
[[package]]
diff --git a/tests/integration/test_pipeline_error_propagation.py b/tests/integration/test_pipeline_error_propagation.py
deleted file mode 100644
index 3e73e54a3..000000000
--- a/tests/integration/test_pipeline_error_propagation.py
+++ /dev/null
@@ -1,163 +0,0 @@
-import asyncio
-from datetime import datetime
-
-import pytest
-
-from nodestream.interpreting import Interpreter
-from nodestream.pipeline import Pipeline, Writer
-from nodestream.pipeline.extractors import Extractor
-from nodestream.pipeline.pipeline import (
- PRECHECK_MESSAGE,
- STOP_EXCEPTION,
- TIMEOUT_MESSAGE,
- WORK_BODY_EXCEPTION,
- PipelineException,
-)
-from nodestream.pipeline.transformers import PassTransformer
-
-"""
-Method ->
- Step 1: Infinite extractor
- Step 2: Ingestion that passes
- Step 3: A writer that fails.
-
- The first test is for the case where the extractor fills up the outbox with a bottlenecked writer that fails.
- Without checking for pipeline failure with a full outbox, the program will freeze waiting for an outbox to obtain space it will never recieve.
-
- The second test is for a slow extractor on a writer that fails.
- The propagation of the error should not occur only when the outbox is full.
-
-
-"""
-
-
-class StopException(Exception):
- pass
-
-
-class EventualFailureWriter(Writer):
- async def write_record(self, _):
- await asyncio.sleep(3)
- raise Exception
-
-
-class ImmediateFailureWriter(Writer):
- def __init__(self):
- self.item_count = 0
-
- async def write_record(self, _):
- raise Exception
-
-
-class ExtractQuickly(Extractor):
- def __init__(self):
- self.item_count = 0
-
- async def extract_records(self):
- while True:
- yield self.item_count
- self.item_count += 1
-
-
-class ExtractSlowly(Extractor):
- def __init__(self):
- self.item_count = 0
-
- async def extract_records(self):
- while True:
- yield self.item_count
- self.item_count += 1
- await asyncio.sleep(0.1)
-
-
-class FailTransformer(PassTransformer):
- async def finish(self):
- raise StopException
-
-
-@pytest.fixture
-def interpreter():
- return Interpreter.from_file_data(interpretations=[])
-
-
-# Test that the pipeline throws an exception as soon as the buffer is full (1.0) and the outbox.put timeout is reached (0.1).
-@pytest.mark.asyncio
-async def test_error_propagation_on_full_buffer(interpreter):
- pipeline = Pipeline([ExtractQuickly(), interpreter, EventualFailureWriter()], 20)
- did_except = False
- try:
- await asyncio.wait_for(pipeline.run(), timeout=3.2 * 2)
- except PipelineException as exception:
- executor_work_body_exception = exception.errors[0].exceptions[
- WORK_BODY_EXCEPTION
- ]
- interpreter_work_body_exception = exception.errors[1].exceptions[
- WORK_BODY_EXCEPTION
- ]
- assert str(executor_work_body_exception) == TIMEOUT_MESSAGE
- assert str(interpreter_work_body_exception) == TIMEOUT_MESSAGE
- did_except = True
- assert did_except
-
-
-"""
-(0) -> Executor, Interpreter, Writer (Fails)
-(0.1) -> Executor, Interpreter (Fails), Writer (Failed)
-(0.2) -> Executer (Fails), Interpreter (Failed), Writer(Failed)
-(0.3) -> PipelineException
-
-"""
-
-
-@pytest.mark.asyncio
-async def test_immediate_error_propogation(interpreter):
- pipeline = Pipeline([ExtractSlowly(), interpreter, ImmediateFailureWriter()], 20)
- beginning_time = datetime.now()
- did_except = False
- try:
- await pipeline.run()
- except PipelineException as exception:
- executor_work_body_exception = exception.errors[0].exceptions[
- WORK_BODY_EXCEPTION
- ]
- writer_work_body_exception = exception.errors[1].exceptions[WORK_BODY_EXCEPTION]
- assert str(executor_work_body_exception) == PRECHECK_MESSAGE
- assert isinstance(writer_work_body_exception, Exception)
- did_except = True
- assert did_except
- ending_time = datetime.now()
- difference = ending_time - beginning_time
- assert difference.total_seconds() < 0.5 * 2
-
-
-# Testing that the exception is propagated. Also testing that we would see a failure in the stop-process if necessary.
-@pytest.mark.asyncio
-async def test_immediate_error_propagation_fails_all_steps():
- steps = (
- [ExtractSlowly()]
- + [PassTransformer() for _ in range(10)]
- + [FailTransformer()]
- + [ImmediateFailureWriter()]
- )
- pipeline = Pipeline(steps, 20)
- beginning_time = datetime.now()
- did_except = False
-
- try:
- await asyncio.wait_for(pipeline.run(), timeout=4.2 * 2)
- except PipelineException as exception:
- # Every step should have a Stepexeption except for the last one that has a Exception
- extractor_work_body_exception = exception.errors[0].exceptions[
- WORK_BODY_EXCEPTION
- ]
- fail_transformer_stop_exception = exception.errors[1].exceptions[STOP_EXCEPTION]
- writer_body_exception = exception.errors[2].exceptions[WORK_BODY_EXCEPTION]
- assert str(extractor_work_body_exception) == PRECHECK_MESSAGE
- assert isinstance(fail_transformer_stop_exception, StopException)
- assert isinstance(writer_body_exception, Exception)
- did_except = True
-
- assert did_except
- ending_time = datetime.now()
- difference = ending_time - beginning_time
- assert difference.total_seconds() < 0.5 * 2
diff --git a/tests/integration/test_pipeline_flush_handling.py b/tests/integration/test_pipeline_flush_handling.py
index 7e323d32b..b98423b49 100644
--- a/tests/integration/test_pipeline_flush_handling.py
+++ b/tests/integration/test_pipeline_flush_handling.py
@@ -2,7 +2,12 @@
from nodestream.databases import GraphDatabaseWriter
from nodestream.interpreting import Interpreter
-from nodestream.pipeline import Flush, IterableExtractor, Pipeline
+from nodestream.pipeline import (
+ Flush,
+ IterableExtractor,
+ Pipeline,
+ PipelineProgressReporter,
+)
@pytest.fixture
@@ -36,5 +41,5 @@ def test_extractor_with_flushes(mocker):
@pytest.mark.asyncio
async def test_flush_handling(writer, interpreter, test_extractor_with_flushes):
pipeline = Pipeline([test_extractor_with_flushes, interpreter, writer], 1000)
- await pipeline.run()
+ await pipeline.run(PipelineProgressReporter())
assert writer.ingest_strategy.flush.call_count == 5
diff --git a/tests/unit/cli/operations/test_run_pipeline.py b/tests/unit/cli/operations/test_run_pipeline.py
index 2f8c843a2..5ae7ddf73 100644
--- a/tests/unit/cli/operations/test_run_pipeline.py
+++ b/tests/unit/cli/operations/test_run_pipeline.py
@@ -3,6 +3,7 @@
from nodestream.cli.operations.run_pipeline import (
WARNING_NO_TARGETS_PROVIDED,
+ ProgressIndicator,
RunPipeline,
SpinnerProgressIndicator,
)
@@ -62,6 +63,16 @@ def test_spinner_progress_callback(mocker):
spinner.progress.set_message.assert_called_once()
+def test_spinner_error_condition(mocker):
+ spinner = SpinnerProgressIndicator(mocker.Mock(), "pipeline_name")
+ spinner.on_start()
+ spinner.on_fatal_error(Exception())
+ spinner.progress.set_message.assert_called_once()
+
+ with pytest.raises(Exception):
+ spinner.on_finish(PipelineContext())
+
+
@pytest.mark.parametrize(
"from_cli,from_pipeline,expected",
[
@@ -109,3 +120,9 @@ def test_get_pipleines_to_run(
cmd.argument.return_value = provided_pipelines
results = RunPipeline(project_with_default_scope).get_pipelines_to_run(cmd)
assert_that([result.name for result in results], equal_to(expected))
+
+
+def test_progress_indicator_error(mocker):
+ indicator = ProgressIndicator(mocker.Mock(), "pipeline_name")
+ indicator.on_fatal_error(Exception("Boom"))
+ indicator.command.line.assert_called_with("Boom")
diff --git a/tests/unit/interpreting/test_interpreter.py b/tests/unit/interpreting/test_interpreter.py
index 4c8ff0b4e..64f070248 100644
--- a/tests/unit/interpreting/test_interpreter.py
+++ b/tests/unit/interpreting/test_interpreter.py
@@ -21,7 +21,6 @@
RecordDecomposer,
WholeRecordDecomposer,
)
-from nodestream.model import DesiredIngestion
from nodestream.pipeline import IterableExtractor
from nodestream.pipeline.value_providers import ProviderContext
@@ -79,18 +78,13 @@ def test_intepret_record_iterates_through_interpretation_process(stubbed_interpr
@pytest.mark.asyncio
@freeze_time("1998-03-25 12:00:01")
-async def test_handle_async_record_stream_returns_iteration_results(
- stubbed_interpreter, mocker
-):
+async def test_transform_record_returns_iteration_results(stubbed_interpreter, mocker):
contexts = [[ProviderContext.fresh(i)] for i in range(5)]
- stubbed_interpreter.gather_used_indexes = mocker.Mock(return_value=[])
stubbed_interpreter.interpret_record = mocker.Mock(side_effect=contexts)
results = [
r
- async for r in stubbed_interpreter.handle_async_record_stream(
- IterableExtractor.range(stop=5).extract_records()
- )
- if isinstance(r, DesiredIngestion)
+ async for input in IterableExtractor.range(stop=5).extract_records()
+ async for r in stubbed_interpreter.transform_record(input)
]
assert_that(results, equal_to([context[0].desired_ingest for context in contexts]))
diff --git a/tests/unit/pipeline/test_channel.py b/tests/unit/pipeline/test_channel.py
new file mode 100644
index 000000000..c0b994ac5
--- /dev/null
+++ b/tests/unit/pipeline/test_channel.py
@@ -0,0 +1,69 @@
+import asyncio
+
+import pytest
+
+from nodestream.pipeline.channel import channel
+
+
+@pytest.mark.asyncio
+async def test_channel_simple_case_racing():
+ input, output = channel(100)
+
+ async def producer():
+ for i in range(1000):
+ await output.put(i)
+ await output.done()
+
+ async def consumer():
+ results = []
+ while True:
+ record = await input.get()
+ if record is None:
+ break
+ results.append(record)
+
+ return results
+
+ results, _ = await asyncio.gather(consumer(), producer())
+ assert results == list(range(1000))
+
+
+@pytest.mark.asyncio
+async def test_channel_consumer_done():
+ input, output = channel(100)
+
+ async def producer():
+ for i in range(1000):
+ if not await output.put(i):
+ return False
+
+ async def consumer():
+ input.done()
+
+ _, result = await asyncio.gather(consumer(), producer())
+ assert result is False
+
+
+@pytest.mark.asyncio
+async def test_channel_times_out_waiting_for_space():
+ input, output = channel(10)
+
+ async def producer():
+ results = [await output.put(i) for i in range(12)]
+ return any(r is False for r in results)
+
+ async def consumer():
+ # wait until the producer has filled the queue to capacity
+ # so that we can be sure that the producer is blocked
+ # waiting for space in the queue
+ while input.channel.queue.qsize() < 10:
+ await asyncio.sleep(0.1)
+
+ # Be extra sure that the producer is blocked by waiting
+ # for a bit longer before calling done on the input channel
+ await asyncio.sleep(0.5)
+
+ # Call done so the input channel will give up waiting for space
+ input.done()
+
+ assert await asyncio.gather(consumer(), producer()) == [None, True]
diff --git a/tests/unit/pipeline/test_filters.py b/tests/unit/pipeline/test_filters.py
index cd4310852..8b0f6c754 100644
--- a/tests/unit/pipeline/test_filters.py
+++ b/tests/unit/pipeline/test_filters.py
@@ -64,10 +64,6 @@ async def test_exclude_possibilities__failing():
@pytest.mark.asyncio
async def test_base_filter_filters_correctly():
- async def records():
- yield 1
- yield 2
-
class TestFilter(Filter):
def __init__(self, results) -> None:
self.results = results
@@ -76,8 +72,10 @@ async def filter_record(self, record):
return self.results.pop(0)
subject = TestFilter([False, True])
- results = [record async for record in subject.handle_async_record_stream(records())]
+ results = [record async for record in subject.process_record(1, None)]
assert_that(results, equal_to([1]))
+ results = [record async for record in subject.process_record(2, None)]
+ assert_that(results, equal_to([]))
REGEX = ".*[\[\]\{\}\(\)\\\/~,]+"
diff --git a/tests/unit/pipeline/test_pipeline.py b/tests/unit/pipeline/test_pipeline.py
index 172230065..3aa59473e 100644
--- a/tests/unit/pipeline/test_pipeline.py
+++ b/tests/unit/pipeline/test_pipeline.py
@@ -1,152 +1,138 @@
-from unittest.mock import patch
-
import pytest
-from nodestream.pipeline import PipelineProgressReporter
from nodestream.pipeline.pipeline import (
- Pipeline,
- PipelineException,
- PipelineState,
- StepException,
+ PipelineOutput,
+ PipelineProgressReporter,
+ Step,
+ StepContext,
StepExecutor,
- empty_async_generator,
+ StepInput,
+ StepOutput,
)
-from nodestream.pipeline.step import PassStep
-
-
-async def async_range(size):
- for i in range(size):
- yield i
@pytest.fixture
-def pipeline_state():
- return PipelineState()
+def step_executor(mocker):
+ step = mocker.Mock(Step)
+ input = mocker.Mock(StepInput)
+ output = mocker.Mock(StepOutput)
+ context = mocker.Mock(StepContext)
+ return StepExecutor(step, input, output, context)
-@pytest.fixture
-def pipeline(mocker):
- s1, s2 = PassStep(), PassStep()
- s1.handle_async_record_stream = mocker.Mock(return_value=empty_async_generator())
- s2.handle_async_record_stream = mocker.Mock(return_value=empty_async_generator())
- s1.finish, s2.finish = mocker.AsyncMock(), mocker.AsyncMock()
- return Pipeline([s1, s2], 10)
-
-@pytest.fixture
-def step_executor(mocker, pipeline_state):
- step = PassStep()
- step.handle_async_record_stream = mocker.Mock(return_value=empty_async_generator())
- step.finish = mocker.AsyncMock()
- return StepExecutor(pipeline_state, step=step, upstream=None)
+@pytest.mark.asyncio
+async def test_start_step(step_executor):
+ await step_executor.start_step()
+ step_executor.step.start.assert_called_once_with(step_executor.context)
@pytest.mark.asyncio
-async def test_pipeline_run(pipeline):
- await pipeline.run()
- for step in pipeline.steps:
- step.handle_async_record_stream.assert_called_once()
- step.finish.assert_awaited_once()
+async def test_start_step_error(step_executor, mocker):
+ step_executor.step.start.side_effect = Exception("Boom")
+ await step_executor.start_step()
+ step_executor.context.report_error.assert_called_once_with(
+ "Error starting step", mocker.ANY
+ )
@pytest.mark.asyncio
-async def test_pipeline_run_with_error_on_start(pipeline, mocker):
- with pytest.raises(expected_exception=PipelineException):
- await pipeline.run(
- PipelineProgressReporter(
- on_start_callback=mocker.Mock(side_effect=Exception("test"))
- )
- )
- for step in pipeline.steps:
- step.handle_async_record_stream.assert_called_once()
- step.finish.assert_awaited_once()
+async def test_stop_step(step_executor):
+ await step_executor.stop_step()
+ step_executor.step.finish.assert_called_once_with(step_executor.context)
@pytest.mark.asyncio
-async def test_pipeline_run_with_error_on_work_body(pipeline):
- for step in pipeline.steps:
- step.handle_async_record_stream.side_effect = Exception("test")
+async def test_stop_step_error(step_executor, mocker):
+ step_executor.step.finish.side_effect = Exception("Boom")
+ await step_executor.stop_step()
+ step_executor.context.report_error.assert_called_once_with(
+ "Error stopping step", mocker.ANY
+ )
- with pytest.raises(expected_exception=PipelineException):
- await pipeline.run()
- for step in pipeline.steps:
- step.finish.assert_called_once()
+@pytest.mark.asyncio
+async def test_emit_record(step_executor, mocker):
+ record = mocker.Mock()
+ step_executor.output.put.return_value = True
+ await step_executor.emit_record(record)
+ step_executor.output.put.assert_called_once_with(record)
@pytest.mark.asyncio
-async def test_pipeline_run_with_error_on_finish(pipeline, mocker):
- with pytest.raises(expected_exception=PipelineException):
- await pipeline.run(
- PipelineProgressReporter(
- on_finish_callback=mocker.Mock(side_effect=Exception("test"))
- )
- )
+async def test_emit_record_full(step_executor, mocker):
+ record = mocker.Mock()
+ step_executor.output.put.return_value = False
+ await step_executor.emit_record(record)
+ step_executor.output.put.assert_called_once_with(record)
+ step_executor.context.debug.assert_called_once_with(
+ "Downstream is not accepting more records. Gracefully stopping."
+ )
@pytest.mark.asyncio
-@patch("nodestream.pipeline.pipeline.StepExecutor.start", side_effect=Exception("test"))
-async def test_step_executor_throws_start_exception(start_mock, step_executor):
- step = step_executor.step
- with pytest.raises(expected_exception=StepException):
- await step_executor.work_loop()
- step.handle_async_record_stream.assert_called_once()
- step.finish.assert_called_once()
+async def test_drive_step(step_executor, mocker):
+ record = mocker.Mock()
+ step = Step()
+ step_executor.step = step
+ step_executor.input.get = mocker.AsyncMock(side_effect=[record, None])
+ await step_executor.drive_step()
+ step_executor.output.put.assert_called_once_with(record)
+ step_executor.context.debug.assert_called_once_with("Step finished emitting")
@pytest.mark.asyncio
-@patch(
- "nodestream.pipeline.pipeline.StepExecutor.work_body", side_effect=Exception("test")
-)
-async def test_step_executor_throws_work_body_exception(work_body_mock, step_executor):
- step = step_executor.step
- with pytest.raises(expected_exception=StepException):
- await step_executor.work_loop()
- step.finish.assert_called_once()
+async def test_drive_step_error(step_executor, mocker):
+ step_executor.input.get.side_effect = Exception("Boom")
+ await step_executor.drive_step()
+ step_executor.context.report_error.assert_called_once_with(
+ "Error running step",
+ mocker.ANY,
+ fatal=True,
+ )
@pytest.mark.asyncio
-@patch("nodestream.pipeline.pipeline.StepExecutor.stop", side_effect=Exception("test"))
-async def test_step_executor_throws_finish_exception(finish_mock, step_executor):
- with pytest.raises(expected_exception=StepException):
- await step_executor.work_loop()
+async def test_drive_step_cannot_continue(step_executor, mocker):
+ record = mocker.Mock()
+ step = Step()
+ step_executor.step = step
+ step_executor.input.get = mocker.AsyncMock(side_effect=[record, None])
+ step_executor.output.put.return_value = False
+ await step_executor.drive_step()
+ step_executor.output.put.assert_called_once_with(record)
@pytest.mark.asyncio
-@patch("nodestream.pipeline.pipeline.StepExecutor.start", side_effect=Exception("test"))
-@patch(
- "nodestream.pipeline.pipeline.StepExecutor.work_body", side_effect=Exception("test")
-)
-async def test_step_executor_collects_multiple_errors(
- start_mock, work_body_mock, step_executor
+async def test_drive_step_cannot_continue_in_emit_outstanding_records(
+ step_executor, mocker
):
- with pytest.raises(expected_exception=StepException):
- await step_executor.work_loop()
- assert len(step_executor.exceptions) == 2
+ record = mocker.Mock()
+ async def emit_outstanding_records():
+ yield record
-@pytest.mark.asyncio
-@patch("nodestream.pipeline.pipeline.StepExecutor.start", side_effect=Exception("test"))
-@patch(
- "nodestream.pipeline.pipeline.StepExecutor.work_body", side_effect=Exception("test")
-)
-async def test_pipeline_errors_are_kept_in_exception(
- start_mock, work_body_mock, pipeline, mocker
-):
- with pytest.raises(expected_exception=PipelineException):
- await pipeline.run()
- assert len(pipeline.errors) == 2
- for error in pipeline.errors:
- assert type(error) == StepException
- assert "Exception in Start Process:" in error.exceptions
- assert "Exception in Work Body:" in error.exceptions
+ step_executor.input.get = mocker.AsyncMock(side_effect=[None])
+ step_executor.step.emit_outstanding_records.return_value = (
+ emit_outstanding_records()
+ )
+ step_executor.output.put.return_value = False
+ await step_executor.drive_step()
+ step_executor.output.put.assert_called_once_with(record)
@pytest.mark.asyncio
-async def test_pipeline_terminates_when_step_executor_raises_exception(mocker):
- steps = mocker.Mock(), mocker.Mock()
- pipeline = Pipeline(steps, 100)
- steps[0].handle_async_record_stream.return_value = async_range(1000)
- steps[1].handle_async_record_stream.side_effect = Exception("test")
- with pytest.raises(PipelineException):
- await pipeline.run()
+async def test_pipeline_output_call_handling_errors(mocker):
+ def on_start_callback():
+ raise Exception("Boom")
+
+ output = PipelineOutput(
+ mocker.Mock(StepInput),
+ PipelineProgressReporter(
+ on_start_callback=on_start_callback,
+ logger=mocker.Mock(),
+ ),
+ )
+
+ await output.call_handling_errors(output.reporter.on_start_callback)
+ output.reporter.logger.exception.assert_called_once()
diff --git a/tests/unit/pipeline/test_step.py b/tests/unit/pipeline/test_step.py
index e69de29bb..7aa523bb9 100644
--- a/tests/unit/pipeline/test_step.py
+++ b/tests/unit/pipeline/test_step.py
@@ -0,0 +1,83 @@
+import pytest
+
+from nodestream.pipeline.progress_reporter import PipelineProgressReporter
+from nodestream.pipeline.step import Step, StepContext
+
+
+@pytest.mark.asyncio
+async def test_default_start_does_nothing(mocker):
+ step = Step()
+ ctx = mocker.Mock(spec=StepContext)
+ await step.start(ctx)
+ ctx.assert_not_called()
+
+
+@pytest.mark.asyncio
+async def test_default_finish_does_nothing(mocker):
+ step = Step()
+ ctx = mocker.Mock(spec=StepContext)
+ await step.finish(ctx)
+ ctx.assert_not_called()
+
+
+@pytest.mark.asyncio
+async def test_default_emit_outstanding_records_does_nothing(mocker):
+ step = Step()
+ async for _ in step.emit_outstanding_records():
+ assert False, "Should not emit any records"
+
+
+@pytest.mark.asyncio
+async def test_default_process_record_yields_input_record(mocker):
+ step = Step()
+ ctx = mocker.Mock(spec=StepContext)
+ record = object()
+ async for output_record in step.process_record(record, ctx):
+ assert output_record is record
+
+
+@pytest.mark.asyncio
+async def test_step_context_report_error(mocker):
+ exception = Exception()
+ ctx = StepContext(
+ "bob", 1, PipelineProgressReporter(on_fatal_error_callback=mocker.Mock())
+ )
+ ctx.report_error("oh no, an error!", exception)
+ ctx.reporter.on_fatal_error_callback.assert_not_called()
+
+
+@pytest.mark.asyncio
+async def test_step_context_report_fatal_error(mocker):
+ exception = Exception()
+ ctx = StepContext(
+ "bob", 1, PipelineProgressReporter(on_fatal_error_callback=mocker.Mock())
+ )
+ ctx.report_error("oh no, a fatal error!", exception, fatal=True)
+ ctx.reporter.on_fatal_error_callback.assert_called_once_with(exception)
+
+
+@pytest.mark.asyncio
+async def test_step_context_report_debug_message(mocker):
+ ctx = StepContext("bob", 1, PipelineProgressReporter(logger=mocker.Mock()))
+ ctx.debug("debug message", x=12)
+ ctx.reporter.logger.debug.assert_called_once_with(
+ "debug message", extra={"index": 1, "x": 12, "step_name": "bob"}
+ )
+
+
+@pytest.mark.asyncio
+async def test_step_context_report_info_message(mocker):
+ ctx = StepContext("bob", 1, PipelineProgressReporter(logger=mocker.Mock()))
+ ctx.info("info message", x=12)
+ ctx.reporter.logger.info.assert_called_once_with(
+ "info message", extra={"index": 1, "x": 12, "step_name": "bob"}
+ )
+
+
+@pytest.mark.asyncio
+async def test_step_context_report_warning_message(mocker):
+ ctx = StepContext("bob", 1, PipelineProgressReporter(logger=mocker.Mock()))
+ ctx.warning("warning message", x=12)
+ ctx.reporter.logger.warning.assert_called_once_with(
+ "warning message", extra={"index": 1, "x": 12, "step_name": "bob"}
+ )
diff --git a/tests/unit/pipeline/test_writers.py b/tests/unit/pipeline/test_writers.py
index e584625e3..96be27933 100644
--- a/tests/unit/pipeline/test_writers.py
+++ b/tests/unit/pipeline/test_writers.py
@@ -19,12 +19,11 @@ async def test_writers_flush_on_write(mocker):
writer.flush = mocker.AsyncMock()
writer.write_record = mocker.AsyncMock()
- async def input():
- yield 1
- yield Flush
- yield 2
+ not_flush = await anext(writer.process_record(1, None))
+ assert_that(not_flush, equal_to(1))
+ assert_that(writer.write_record.await_count, equal_to(1))
+ assert_that(writer.flush.await_count, equal_to(0))
- results = [r async for r in writer.handle_async_record_stream(input())]
- assert_that(results, equal_to([1, Flush, 2]))
- assert_that(writer.write_record.await_count, equal_to(2))
+ a_flush = await anext(writer.process_record(Flush, None))
+ assert_that(a_flush, equal_to(Flush))
assert_that(writer.flush.await_count, equal_to(1))
diff --git a/tests/unit/pipeline/transformers/test_expand_json_field.py b/tests/unit/pipeline/transformers/test_expand_json_field.py
index 67bccd85d..f5c9c594e 100644
--- a/tests/unit/pipeline/transformers/test_expand_json_field.py
+++ b/tests/unit/pipeline/transformers/test_expand_json_field.py
@@ -22,9 +22,5 @@
)
async def test_expand_json_fields(input, output, path):
subject = ExpandJsonField.from_file_data(path)
-
- async def upstream():
- yield input
-
- results = [r async for r in subject.handle_async_record_stream(upstream())]
+ results = [r async for r in subject.process_record(input, None)]
assert_that(results, equal_to([output]))
diff --git a/tests/unit/pipeline/transformers/test_transformer.py b/tests/unit/pipeline/transformers/test_transformer.py
index 28aa8fdfe..57400ce1f 100644
--- a/tests/unit/pipeline/transformers/test_transformer.py
+++ b/tests/unit/pipeline/transformers/test_transformer.py
@@ -22,24 +22,15 @@ async def transform_record(self, record):
return record + 1
-class AddOneConcurrentlyGreedy(AddOneConcurrently):
- async def yield_processor(self):
- pass
-
-
ITEM_COUNT = 100
@pytest.mark.asyncio
async def test_concurrent_transformer_alL_items_collect():
items = list(range(100))
-
- async def input_record_stream():
- for i in items:
- yield i
-
add = AddOneConcurrently()
- result = [r async for r in add.handle_async_record_stream(input_record_stream())]
+ result = [r for i in items async for r in add.process_record(i, None)]
+ result.extend([r async for r in add.emit_outstanding_records()])
assert_that(result, contains_inanyorder(*[i + 1 for i in items]))
assert_that(result, has_length(len(items)))
@@ -53,7 +44,7 @@ async def input_record_stream():
How do we know?
If it hogs the processor the downstream client will only recieve the results in bulk
-If it doesn't the downstream client will recieve the result in aync pieces.
+If it doesn't the downstream client will recieve the result in aync pieces.
Test:
A transformer ingests data from a mock input stream
@@ -62,44 +53,15 @@ async def input_record_stream():
"""
-@pytest.mark.asyncio
-async def test_greedy_concurrent_transformer_does_not_pass_processor():
- items = list(range(ITEM_COUNT))
- transformer = AddOneConcurrentlyGreedy()
-
- async def input_record_stream():
- for i in items:
- yield i
-
- async def transform():
- async for _ in transformer.handle_async_record_stream(input_record_stream()):
- transformer.queue_size += 1
- transformer.done = True
-
- async def downstream_client():
- should_continue = True
- while should_continue:
- if transformer.queue_size > 0:
- assert transformer.queue_size >= ITEM_COUNT
- transformer.queue_size -= 1
- should_continue = not transformer.done
- await asyncio.sleep(0)
-
- await asyncio.gather(transform(), downstream_client())
-
-
@pytest.mark.asyncio
async def test_normal_concurrent_transformer_passes_processor():
- items = list(range(ITEM_COUNT))
+ items = list(range(ITEM_COUNT)) + [Flush]
transformer = AddOneConcurrently()
- async def input_record_stream():
- for i in items:
- yield i
-
async def transform():
- async for _ in transformer.handle_async_record_stream(input_record_stream()):
- transformer.queue_size += 1
+ for i in items:
+ async for _ in transformer.process_record(i, None):
+ transformer.queue_size += 1
transformer.done = True
async def downstream_client():
@@ -117,19 +79,17 @@ async def downstream_client():
async def test_concurrent_transformer_worker_cleanup(mocker):
add = AddOneConcurrently()
add.thread_pool = mocker.Mock()
- await add.finish()
- add.thread_pool.shutdown.assert_called_once_with(wait=True)
+ await add.finish(None)
+ add.thread_pool.shutdown.assert_called_once_with(True)
@pytest.mark.asyncio
async def test_concurrent_transformer_flush(mocker):
- async def input_record_stream():
- yield 1
- yield Flush
- yield 2
-
add = AddOneConcurrently()
- result = [r async for r in add.handle_async_record_stream(input_record_stream())]
+ result = [
+ r for record in (1, Flush, 2) async for r in add.process_record(record, None)
+ ]
+ result.extend([r async for r in add.emit_outstanding_records()])
assert_that(result, contains_inanyorder(2, 3, Flush))
@@ -176,11 +136,6 @@ async def transform_record(self, record):
]
-async def switch_input_record_stream():
- for record in TEST_DATA:
- yield record
-
-
@pytest.mark.asyncio
async def test_switch_transformer_with_default():
switch_transformer = SwitchTransformer.from_file_data(
@@ -189,9 +144,8 @@ async def test_switch_transformer_with_default():
results = [
r
- async for r in switch_transformer.handle_async_record_stream(
- switch_input_record_stream()
- )
+ for record in TEST_DATA
+ async for r in switch_transformer.process_record(record, None)
]
assert results == TEST_RESULTS_WITH_DEFAULT
@@ -203,8 +157,7 @@ async def test_switch_transformer_without_default():
)
results = [
r
- async for r in switch_transformer.handle_async_record_stream(
- switch_input_record_stream()
- )
+ for record in TEST_DATA
+ async for r in switch_transformer.process_record(record, None)
]
assert results == TEST_RESULTS_WITH_NO_DEFAULT
diff --git a/tests/unit/pipeline/transformers/test_value_projection.py b/tests/unit/pipeline/transformers/test_value_projection.py
index 0d2899e8d..d17a74301 100644
--- a/tests/unit/pipeline/transformers/test_value_projection.py
+++ b/tests/unit/pipeline/transformers/test_value_projection.py
@@ -16,7 +16,7 @@ async def test_value_projection_transform_record():
subject = ValueProjection(
projection=JmespathValueProvider.from_string_expression("items[*]")
)
- results = [r async for r in subject.handle_async_record_stream(record())]
+ results = [r async for r in subject.process_record(TEST_DATA, None)]
assert_that(results, equal_to([{"index": 1}, {"index": 2}, {"index": 3}]))
@@ -29,7 +29,7 @@ async def test_value_projection_with_additional_values():
},
)
- results = [r async for r in subject.handle_async_record_stream(record())]
+ results = [r async for r in subject.process_record(TEST_DATA, None)]
assert_that(
results,
equal_to(