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(