From 9c34772bd1ce6b6b080f2e4967afd71a7064791b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:07:29 +0100 Subject: [PATCH 01/88] add component checks --- haystack/core/pipeline/component_checks.py | 205 +++++++++++++++++++++ 1 file changed, 205 insertions(+) create mode 100644 haystack/core/pipeline/component_checks.py diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py new file mode 100644 index 0000000000..3079878ad3 --- /dev/null +++ b/haystack/core/pipeline/component_checks.py @@ -0,0 +1,205 @@ +# SPDX-FileCopyrightText: 2022-present deepset GmbH +# +# SPDX-License-Identifier: Apache-2.0 + +from typing import Any, Dict, List + +from haystack.core.component.types import _empty, InputSocket + +_NO_OUTPUT_PRODUCED = _empty + +def can_component_run(component: Dict, inputs: Dict) -> bool: + """ + Checks if the component can run, given the current state of its inputs. + A component needs to pass two gates so that it is ready to run: + 1. It has received all mandatory inputs. + 2. It has received a trigger. + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + received_all_mandatory_inputs = are_all_mandatory_sockets_ready(component, inputs) + received_trigger = has_any_trigger(component, inputs) + + return received_all_mandatory_inputs and received_trigger + +def has_any_trigger(component: Dict, inputs: Dict) -> bool: + """ + Checks if a component was triggered to execute. + + There are 3 triggers: + 1. A predecessor provided input to the component. + 2. Input to the component was provided from outside the pipeline (e.g. user input). + 3. The component does not receive input from any other components in the pipeline and `Pipeline.run` was called. + + A trigger can only cause a component to execute ONCE because: + 1. Components consume inputs from predecessors before execution (they are deleted). + 2. Inputs from outside the pipeline can only trigger a component when it is executed for the first time. + 3. `Pipeline.run` can only trigger a component when it is executed for the first time. + + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + trigger_from_predecessor = any_predecessors_provided_input(component, inputs) + trigger_from_user = has_user_input(inputs) and component["visits"] == 0 + trigger_without_inputs = can_not_receive_inputs_from_pipeline(component) and component["visits"] == 0 + + return trigger_from_predecessor or trigger_from_user or trigger_without_inputs + +def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: + """ + Checks if all mandatory sockets of a component have enough inputs for the component to execute. + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + filled_mandatory_sockets = set() + expected_mandatory_sockets = set() + for socket_name, socket in component["input_sockets"].items(): + if socket.is_mandatory: + socket_inputs = inputs.get(socket_name, []) + expected_mandatory_sockets.add(socket_name) + if is_socket_lazy_variadic(socket) and any_socket_input_received(socket_inputs): + filled_mandatory_sockets.add(socket_name) + elif has_socket_received_all_inputs(socket, socket_inputs): + filled_mandatory_sockets.add(socket_name) + + return filled_mandatory_sockets == expected_mandatory_sockets + +def any_predecessors_provided_input(component: Dict, inputs: Dict) -> bool: + """ + Checks if a component received inputs from any predecessors. + + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + return any(any_socket_value_from_predecessor_received(inputs.get(socket_name, [])) for socket_name in component["input_sockets"].keys()) + +def any_socket_value_from_predecessor_received(socket_inputs: List[Dict[str, Any]]) -> bool: + """ + Checks if a component socket received input from any predecessors. + :param socket_inputs: Inputs for the component's socket. + """ + # When sender is None, the input was provided from outside the pipeline. + return any(inp["value"] != _NO_OUTPUT_PRODUCED and inp["sender"] is not None for inp in socket_inputs) + +def has_user_input(inputs: Dict) -> bool: + """ + Checks if a component has received input from outside the pipeline (e.g. user input). + :param inputs: Inputs for the component. + """ + return any(inp for socket in inputs.values() for inp in socket if inp["sender"] is None) + +def can_not_receive_inputs_from_pipeline(component: Dict) -> bool: + """ + Checks if a component can not receive inputs from any other components in the pipeline. + :param: Component metadata and the component instance. + """ + return all([len(sock.senders) == 0 for sock in component["input_sockets"].values()]) + +def all_socket_predecessors_executed(socket: InputSocket, socket_inputs: List[Dict]) -> bool: + """ + Checks if all components connecting to an InputSocket have executed. + :param: The InputSocket of a component. + :param: socket_inputs: Inputs for the socket. + """ + expected_senders = set(socket.senders) + executed_senders = {inp["sender"] for inp in socket_inputs if inp["sender"] is not None} + + return expected_senders == executed_senders + +def any_socket_input_received(socket_inputs: List[Dict]) -> bool: + """ + Checks if a socket has received any input from any other components in the pipeline or from outside the pipeline. + :param socket_inputs: Inputs for the socket. + """ + return any(inp["value"] != _NO_OUTPUT_PRODUCED for inp in socket_inputs) + +def has_lazy_variadic_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict]) -> bool: + """ + Checks if a lazy variadic socket has received all expected inputs from other components in the pipeline. + :param socket: The InputSocket of a component. + :param socket_inputs: Inputs for the socket. + """ + expected_senders = set(socket.senders) + actual_senders = { + sock["sender"] for sock in socket_inputs if sock["value"] != _NO_OUTPUT_PRODUCED and sock["sender"] is not None + } + + return expected_senders == actual_senders + +def is_socket_lazy_variadic(socket: InputSocket) -> bool: + """ + Checks if an InputSocket is a lazy variadic socket. + :param socket: The InputSocket of a component. + """ + return socket.is_variadic and not socket.is_greedy + +def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict]) -> bool: + """ + Checks if a socket has received all expected inputs. + :param socket: The InputSocket of a component. + :param socket_inputs: Inputs for the socket. + """ + # No inputs received for the socket, it is not filled. + if len(socket_inputs) == 0: + return False + + # The socket is greedy variadic and at least one input was produced, it is complete. + if ( + socket.is_variadic + and socket.is_greedy + and len(socket_inputs) > 0 + and any([sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs]) + ): + return True + + + # The socket is lazy variadic and all expected inputs were produced. + if is_socket_lazy_variadic(socket) and has_lazy_variadic_socket_received_all_inputs(socket, socket_inputs): + return True + + # The socket is not variadic and the only expected input is complete. + if not socket.is_variadic and socket_inputs[0]["value"] != _NO_OUTPUT_PRODUCED: + return True + + return False + +def all_predecessors_executed(component: Dict, inputs: Dict) -> bool: + """ + Checks if all predecessors of a component have executed. + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + return all( + all_socket_predecessors_executed(socket, inputs.get(socket_name, [])) + for socket_name, socket in component["input_sockets"].items() + ) + +def are_all_lazy_variadic_sockets_resolved(component: Dict, inputs: Dict) -> bool: + """ + Checks if the final state for all lazy variadic sockets of a component is resolved. + Either because all inputs were received, or because all predecessors executed. + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + for socket_name, socket in component["input_sockets"].items(): + if is_socket_lazy_variadic(socket): + socket_inputs = inputs.get(socket_name, []) + if not ( + has_lazy_variadic_socket_received_all_inputs(socket, socket_inputs) + or all_socket_predecessors_executed(socket, socket_inputs) + ): + return False + + return True + +def is_any_greedy_socket_ready(component: Dict, inputs: Dict) -> bool: + """ + Checks if the component has any greedy socket that is ready to run. + :param component: Component metadata and the component instance. + :param inputs: Inputs for the component. + """ + for socket_name, socket in component["input_sockets"].items(): + if socket.is_greedy and has_socket_received_all_inputs(socket, inputs.get(socket_name, [])): + return True + + return False From fc2f2da3e48bdaf483b87b6bec0cd2b797e69e06 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:11:07 +0100 Subject: [PATCH 02/88] pipeline should run deterministically --- haystack/core/pipeline/pipeline.py | 634 ++++++++++++----------------- 1 file changed, 260 insertions(+), 374 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 622c4ef6d7..fdfc90d46a 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -3,26 +3,28 @@ # SPDX-License-Identifier: Apache-2.0 from copy import deepcopy -from typing import Any, Dict, List, Mapping, Optional, Set, Tuple -from warnings import warn - -import networkx as nx +from typing import Any, Dict, List, Mapping, Optional, Set, Tuple, Union +from enum import IntEnum from haystack import logging, tracing from haystack.core.component import Component from haystack.core.errors import PipelineMaxComponentRuns, PipelineRuntimeError -from haystack.core.pipeline.base import ( - _dequeue_component, - _dequeue_waiting_component, - _enqueue_component, - _enqueue_waiting_component, -) -from haystack.telemetry import pipeline_running +from haystack.core.pipeline.base import PipelineBase +from haystack.core.pipeline.component_checks import can_component_run, is_any_greedy_socket_ready, \ + is_socket_lazy_variadic +from haystack.core.pipeline.component_checks import all_predecessors_executed, are_all_lazy_variadic_sockets_resolved, _NO_OUTPUT_PRODUCED +from haystack.core.pipeline.utils import FIFOPriorityQueue -from .base import PipelineBase, _add_missing_input_defaults, _is_lazy_variadic +from haystack.telemetry import pipeline_running logger = logging.getLogger(__name__) +class ComponentPriority(IntEnum): + HIGHEST = 1 + READY = 2 + DEFER = 3 + DEFER_LAST = 4 + BLOCKED = 5 class Pipeline(PipelineBase): """ @@ -32,24 +34,24 @@ class Pipeline(PipelineBase): """ def _run_component( - self, name: str, inputs: Dict[str, Any], parent_span: Optional[tracing.Span] = None - ) -> Dict[str, Any]: + self, component: Dict[str, Any], inputs: Dict[str, Any], parent_span: Optional[tracing.Span] = None + ) -> Tuple[Dict, Dict]: """ Runs a Component with the given inputs. - :param name: Name of the Component as defined in the Pipeline. + :param component: Component with component metadata. :param inputs: Inputs for the Component. :param parent_span: The parent span to use for the newly created span. This is to allow tracing to be correctly linked to the pipeline run. :raises PipelineRuntimeError: If Component doesn't return a dictionary. - :return: The output of the Component. + :return: The output of the Component and the new state of inputs. """ - instance: Component = self.graph.nodes[name]["instance"] - + instance: Component = component["instance"] + component_name = self.get_component_name(instance) with tracing.tracer.trace( "haystack.component.run", tags={ - "haystack.component.name": name, + "haystack.component.name": component_name, "haystack.component.type": instance.__class__.__name__, "haystack.component.input_types": {k: type(v).__name__ for k, v in inputs.items()}, "haystack.component.input_spec": { @@ -69,193 +71,217 @@ def _run_component( }, parent_span=parent_span, ) as span: + component_inputs, inputs = self._consume_component_inputs( + component_name=component_name, + component=component, + inputs=inputs + ) # We deepcopy the inputs otherwise we might lose that information # when we delete them in case they're sent to other Components - span.set_content_tag("haystack.component.input", deepcopy(inputs)) - logger.info("Running component {component_name}", component_name=name) - res: Dict[str, Any] = instance.run(**inputs) - self.graph.nodes[name]["visits"] += 1 - - # After a Component that has variadic inputs is run, we need to reset the variadic inputs that were consumed - for socket in instance.__haystack_input__._sockets_dict.values(): # type: ignore - if socket.name not in inputs: - continue - if socket.is_variadic: - inputs[socket.name] = [] - - if not isinstance(res, Mapping): + span.set_content_tag("haystack.component.input", deepcopy(component_inputs)) + logger.info("Running component {component_name}", component_name=component_name) + component_output = instance.run(**component_inputs) + component["visits"] += 1 + + if not isinstance(component_output, Mapping): raise PipelineRuntimeError( - f"Component '{name}' didn't return a dictionary. " + f"Component '{component_name}' didn't return a dictionary. " "Components must always return dictionaries: check the documentation." ) - span.set_tag("haystack.component.visits", self.graph.nodes[name]["visits"]) - span.set_content_tag("haystack.component.output", res) - - return res - - def _run_subgraph( # noqa: PLR0915 - self, - cycle: List[str], - component_name: str, - components_inputs: Dict[str, Dict[str, Any]], - *, - include_outputs_from: Optional[Set[str]] = None, - parent_span: Optional[tracing.Span] = None, - ) -> Tuple[Dict[str, Any], Dict[str, Any]]: + + span.set_tag("haystack.component.visits", component["visits"]) + span.set_content_tag("haystack.component.output", component_output) + + return component_output, inputs + + @staticmethod + def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict) -> Tuple[Dict, Dict]: + """ + Extracts the inputs needed to run for the component and removes them from the global inputs state. + :param component: Component with component metadata. + :param inputs: Global inputs state. + :returns: The inputs for the component and the new state of global inputs. + """ + component_inputs = inputs.get(component_name, {}) + consumed_inputs = {} + greedy_inputs_to_remove = set() + for socket_name, socket in component["input_sockets"].items(): + socket_inputs = component_inputs.get(socket_name, []) + socket_inputs = [sock["value"] for sock in socket_inputs if sock["value"] != _NO_OUTPUT_PRODUCED] + if socket_inputs: + if not socket.is_variadic: + # We only care about the first input provided to the socket. + consumed_inputs[socket_name] = socket_inputs[0] + elif socket.is_greedy: + # We need to keep track of greedy inputs because we always remove them, even if they come from + # outside the pipeline. Otherwise, a greedy input from the user would trigger a pipeline to run + # indefinitely. + greedy_inputs_to_remove.add(socket_name) + consumed_inputs[socket_name] = [socket_inputs[0]] + elif is_socket_lazy_variadic(socket): + # We use all inputs provided to the socket on a lazy variadic socket. + consumed_inputs[socket_name] = socket_inputs + + # We prune all inputs except for those that were provided from outside the pipeline (e.g. user inputs). + pruned_inputs = { + socket_name: [ + sock for sock in socket if sock["sender"] is None + and not socket_name in greedy_inputs_to_remove + ] for socket_name, socket in component_inputs.items() + } + pruned_inputs = {socket_name: socket for socket_name, socket in pruned_inputs.items() if len(socket) > 0} + + inputs[component_name] = pruned_inputs + + return consumed_inputs, inputs + + @staticmethod + def _convert_from_legacy_format(pipeline_inputs: Dict[str, Any]) -> Dict[str, Dict[str, List]]: """ - Runs a `cycle` in the Pipeline starting from `component_name`. + Converts the inputs to the pipeline to the format that is needed for the internal `Pipeline.run` logic. + :param pipeline_inputs: Inputs to the pipeline. + :returns: Converted inputs that can be used by the internal `Pipeline.run` logic. + """ + inputs = {} + for component_name, socket_dict in pipeline_inputs.items(): + inputs[component_name] = {} + for socket_name, value in socket_dict.items(): + inputs[component_name][socket_name] = [{"sender": None, "value": value}] - This will return once there are no inputs for the Components in `cycle`. + return inputs - This is an internal method meant to be used in `Pipeline.run()` only. + def _fill_queue(self, component_names: List[str], inputs: Dict[str, Any]) -> FIFOPriorityQueue: + """ + Calculates the execution priority for each component and inserts it into the priority queue. - :param cycle: - List of Components that are part of the cycle being run - :param component_name: - Name of the Component that will start execution of the cycle - :param components_inputs: - Components inputs, this might include inputs for Components that are not part - of the cycle but part of the wider Pipeline's graph - :param include_outputs_from: - Set of component names whose individual outputs are to be - included in the cycle's output. In case a Component is executed multiple times - only the last-produced output is included. - :returns: - Outputs of all the Components that are not connected to other Components in `cycle`. - If `include_outputs_from` is set those Components' outputs will be included. - :raises PipelineMaxComponentRuns: - If a Component reaches the maximum number of times it can be run in this Pipeline + :param component_names: Names of the components to put into the queue. + :param inputs: Inputs to the components. + :returns: A prioritized queue of component names. + """ + priority_queue = FIFOPriorityQueue() + for component_name in component_names: + component = self._get_component_with_graph_metadata(component_name) + priority = self._calculate_priority(component, inputs.get(component_name, {})) + priority_queue.push(component_name, priority) + + return priority_queue + + @staticmethod + def _calculate_priority(component: Dict, inputs: Dict) -> ComponentPriority: + """ + Calculates the execution priority for a component depending on the component's inputs. + :param component: Component metadata and component instance. + :param inputs: Inputs to the component. + :returns: Priority value for the component. """ - waiting_queue: List[Tuple[str, Component]] = [] - run_queue: List[Tuple[str, Component]] = [] - - # Create the run queue starting with the component that needs to run first - start_index = cycle.index(component_name) - for node in cycle[start_index:]: - run_queue.append((node, self.graph.nodes[node]["instance"])) - - include_outputs_from = set() if include_outputs_from is None else include_outputs_from - - before_last_waiting_queue: Optional[Set[str]] = None - last_waiting_queue: Optional[Set[str]] = None - - subgraph_outputs = {} - # These are outputs that are sent to other Components but the user explicitly - # asked to include them in the final output. - extra_outputs = {} - - # This variable is used to keep track if we still need to run the cycle or not. - # When a Component doesn't send outputs to another Component - # that's inside the subgraph, we stop running this subgraph. - cycle_received_inputs = False - - while not cycle_received_inputs: - # Here we run the Components - name, comp = run_queue.pop(0) - if _is_lazy_variadic(comp) and not all(_is_lazy_variadic(comp) for _, comp in run_queue): - # We run Components with lazy variadic inputs only if there only Components with - # lazy variadic inputs left to run - _enqueue_waiting_component((name, comp), waiting_queue) - continue - - # As soon as a Component returns only output that is not part of the cycle, we can stop - if self._component_has_enough_inputs_to_run(name, components_inputs): - if self.graph.nodes[name]["visits"] > self._max_runs_per_component: - msg = f"Maximum run count {self._max_runs_per_component} reached for component '{name}'" - raise PipelineMaxComponentRuns(msg) - - res: Dict[str, Any] = self._run_component(name, components_inputs[name], parent_span=parent_span) - - # Delete the inputs that were consumed by the Component and are not received from - # the user or from Components that are part of this cycle - sockets = list(components_inputs[name].keys()) - for socket_name in sockets: - senders = comp.__haystack_input__._sockets_dict[socket_name].senders # type: ignore - if not senders: - # We keep inputs that came from the user - continue - all_senders_in_cycle = all(sender in cycle for sender in senders) - if all_senders_in_cycle: - # All senders are in the cycle, we can remove the input. - # We'll receive it later at a certain point. - del components_inputs[name][socket_name] - - if name in include_outputs_from: - # Deepcopy the outputs to prevent downstream nodes from modifying them - # We don't care about loops - Always store the last output. - extra_outputs[name] = deepcopy(res) - - # Reset the waiting for input previous states, we managed to run a component - before_last_waiting_queue = None - last_waiting_queue = None - - # Check if a component doesn't send any output to components that are part of the cycle - final_output_reached = False - for output_socket in res.keys(): - for receiver in comp.__haystack_output__._sockets_dict[output_socket].receivers: # type: ignore - if receiver in cycle: - final_output_reached = True - break - if final_output_reached: - break - - if not final_output_reached: - # We stop only if the Component we just ran doesn't send any output to sockets that - # are part of the cycle - cycle_received_inputs = True - - # We manage to run this component that was in the waiting list, we can remove it. - # This happens when a component was put in the waiting list but we reached it from another edge. - _dequeue_waiting_component((name, comp), waiting_queue) - for pair in self._find_components_that_will_receive_no_input(name, res, components_inputs): - _dequeue_component(pair, run_queue, waiting_queue) - - receivers = [item for item in self._find_receivers_from(name) if item[0] in cycle] - - res = self._distribute_output(receivers, res, components_inputs, run_queue, waiting_queue) - - # We treat a cycle as a completely independent graph, so we keep track of output - # that is not sent inside the cycle. - # This output is going to get distributed to the wider graph after we finish running - # a cycle. - # All values that are left at this point go outside the cycle. - if len(res) > 0: - subgraph_outputs[name] = res + if not can_component_run(component, inputs): + return ComponentPriority.BLOCKED + elif is_any_greedy_socket_ready(component, inputs): + return ComponentPriority.HIGHEST + elif all_predecessors_executed(component, inputs): + return ComponentPriority.READY + elif are_all_lazy_variadic_sockets_resolved(component, inputs): + return ComponentPriority.DEFER + else: + return ComponentPriority.DEFER_LAST + + def _get_component_with_graph_metadata(self, component_name: str) -> Dict[str, Any]: + return self.graph.nodes[component_name] + + def _get_next_runnable_component(self, priority_queue: FIFOPriorityQueue) -> Union[Tuple[Component, str, Dict], None]: + """ + Returns the next runnable component alongside its metadata from the priority queue. + :param priority_queue: Priority queue of component names. + :returns: The next runnable component, the component name, and its priority or None if no component in the queue can run. + :raises: PipelineMaxComponentRuns if the next runnable component has exceeded the maximum number of runs. + """ + priority_and_component_name = priority_queue.get() + + if priority_and_component_name is not None and priority_and_component_name[0] != ComponentPriority.BLOCKED: + priority, component_name = priority_and_component_name + component = self._get_component_with_graph_metadata(component_name) + if component["visits"] > self._max_runs_per_component: + msg = f"Maximum run count {self._max_runs_per_component} reached for component '{component_name}'" + raise PipelineMaxComponentRuns(msg) + + return priority, component_name, component + + return None + + @staticmethod + def _write_component_outputs(component_name, component_outputs, inputs, receivers, include_outputs_from) -> Tuple[Dict, Dict]: + """ + Distributes the outputs of a component to the input sockets that it is connected to. + :param component_name: The name of the component. + :param component_outputs: The outputs of the component. + :param inputs: The current global input state. + :param receivers: List of receiver_name, sender_socket, receiver_socket for connected components. + :param include_outputs_from: List of component names that should always return an output from the pipeline. + """ + for receiver_name, sender_socket, receiver_socket in receivers: + # We either get the value that was produced by the actor or we use the _NO_OUTPUT_PRODUCED class to indicate + # that the sender did not produce an output for this socket. + # This allows us to track if a pre-decessor already ran but did not produce an output. + value = component_outputs.get(sender_socket.name, _NO_OUTPUT_PRODUCED) + if receiver_name not in inputs: + inputs[receiver_name] = {} + + # If we have a non-variadic or a greedy variadic receiver socket, we can just overwrite any inputs + # that might already exist (to be reconsidered but mirrors current behavior). + if not is_socket_lazy_variadic(receiver_socket): + inputs[receiver_name][receiver_socket.name] = [{"sender": component_name, "value": value}] + + # If the receiver socket is lazy variadic, and it already has an input, we need to append the new input. + # Lazy variadic sockets can collect multiple inputs. else: - # This component doesn't have enough inputs so we can't run it yet - _enqueue_waiting_component((name, comp), waiting_queue) - - if len(run_queue) == 0 and len(waiting_queue) > 0: - # Check if we're stuck in a loop. - # It's important to check whether previous waitings are None as it could be that no - # Component has actually been run yet. - if ( - before_last_waiting_queue is not None - and last_waiting_queue is not None - and before_last_waiting_queue == last_waiting_queue - ): - if self._is_stuck_in_a_loop(waiting_queue): - # We're stuck! We can't make any progress. - msg = ( - "Pipeline is stuck running in a loop. Partial outputs will be returned. " - "Check the Pipeline graph for possible issues." - ) - warn(RuntimeWarning(msg)) - break - - (name, comp) = self._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - _add_missing_input_defaults(name, comp, components_inputs) - _enqueue_component((name, comp), run_queue, waiting_queue) - continue - - before_last_waiting_queue = last_waiting_queue.copy() if last_waiting_queue is not None else None - last_waiting_queue = {item[0] for item in waiting_queue} - - (name, comp) = self._find_next_runnable_component(components_inputs, waiting_queue) - _add_missing_input_defaults(name, comp, components_inputs) - _enqueue_component((name, comp), run_queue, waiting_queue) - - return subgraph_outputs, extra_outputs + if not inputs[receiver_name].get(receiver_socket.name): + inputs[receiver_name][receiver_socket.name] = [] + + inputs[receiver_name][receiver_socket.name].append({"sender": component_name, "value": value}) + + # If we want to include all outputs from this actor in the final outputs, we don't need to prune any consumed + # outputs + if component_name in include_outputs_from: + return component_outputs, inputs + + # We prune outputs that were consumed by any receiving sockets. + # All remaining outputs will be added to the final outputs of the pipeline. + consumed_outputs = {sender_socket.name for _, sender_socket, __ in receivers} + pruned_outputs = {key: value for key, value in component_outputs.items() if key not in consumed_outputs} + + return pruned_outputs, inputs + + @staticmethod + def _merge_component_and_pipeline_outputs(component_name: str, component_outputs: Dict, pipeline_outputs: Dict) -> Dict: + """ + Merges the outputs of a component with the current pipeline outputs. + :param component_name: The name of the component. + :param component_outputs: The outputs of the component. + :param pipeline_outputs: The pipeline outputs. + :returns: New pipeline outputs. + """ + if not component_outputs: + return pipeline_outputs + elif component_name not in pipeline_outputs: + pipeline_outputs[component_name] = component_outputs + else: + for key, value in component_outputs.items(): + if key not in pipeline_outputs[component_name]: + pipeline_outputs[component_name][key] = value + + return pipeline_outputs + + @staticmethod + def _is_queue_stale(priority_queue: FIFOPriorityQueue) -> bool: + """ + Checks if the priority queue needs to be recomputed because the priorities might have changed. + :param priority_queue: Priority queue of component names. + """ + next_priority_and_component = priority_queue and priority_queue.peek() + if not next_priority_and_component or next_priority_and_component[0] > ComponentPriority.READY: + return True + + return False def run( # noqa: PLR0915, PLR0912 self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None @@ -362,191 +388,51 @@ def run( # noqa: PLR0915, PLR0912 # Raise if input is malformed in some way self._validate_input(data) - # Normalize the input data - components_inputs: Dict[str, Dict[str, Any]] = self._normalize_varidiac_input_data(data) - - # These variables are used to detect when we're stuck in a loop. - # Stuck loops can happen when one or more components are waiting for input but - # no other component is going to run. - # This can happen when a whole branch of the graph is skipped for example. - # When we find that two consecutive iterations of the loop where the waiting_queue is the same, - # we know we're stuck in a loop and we can't make any progress. - # - # They track the previous two states of the waiting_queue. So if waiting_queue would n, - # before_last_waiting_queue would be n-2 and last_waiting_queue would be n-1. - # When we run a component, we reset both. - before_last_waiting_queue: Optional[Set[str]] = None - last_waiting_queue: Optional[Set[str]] = None - - # The waiting_for_input list is used to keep track of components that are waiting for input. - waiting_queue: List[Tuple[str, Component]] = [] - - include_outputs_from = set() if include_outputs_from is None else include_outputs_from - - # This is what we'll return at the end - final_outputs: Dict[Any, Any] = {} - - # Break cycles in case there are, this is a noop if no cycle is found. - # This will raise if a cycle can't be broken. - graph_without_cycles, components_in_cycles = self._break_supported_cycles_in_graph() - - run_queue: List[Tuple[str, Component]] = [] - for node in nx.topological_sort(graph_without_cycles): - run_queue.append((node, self.graph.nodes[node]["instance"])) - - # Set defaults inputs for those sockets that don't receive input neither from the user - # nor from other Components. - # If they have no default nothing is done. - # This is important to ensure correct order execution, otherwise some variadic - # Components that receive input from the user might be run before than they should. - for name, comp in self.graph.nodes(data="instance"): - if name not in components_inputs: - components_inputs[name] = {} - for socket_name, socket in comp.__haystack_input__._sockets_dict.items(): - if socket_name in components_inputs[name]: - continue - if not socket.senders: - value = socket.default_value - if socket.is_variadic: - value = [value] - components_inputs[name][socket_name] = value + if include_outputs_from is None: + include_outputs_from = set() + + # We create a list of components in the pipeline sorted by name, so that the algorithm runs deterministically + # and independent of insertion order into the pipeline. + ordered_component_names = sorted(self.graph.nodes.keys()) + # We need to access a component's receivers multiple times during a pipeline run. + # We store them here for easy access. + cached_receivers = {name: self._find_receivers_from(name) for name in ordered_component_names} + + pipeline_outputs: Dict[str, Any] = {} with tracing.tracer.trace( "haystack.pipeline.run", tags={ "haystack.pipeline.input_data": data, - "haystack.pipeline.output_data": final_outputs, + "haystack.pipeline.output_data": pipeline_outputs, "haystack.pipeline.metadata": self.metadata, "haystack.pipeline.max_runs_per_component": self._max_runs_per_component, }, ) as span: - # Cache for extra outputs, if enabled. - extra_outputs: Dict[Any, Any] = {} - - while len(run_queue) > 0: - name, comp = run_queue.pop(0) - - if _is_lazy_variadic(comp) and not all(_is_lazy_variadic(comp) for _, comp in run_queue): - # We run Components with lazy variadic inputs only if there only Components with - # lazy variadic inputs left to run - _enqueue_waiting_component((name, comp), waiting_queue) - continue - if self._component_has_enough_inputs_to_run(name, components_inputs) and components_in_cycles.get( - name, [] - ): - cycles = components_in_cycles.get(name, []) - - # This component is part of one or more cycles, let's get the first one and run it. - # We can reliably pick any of the cycles if there are multiple ones, the way cycles - # are run doesn't make a different whether we pick the first or any of the others a - # Component is part of. - subgraph_output, subgraph_extra_output = self._run_subgraph( - cycles[0], name, components_inputs, include_outputs_from=include_outputs_from, parent_span=span - ) - - # After a cycle is run the previous run_queue can't be correct anymore cause it's - # not modified when running the subgraph. - # So we reset it given the output returned by the subgraph. - run_queue = [] - - # Reset the waiting for input previous states, we managed to run at least one component - before_last_waiting_queue = None - last_waiting_queue = None - - # Merge the extra outputs - extra_outputs.update(subgraph_extra_output) - - for component_name, component_output in subgraph_output.items(): - receivers = self._find_receivers_from(component_name) - component_output = self._distribute_output( - receivers, component_output, components_inputs, run_queue, waiting_queue - ) - - if len(component_output) > 0: - final_outputs[component_name] = component_output - - elif self._component_has_enough_inputs_to_run(name, components_inputs): - if self.graph.nodes[name]["visits"] > self._max_runs_per_component: - msg = f"Maximum run count {self._max_runs_per_component} reached for component '{name}'" - raise PipelineMaxComponentRuns(msg) - - res: Dict[str, Any] = self._run_component(name, components_inputs[name], parent_span=span) - - # Delete the inputs that were consumed by the Component and are not received from the user - sockets = list(components_inputs[name].keys()) - for socket_name in sockets: - senders = comp.__haystack_input__._sockets_dict[socket_name].senders - if senders: - # Delete all inputs that are received from other Components - del components_inputs[name][socket_name] - # We keep inputs that came from the user - - if name in include_outputs_from: - # Deepcopy the outputs to prevent downstream nodes from modifying them - # We don't care about loops - Always store the last output. - extra_outputs[name] = deepcopy(res) - - # Reset the waiting for input previous states, we managed to run a component - before_last_waiting_queue = None - last_waiting_queue = None - - # We manage to run this component that was in the waiting list, we can remove it. - # This happens when a component was put in the waiting list but we reached it from another edge. - _dequeue_waiting_component((name, comp), waiting_queue) - - for pair in self._find_components_that_will_receive_no_input(name, res, components_inputs): - _dequeue_component(pair, run_queue, waiting_queue) - receivers = self._find_receivers_from(name) - res = self._distribute_output(receivers, res, components_inputs, run_queue, waiting_queue) - - if len(res) > 0: - final_outputs[name] = res - else: - # This component doesn't have enough inputs so we can't run it yet - _enqueue_waiting_component((name, comp), waiting_queue) - - if len(run_queue) == 0 and len(waiting_queue) > 0: - # Check if we're stuck in a loop. - # It's important to check whether previous waitings are None as it could be that no - # Component has actually been run yet. - if ( - before_last_waiting_queue is not None - and last_waiting_queue is not None - and before_last_waiting_queue == last_waiting_queue - ): - if self._is_stuck_in_a_loop(waiting_queue): - # We're stuck! We can't make any progress. - msg = ( - "Pipeline is stuck running in a loop. Partial outputs will be returned. " - "Check the Pipeline graph for possible issues." - ) - warn(RuntimeWarning(msg)) - break - - (name, comp) = self._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - _add_missing_input_defaults(name, comp, components_inputs) - _enqueue_component((name, comp), run_queue, waiting_queue) - continue - - before_last_waiting_queue = last_waiting_queue.copy() if last_waiting_queue is not None else None - last_waiting_queue = {item[0] for item in waiting_queue} - - (name, comp) = self._find_next_runnable_component(components_inputs, waiting_queue) - _add_missing_input_defaults(name, comp, components_inputs) - _enqueue_component((name, comp), run_queue, waiting_queue) - - if len(include_outputs_from) > 0: - for name, output in extra_outputs.items(): - inner = final_outputs.get(name) - if inner is None: - final_outputs[name] = output - else: - # Let's not override any keys that are already - # in the final_outputs as they might be different - # from what we cached in extra_outputs, e.g. when loops - # are involved. - for k, v in output.items(): - if k not in inner: - inner[k] = v - - return final_outputs + inputs = self._convert_from_legacy_format(pipeline_inputs=data) + + priority_queue = self._fill_queue(ordered_component_names, inputs) + + while True: + candidate = self._get_next_runnable_component(priority_queue) + if candidate is None: + break + + priority, component_name, component = candidate + component_outputs, inputs = self._run_component(component, inputs, parent_span=span) + component_pipeline_outputs, inputs = self._write_component_outputs( + component_name=component_name, + component_outputs=component_outputs, + inputs=inputs, + receivers=cached_receivers[component_name], + include_outputs_from=include_outputs_from + ) + # TODO check original logic in pipeline, it looks like we don't want to override existing outputs + # e.g. for cycles but the tests check if intermediate outputs from components in cycles are overwritten + # pipeline_outputs = self._merge_component_and_pipeline_outputs(component_name, component_pipeline_outputs, pipeline_outputs) + if component_pipeline_outputs: + pipeline_outputs = {**pipeline_outputs, component_name: component_pipeline_outputs} + if self._is_queue_stale(priority_queue): + priority_queue = self._fill_queue(ordered_component_names, inputs) + + return pipeline_outputs From c28353db5ae10699e327520188b88ba98bbe9e02 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:11:52 +0100 Subject: [PATCH 03/88] add FIFOQueue --- haystack/core/pipeline/utils.py | 46 +++++++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index f9f858a39d..5950296ab3 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -2,6 +2,9 @@ # # SPDX-License-Identifier: Apache-2.0 +import heapq + +from itertools import count from typing import Optional, Tuple @@ -18,3 +21,46 @@ def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: split_str = connection.split(".", maxsplit=1) return (split_str[0], split_str[1]) return connection, None + +class FIFOPriorityQueue: + """ + A priority queue that maintains FIFO order for items of equal priority. + Items with the same priority are placed to the right of existing items. + """ + + def __init__(self): + self._queue = [] # list of tuples (priority, count, item) + self._counter = count() # unique sequence of numbers + + def push(self, item, priority): + """Push an item with given priority. Items with equal priority maintain FIFO order.""" + count = next(self._counter) + entry = (priority, count, item) + heapq.heappush(self._queue, entry) + + def pop(self): + """Remove and return tuple of (priority, item) with lowest priority.""" + if not self._queue: + raise IndexError("pop from empty queue") + priority, count, item = heapq.heappop(self._queue) + return priority, item + + def peek(self): + """Return but don't remove tuple of (priority, item) with lowest priority.""" + if not self._queue: + raise IndexError("peek at empty queue") + priority, count, item = self._queue[0] + return priority, item + + def get(self): + """Remove and return tuple of (priority, item), or None if queue is empty.""" + if not self._queue: + return None + priority, count, item = heapq.heappop(self._queue) + return priority, item + + def __len__(self): + return len(self._queue) + + def __bool__(self): + return bool(self._queue) From 64f4afc7ed20f89673bbd426fd95e13e76c6a4c3 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:19:33 +0100 Subject: [PATCH 04/88] add agent tests --- test/core/pipeline/features/test_run.py | 538 ++++++++++++++++++++++++ 1 file changed, 538 insertions(+) diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 652fea7c30..bdc81e45bf 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -2224,3 +2224,541 @@ def that_has_a_string_variadic_component(): ) ], ) + +@given("a pipeline that is an agent that can use RAG", target_fixture="pipeline_data") +def an_agent_that_can_use_RAG(): + @component + class FixedGenerator: + def __init__(self, replies): + self.replies = replies + self.idx = 0 + + @component.output_types(replies=List[str]) + def run(self, prompt: str): + if self.idx < len(self.replies): + replies = [self.replies[self.idx]] + self.idx += 1 + else: + self.idx = 0 + replies = [self.replies[self.idx]] + self.idx += 1 + + return {"replies": replies} + + @component + class FakeRetriever: + @component.output_types(documents=List[Document]) + def run(self, query: str): + return { + "documents": [ + Document(content="This is a document potentially answering the question.", meta={"access_group": 1}) + ] + } + + agent_prompt_template = """ +Your task is to answer the user's question. +You can use a RAG system to find information. +Use the RAG system until you have sufficient information to answer the question. +To use the RAG system, output "search:" followed by your question. +Once you have an answer, output "answer:" followed by your answer. + +Here is the question: {{query}} + """ + + rag_prompt_template = """ +Answer the question based on the provided documents. +Question: {{ query }} +Documents: +{% for document in documents %} +{{ document.content }} +{% endfor %} + """ + + joiner = BranchJoiner(type_=str) + + agent_llm = FixedGenerator(replies=["search: Can you help me?", "answer: here is my answer"]) + agent_prompt = PromptBuilder(template=agent_prompt_template) + + rag_llm = FixedGenerator(replies=["This is all the information I found!"]) + rag_prompt = PromptBuilder(template=rag_prompt_template) + + retriever = FakeRetriever() + + routes = [ + { + "condition": "{{ 'search:' in replies[0] }}", + "output": "{{ replies[0] }}", + "output_name": "search", + "output_type": str, + }, + { + "condition": "{{ 'answer:' in replies[0] }}", + "output": "{{ replies }}", + "output_name": "answer", + "output_type": List[str], + }, + ] + + router = ConditionalRouter(routes=routes) + + concatenator = OutputAdapter(template="{{current_prompt + '\n' + rag_answer[0]}}", output_type=str) + + answer_builder = AnswerBuilder() + + pp = Pipeline(max_runs_per_component=2) + + pp.add_component("joiner", joiner) + pp.add_component("rag_llm", rag_llm) + pp.add_component("rag_prompt", rag_prompt) + pp.add_component("agent_prompt", agent_prompt) + pp.add_component("agent_llm", agent_llm) + pp.add_component("router", router) + pp.add_component("concatenator", concatenator) + pp.add_component("retriever", retriever) + pp.add_component("answer_builder", answer_builder) + + pp.connect("agent_prompt.prompt", "joiner.value") + pp.connect("joiner.value", "agent_llm.prompt") + pp.connect("agent_llm.replies", "router.replies") + pp.connect("router.search", "retriever.query") + pp.connect("router.answer", "answer_builder.replies") + pp.connect("retriever.documents", "rag_prompt.documents") + pp.connect("rag_prompt.prompt", "rag_llm.prompt") + pp.connect("rag_llm.replies", "concatenator.rag_answer") + pp.connect("joiner.value", "concatenator.current_prompt") + pp.connect("concatenator.output", "joiner.value") + + query = "Does this run reliably?" + + return ( + pp, + [ + PipelineRunData( + inputs={ + "agent_prompt": {"query": query}, + "rag_prompt": {"query": query}, + "answer_builder": {"query": query}, + }, + expected_outputs={ + "answer_builder": { + "answers": [GeneratedAnswer(data="answer: here is my answer", query=query, documents=[])] + } + }, + expected_run_order=[ + "agent_prompt", + "joiner", + "agent_llm", + "router", + "retriever", + "rag_prompt", + "rag_llm", + "concatenator", + "joiner", + "agent_llm", + "router", + "answer_builder", + ], + ) + ], + ) + + +@given("a pipeline that has a feedback loop", target_fixture="pipeline_data") +def has_feedback_loop(): + @component + class FixedGenerator: + def __init__(self, replies): + self.replies = replies + self.idx = 0 + + @component.output_types(replies=List[str]) + def run(self, prompt: str): + if self.idx < len(self.replies): + replies = [self.replies[self.idx]] + self.idx += 1 + else: + self.idx = 0 + replies = [self.replies[self.idx]] + self.idx += 1 + + return {"replies": replies} + + code_prompt_template = """ +Generate code to solve the task: {{ task }} + +{% if feedback %} +Here is your initial attempt and some feedback: +{{ feedback }} +{% endif %} + """ + + feedback_prompt_template = """ +Check if this code is valid and can run: {{ code[0] }} +Return "PASS" if it passes and "FAIL" if it fails. +Provide additional feedback on why it fails. + """ + + code_llm = FixedGenerator(replies=["invalid code", "valid code"]) + code_prompt = PromptBuilder(template=code_prompt_template) + + feedback_llm = FixedGenerator(replies=["FAIL", "PASS"]) + feedback_prompt = PromptBuilder(template=feedback_prompt_template) + + routes = [ + { + "condition": "{{ 'FAIL' in replies[0] }}", + "output": "{{ replies[0] }}", + "output_name": "fail", + "output_type": str, + }, + { + "condition": "{{ 'PASS' in replies[0] }}", + "output": "{{ code }}", + "output_name": "pass", + "output_type": List[str], + }, + ] + + router = ConditionalRouter(routes=routes) + + concatenator = OutputAdapter(template="{{current_prompt[0] + '\n' + feedback[0]}}", output_type=str) + + answer_builder = AnswerBuilder() + + pp = Pipeline(max_runs_per_component=100) + + pp.add_component("code_llm", code_llm) + pp.add_component("code_prompt", code_prompt) + pp.add_component("feedback_prompt", feedback_prompt) + pp.add_component("feedback_llm", feedback_llm) + pp.add_component("router", router) + pp.add_component("concatenator", concatenator) + pp.add_component("answer_builder", answer_builder) + + pp.connect("code_prompt.prompt", "code_llm.prompt") + pp.connect("code_llm.replies", "feedback_prompt.code") + pp.connect("feedback_llm.replies", "router.replies") + pp.connect("router.fail", "concatenator.feedback") + pp.connect("router.pass", "answer_builder.replies") + pp.connect("code_llm.replies", "router.code") + pp.connect("feedback_prompt.prompt", "feedback_llm.prompt") + pp.connect("code_llm.replies", "concatenator.current_prompt") + pp.connect("concatenator.output", "code_prompt.feedback") + + task = "Generate code to generate christmas ascii-art" + + return ( + pp, + [ + PipelineRunData( + inputs={"code_prompt": {"task": task}, "answer_builder": {"query": task}}, + expected_outputs={ + "answer_builder": {"answers": [GeneratedAnswer(data="valid code", query=task, documents=[])]} + }, + expected_run_order=[ + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "concatenator", + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "answer_builder", + ], + ) + ], + ) + + +@given("a pipeline created in a non-standard order that has a loop", target_fixture="pipeline_data") +def has_non_standard_order_loop(): + @component + class FixedGenerator: + def __init__(self, replies): + self.replies = replies + self.idx = 0 + + @component.output_types(replies=List[str]) + def run(self, prompt: str): + if self.idx < len(self.replies): + replies = [self.replies[self.idx]] + self.idx += 1 + else: + self.idx = 0 + replies = [self.replies[self.idx]] + self.idx += 1 + + return {"replies": replies} + + code_prompt_template = """ +Generate code to solve the task: {{ task }} + +{% if feedback %} +Here is your initial attempt and some feedback: +{{ feedback }} +{% endif %} + """ + + feedback_prompt_template = """ +Check if this code is valid and can run: {{ code[0] }} +Return "PASS" if it passes and "FAIL" if it fails. +Provide additional feedback on why it fails. + """ + + code_llm = FixedGenerator(replies=["invalid code", "valid code"]) + code_prompt = PromptBuilder(template=code_prompt_template) + + feedback_llm = FixedGenerator(replies=["FAIL", "PASS"]) + feedback_prompt = PromptBuilder(template=feedback_prompt_template) + + routes = [ + { + "condition": "{{ 'FAIL' in replies[0] }}", + "output": "{{ replies[0] }}", + "output_name": "fail", + "output_type": str, + }, + { + "condition": "{{ 'PASS' in replies[0] }}", + "output": "{{ code }}", + "output_name": "pass", + "output_type": List[str], + }, + ] + + router = ConditionalRouter(routes=routes) + + concatenator = OutputAdapter(template="{{current_prompt[0] + '\n' + feedback[0]}}", output_type=str) + + answer_builder = AnswerBuilder() + + pp = Pipeline(max_runs_per_component=100) + + pp.add_component("concatenator", concatenator) + pp.add_component("code_llm", code_llm) + pp.add_component("code_prompt", code_prompt) + pp.add_component("feedback_prompt", feedback_prompt) + pp.add_component("feedback_llm", feedback_llm) + pp.add_component("router", router) + + pp.add_component("answer_builder", answer_builder) + + pp.connect("concatenator.output", "code_prompt.feedback") + pp.connect("code_prompt.prompt", "code_llm.prompt") + pp.connect("code_llm.replies", "feedback_prompt.code") + pp.connect("feedback_llm.replies", "router.replies") + pp.connect("router.fail", "concatenator.feedback") + pp.connect("feedback_prompt.prompt", "feedback_llm.prompt") + pp.connect("router.pass", "answer_builder.replies") + pp.connect("code_llm.replies", "router.code") + pp.connect("code_llm.replies", "concatenator.current_prompt") + + task = "Generate code to generate christmas ascii-art" + + return ( + pp, + [ + PipelineRunData( + inputs={"code_prompt": {"task": task}, "answer_builder": {"query": task}}, + expected_outputs={ + "answer_builder": {"answers": [GeneratedAnswer(data="valid code", query=task, documents=[])]} + }, + expected_run_order=[ + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "concatenator", + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "answer_builder", + ], + ) + ], + ) + + +@given("a pipeline that has an agent with a feedback cycle", target_fixture="pipeline_data") +def agent_with_feedback_cycle(): + @component + class FixedGenerator: + def __init__(self, replies): + self.replies = replies + self.idx = 0 + + @component.output_types(replies=List[str]) + def run(self, prompt: str): + if self.idx < len(self.replies): + replies = [self.replies[self.idx]] + self.idx += 1 + else: + self.idx = 0 + replies = [self.replies[self.idx]] + self.idx += 1 + + return {"replies": replies} + + @component + class FakeFileEditor: + @component.output_types(files=str) + def run(self, replies: List[str]): + return {"files": "This is the edited file content."} + + code_prompt_template = """ +Generate code to solve the task: {{ task }} + +You can edit files by returning: +Edit: file_name + +Once you solved the task, respond with: +Task finished! + +{% if feedback %} +Here is your initial attempt and some feedback: +{{ feedback }} +{% endif %} + """ + + feedback_prompt_template = """ +{% if task_finished %} +Check if this code is valid and can run: {{ code }} +Return "PASS" if it passes and "FAIL" if it fails. +Provide additional feedback on why it fails. +{% endif %} + """ + + code_llm = FixedGenerator(replies=["Edit: file_1.py", "Edit: file_2.py", "Edit: file_3.py", "Task finished!"]) + code_prompt = PromptBuilder(template=code_prompt_template) + file_editor = FakeFileEditor() + + feedback_llm = FixedGenerator(replies=["FAIL", "PASS"]) + feedback_prompt = PromptBuilder(template=feedback_prompt_template, required_variables=["task_finished"]) + + routes = [ + { + "condition": "{{ 'FAIL' in replies[0] }}", + "output": "{{ current_prompt + '\n' + replies[0] }}", + "output_name": "fail", + "output_type": str, + }, + { + "condition": "{{ 'PASS' in replies[0] }}", + "output": "{{ replies }}", + "output_name": "pass", + "output_type": List[str], + }, + ] + feedback_router = ConditionalRouter(routes=routes) + + tool_use_routes = [ + { + "condition": "{{ 'Edit:' in replies[0] }}", + "output": "{{ replies }}", + "output_name": "edit", + "output_type": List[str], + }, + { + "condition": "{{ 'Task finished!' in replies[0] }}", + "output": "{{ replies }}", + "output_name": "done", + "output_type": List[str], + }, + ] + tool_use_router = ConditionalRouter(routes=tool_use_routes) + + joiner = BranchJoiner(type_=str) + agent_concatenator = OutputAdapter(template="{{current_prompt + '\n' + files}}", output_type=str) + + pp = Pipeline(max_runs_per_component=100) + + pp.add_component("code_prompt", code_prompt) + pp.add_component("joiner", joiner) + pp.add_component("code_llm", code_llm) + pp.add_component("tool_use_router", tool_use_router) + pp.add_component("file_editor", file_editor) + pp.add_component("agent_concatenator", agent_concatenator) + pp.add_component("feedback_prompt", feedback_prompt) + pp.add_component("feedback_llm", feedback_llm) + pp.add_component("feedback_router", feedback_router) + + # Main Agent + pp.connect("code_prompt.prompt", "joiner.value") + pp.connect("joiner.value", "code_llm.prompt") + pp.connect("code_llm.replies", "tool_use_router.replies") + pp.connect("tool_use_router.edit", "file_editor.replies") + pp.connect("file_editor.files", "agent_concatenator.files") + pp.connect("joiner.value", "agent_concatenator.current_prompt") + pp.connect("agent_concatenator.output", "joiner.value") + + # Feedback Cycle + pp.connect("tool_use_router.done", "feedback_prompt.task_finished") + pp.connect("agent_concatenator.output", "feedback_prompt.code") + pp.connect("feedback_prompt.prompt", "feedback_llm.prompt") + pp.connect("feedback_llm.replies", "feedback_router.replies") + pp.connect("agent_concatenator.output", "feedback_router.current_prompt") + pp.connect("feedback_router.fail", "joiner.value") + + task = "Generate code to generate christmas ascii-art" + + return ( + pp, + [ + PipelineRunData( + inputs={"code_prompt": {"task": task}}, + expected_outputs={"feedback_router": {"pass": ["PASS"]}}, + expected_run_order=[ + "code_prompt", + "joiner", + "code_llm", + "tool_use_router", + "file_editor", + "agent_concatenator", + "joiner", + "code_llm", + "tool_use_router", + "file_editor", + "agent_concatenator", + "joiner", + "code_llm", + "tool_use_router", + "file_editor", + "agent_concatenator", + "joiner", + "code_llm", + "tool_use_router", + "feedback_prompt", + "feedback_llm", + "feedback_router", + "joiner", + "code_llm", + "tool_use_router", + "file_editor", + "agent_concatenator", + "joiner", + "code_llm", + "tool_use_router", + "file_editor", + "agent_concatenator", + "joiner", + "code_llm", + "tool_use_router", + "file_editor", + "agent_concatenator", + "joiner", + "code_llm", + "tool_use_router", + "feedback_prompt", + "feedback_llm", + "feedback_router", + ], + ) + ], + ) From 966552e2dc9599735852c402801eb6b1708f0303 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:21:45 +0100 Subject: [PATCH 05/88] add order dependent tests --- test/core/pipeline/features/test_run.py | 58 +++++++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index bdc81e45bf..6d1ba8bbf7 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -1808,6 +1808,64 @@ def run(self, create_document: bool = False): ], ) +@given( + "a pipeline that has a variadic component that receives partial inputs in a different order", + target_fixture="pipeline_data", +) +def that_has_a_variadic_component_that_receives_partial_inputs_different_order(): + @component + class ConditionalDocumentCreator: + def __init__(self, content: str): + self._content = content + + @component.output_types(documents=List[Document], noop=None) + def run(self, create_document: bool = False): + if create_document: + return {"documents": [Document(id=self._content, content=self._content)]} + return {"noop": None} + + pipeline = Pipeline(max_runs_per_component=1) + pipeline.add_component("third_creator", ConditionalDocumentCreator(content="Third document")) + pipeline.add_component("first_creator", ConditionalDocumentCreator(content="First document")) + pipeline.add_component("second_creator", ConditionalDocumentCreator(content="Second document")) + pipeline.add_component("documents_joiner", DocumentJoiner()) + + pipeline.connect("first_creator.documents", "documents_joiner.documents") + pipeline.connect("second_creator.documents", "documents_joiner.documents") + pipeline.connect("third_creator.documents", "documents_joiner.documents") + + return ( + pipeline, + [ + PipelineRunData( + inputs={"first_creator": {"create_document": True}, "third_creator": {"create_document": True}}, + expected_outputs={ + "second_creator": {"noop": None}, + "documents_joiner": { + "documents": [ + Document(id="First document", content="First document"), + Document(id="Third document", content="Third document"), + ] + }, + }, + expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], + ), + PipelineRunData( + inputs={"first_creator": {"create_document": True}, "second_creator": {"create_document": True}}, + expected_outputs={ + "third_creator": {"noop": None}, + "documents_joiner": { + "documents": [ + Document(id="First document", content="First document"), + Document(id="Second document", content="Second document"), + ] + }, + }, + expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], + ), + ], + ) + @given("a pipeline that has an answer joiner variadic component", target_fixture="pipeline_data") def that_has_an_answer_joiner_variadic_component(): From 3d0e948af564e1440804238bc53a096326862dc3 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:24:44 +0100 Subject: [PATCH 06/88] run new tests --- test/core/pipeline/features/pipeline_run.feature | 5 +++++ test/core/pipeline/features/test_run.py | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index db064ea2b1..398c4f3c2c 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -39,11 +39,16 @@ Feature: Pipeline running | that is linear with conditional branching and multiple joins | | that is a simple agent | | that has a variadic component that receives partial inputs | + | that has a variadic component that receives partial inputs in a different order | | that has an answer joiner variadic component | | that is linear and a component in the middle receives optional input from other components and input from the user | | that has a loop in the middle | | that has variadic component that receives a conditional input | | that has a string variadic component | + | that is an agent that can use RAG | + | that has a feedback loop | + | created in a non-standard order that has a loop | + | that has an agent with a feedback cycle | Scenario Outline: Running a bad Pipeline Given a pipeline diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 6d1ba8bbf7..49ac9ba018 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -10,7 +10,8 @@ from haystack.dataclasses import ChatMessage, GeneratedAnswer from haystack.components.routers import ConditionalRouter from haystack.components.builders import PromptBuilder, AnswerBuilder, ChatPromptBuilder -from haystack.components.preprocessors import DocumentCleaner, DocumentSplitter +from haystack.components.converters.output_adapter import OutputAdapter +from haystack.components.preprocessors import DocumentCleaner from haystack.components.retrievers.in_memory import InMemoryBM25Retriever from haystack.document_stores.in_memory import InMemoryDocumentStore from haystack.components.joiners import BranchJoiner, DocumentJoiner, AnswerJoiner, StringJoiner From 7664254dd25def0932a16583e4810523c39ffd14 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Sat, 11 Jan 2025 16:27:57 +0100 Subject: [PATCH 07/88] remove code that is not needed --- haystack/core/pipeline/base.py | 533 --------------------------------- 1 file changed, 533 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index d8f2a65932..1bfbf42f6f 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -765,26 +765,6 @@ def _prepare_component_input_data(self, data: Dict[str, Any]) -> Dict[str, Dict[ return data - def _normalize_varidiac_input_data(self, data: Dict[str, Dict[str, Any]]) -> Dict[str, Dict[str, Any]]: - """ - Variadic inputs expect their value to be a list, this utility method creates that list from the user's input. - """ - for component_name, component_inputs in data.items(): - if component_name not in self.graph.nodes: - # This is not a component name, it must be the name of one or more input sockets. - # Those are handled in a different way, so we skip them here. - continue - instance = self.graph.nodes[component_name]["instance"] - for component_input, input_value in component_inputs.items(): - if instance.__haystack_input__._sockets_dict[component_input].is_variadic: - # Components that have variadic inputs need to receive lists as input. - # We don't want to force the user to always pass lists, so we convert single values to lists here. - # If it's already a list we assume the component takes a variadic input of lists, so we - # convert it in any case. - data[component_name][component_input] = [input_value] - - return {**data} - @classmethod def from_template( cls, predefined_pipeline: PredefinedPipeline, template_params: Optional[Dict[str, Any]] = None @@ -835,413 +815,6 @@ def _find_receivers_from(self, component_name: str) -> List[Tuple[str, OutputSoc res.append((receiver_name, sender_socket, receiver_socket)) return res - def _distribute_output( # pylint: disable=too-many-positional-arguments - self, - receiver_components: List[Tuple[str, OutputSocket, InputSocket]], - component_result: Dict[str, Any], - components_inputs: Dict[str, Dict[str, Any]], - run_queue: List[Tuple[str, Component]], - waiting_queue: List[Tuple[str, Component]], - ) -> Dict[str, Any]: - """ - Distributes the output of a Component to the next Components that need it. - - This also updates the queues that keep track of which Components are ready to run and which are waiting for - input. - - :param receiver_components: - List of tuples containing name of receiver Components and relative sender OutputSocket - and receiver InputSocket instances - :param component_result: - The output of the Component - :param components_inputs: - The current state of the inputs divided by Component name - :param run_queue: - Queue of Components to run - :param waiting_queue: - Queue of Components waiting for input - - :returns: - The updated output of the Component without the keys that were distributed to other Components - """ - # We keep track of which keys to remove from component_result at the end of the loop. - # This is done after the output has been distributed to the next components, so that - # we're sure all components that need this output have received it. - to_remove_from_component_result = set() - - for receiver_name, sender_socket, receiver_socket in receiver_components: - if sender_socket.name not in component_result: - # This output wasn't created by the sender, nothing we can do. - # - # Some Components might have conditional outputs, so we need to check if they actually returned - # some output while iterating over their output sockets. - # - # A perfect example of this would be the ConditionalRouter, which will have an output for each - # condition it has been initialized with. - # Though it will return only one output at a time. - continue - - if receiver_name not in components_inputs: - components_inputs[receiver_name] = {} - - # We keep track of the keys that were distributed to other Components. - # This key will be removed from component_result at the end of the loop. - to_remove_from_component_result.add(sender_socket.name) - - value = component_result[sender_socket.name] - - if receiver_socket.is_variadic: - # Usually Component inputs can only be received from one sender, the Variadic type allows - # instead to receive inputs from multiple senders. - # - # To keep track of all the inputs received internally we always store them in a list. - if receiver_socket.name not in components_inputs[receiver_name]: - # Create the list if it doesn't exist - components_inputs[receiver_name][receiver_socket.name] = [] - else: - # Check if the value is actually a list - assert isinstance(components_inputs[receiver_name][receiver_socket.name], list) - components_inputs[receiver_name][receiver_socket.name].append(value) - else: - components_inputs[receiver_name][receiver_socket.name] = value - - receiver = self.graph.nodes[receiver_name]["instance"] - pair = (receiver_name, receiver) - - if receiver_socket.is_variadic: - if receiver_socket.is_greedy: - # If the receiver is greedy, we can run it as soon as possible. - # First we remove it from the status lists it's in if it's there or - # we risk running it multiple times. - if pair in run_queue: - run_queue.remove(pair) - if pair in waiting_queue: - waiting_queue.remove(pair) - run_queue.insert(0, pair) - else: - # If the receiver Component has a variadic input that is not greedy - # we put it in the waiting queue. - # This make sure that we don't run it earlier than necessary and we can collect - # as many inputs as we can before running it. - if pair not in waiting_queue: - waiting_queue.append(pair) - - if pair not in waiting_queue and pair not in run_queue: - # Queue up the Component that received this input to run, only if it's not already waiting - # for input or already ready to run. - run_queue.append(pair) - - # Returns the output without the keys that were distributed to other Components - return {k: v for k, v in component_result.items() if k not in to_remove_from_component_result} - - def _find_next_runnable_component( - self, components_inputs: Dict[str, Dict[str, Any]], waiting_queue: List[Tuple[str, Component]] - ) -> Tuple[str, Component]: - """ - Finds the next Component that can be run and returns it. - - :param components_inputs: The current state of the inputs divided by Component name - :param waiting_queue: Queue of Components waiting for input - - :returns: The name and the instance of the next Component that can be run - """ - all_lazy_variadic = True - all_with_default_inputs = True - - filtered_waiting_queue = [] - - for name, comp in waiting_queue: - if not _is_lazy_variadic(comp): - # Components with variadic inputs that are not greedy must be removed only if there's nothing else to - # run at this stage. - # We need to wait as long as possible to run them, so we can collect as most inputs as we can. - all_lazy_variadic = False - - if not _has_all_inputs_with_defaults(comp): - # Components that have defaults for all their inputs must be treated the same identical way as we treat - # lazy variadic components. If there are only components with defaults we can run them. - # If we don't do this the order of execution of the Pipeline's Components will be affected cause we - # enqueue the Components in `run_queue` at the start using the order they are added in the Pipeline. - # If a Component A with defaults is added before a Component B that has no defaults, but in the Pipeline - # logic A must be executed after B. However, B could run before A if we don't do this check. - all_with_default_inputs = False - - if not _is_lazy_variadic(comp) and not _has_all_inputs_with_defaults(comp): - # Keep track of the Components that are not lazy variadic and don't have all inputs with defaults. - # We'll handle these later if necessary. - filtered_waiting_queue.append((name, comp)) - - # If all Components are lazy variadic or all Components have all inputs with defaults we can get one to run - if all_lazy_variadic or all_with_default_inputs: - return waiting_queue[0] - - for name, comp in filtered_waiting_queue: - # Find the first component that has all the inputs it needs to run - has_enough_inputs = True - for input_socket in comp.__haystack_input__._sockets_dict.values(): # type: ignore - if input_socket.name not in components_inputs.get(name, {}) and input_socket.is_mandatory: - has_enough_inputs = False - break - - if has_enough_inputs: - return name, comp - - # If we reach this point it means that we found no Component that has enough inputs to run. - # Ideally we should never reach this point, though we can't raise an exception either as - # existing use cases rely on this behavior. - # So we return the last Component, that could be the last from waiting_queue or filtered_waiting_queue. - return name, comp - - def _find_next_runnable_lazy_variadic_or_default_component( - self, waiting_queue: List[Tuple[str, Component]] - ) -> Tuple[str, Component]: - """ - Finds the next Component that can be run and has a lazy variadic input or all inputs with default values. - - :param waiting_queue: Queue of Components waiting for input - - :returns: The name and the instance of the next Component that can be run - """ - for name, comp in waiting_queue: - is_lazy_variadic = _is_lazy_variadic(comp) - has_only_defaults = _has_all_inputs_with_defaults(comp) - if is_lazy_variadic or has_only_defaults: - return name, comp - - # If we reach this point it means that we found no Component that has a lazy variadic input or all inputs with - # default values to run. - # Similar to `_find_next_runnable_component` we might not find the Component we want, so we optimistically - # return the last Component in the list. - # We're probably stuck in a loop in this case, but we can't raise an exception as existing use cases might - # rely on this behaviour. - # The loop detection will be handled later on. - return name, comp - - def _find_components_that_will_receive_no_input( - self, component_name: str, component_result: Dict[str, Any], components_inputs: Dict[str, Dict[str, Any]] - ) -> Set[Tuple[str, Component]]: - """ - Find all the Components that are connected to component_name and didn't receive any input from it. - - Components that have a Variadic input and received already some input from other Components - but not from component_name won't be returned as they have enough inputs to run. - - This includes the descendants of the Components that didn't receive any input from component_name. - That is necessary to avoid getting stuck into infinite loops waiting for inputs that will never arrive. - - :param component_name: Name of the Component that created the output - :param component_result: Output of the Component - :param components_inputs: The current state of the inputs divided by Component name - :return: A set of Components that didn't receive any input from component_name - """ - - # Simplifies the check if a Component is Variadic and received some input from other Components. - def has_variadic_socket_with_existing_inputs( - component: Component, component_name: str, sender_name: str, components_inputs: Dict[str, Dict[str, Any]] - ) -> bool: - for socket in component.__haystack_input__._sockets_dict.values(): # type: ignore - if sender_name not in socket.senders: - continue - if socket.is_variadic and len(components_inputs.get(component_name, {}).get(socket.name, [])) > 0: - return True - return False - - # Makes it easier to verify if all connections between two Components are optional - def all_connections_are_optional(sender_name: str, receiver: Component) -> bool: - for socket in receiver.__haystack_input__._sockets_dict.values(): # type: ignore - if sender_name not in socket.senders: - continue - if socket.is_mandatory: - return False - return True - - # Eases checking if other connections that are not between sender_name and receiver_name - # already received inputs - def other_connections_received_input(sender_name: str, receiver_name: str) -> bool: - receiver: Component = self.graph.nodes[receiver_name]["instance"] - for receiver_socket in receiver.__haystack_input__._sockets_dict.values(): # type: ignore - if sender_name in receiver_socket.senders: - continue - if components_inputs.get(receiver_name, {}).get(receiver_socket.name) is not None: - return True - return False - - components = set() - instance: Component = self.graph.nodes[component_name]["instance"] - for socket_name, socket in instance.__haystack_output__._sockets_dict.items(): # type: ignore - if socket_name in component_result: - continue - for receiver in socket.receivers: - receiver_instance: Component = self.graph.nodes[receiver]["instance"] - - if has_variadic_socket_with_existing_inputs( - receiver_instance, receiver, component_name, components_inputs - ): - # Components with Variadic input that already received some input - # can still run, even if branch is skipped. - # If we remove them they won't run. - continue - - if all_connections_are_optional(component_name, receiver_instance) and other_connections_received_input( - component_name, receiver - ): - # If all the connections between component_name and receiver are optional - # and receiver received other inputs already it still has enough inputs to run. - # Even if it didn't receive input from component_name, so we can't remove it or its - # descendants. - continue - - components.add((receiver, receiver_instance)) - # Get the descendants too. When we remove a Component that received no input - # it's extremely likely that its descendants will receive no input as well. - # This is fine even if the Pipeline will merge back into a single Component - # at a certain point. The merging Component will be put back into the run - # queue at a later stage. - for descendant_name in networkx.descendants(self.graph, receiver): - descendant = self.graph.nodes[descendant_name]["instance"] - - # Components with Variadic input that already received some input - # can still run, even if branch is skipped. - # If we remove them they won't run. - if has_variadic_socket_with_existing_inputs( - descendant, descendant_name, receiver, components_inputs - ): - continue - - components.add((descendant_name, descendant)) - - return components - - def _is_stuck_in_a_loop(self, waiting_queue: List[Tuple[str, Component]]) -> bool: - """ - Checks if the Pipeline is stuck in a loop. - - :param waiting_queue: Queue of Components waiting for input - - :returns: True if the Pipeline is stuck in a loop, False otherwise - """ - # Are we actually stuck or there's a lazy variadic or a component with has only default inputs - # waiting for input? - # This is our last resort, if there's no lazy variadic or component with only default inputs - # waiting for input we're stuck for real and we can't make any progress. - component_found = False - for _, comp in waiting_queue: - if _is_lazy_variadic(comp) or _has_all_inputs_with_defaults(comp): - component_found = True - break - - if not component_found: - # We're stuck in a loop for real, we can't make any progress. - # BAIL! - return True - - # If we have a single component with no variadic input or only default inputs waiting for input - # it means it has been waiting for input for at least 2 iterations. - # This will never run. - # BAIL! - return len(waiting_queue) == 1 - - def _component_has_enough_inputs_to_run(self, name: str, inputs: Dict[str, Dict[str, Any]]) -> bool: - """ - Returns True if the Component has all the inputs it needs to run. - - :param name: Name of the Component as defined in the Pipeline. - :param inputs: The current state of the inputs divided by Component name. - - :return: Whether the Component can run or not. - """ - instance: Component = self.graph.nodes[name]["instance"] - if name not in inputs: - return False - expected_inputs = instance.__haystack_input__._sockets_dict.keys() # type: ignore - current_inputs = inputs[name].keys() - return expected_inputs == current_inputs - - def _break_supported_cycles_in_graph(self) -> Tuple[networkx.MultiDiGraph, Dict[str, List[List[str]]]]: - """ - Utility function to remove supported cycles in the Pipeline's graph. - - Given that the Pipeline execution would wait to run a Component until it has received - all its mandatory inputs, it doesn't make sense for us to try and break cycles by - removing a connection to a mandatory input. The Pipeline would just get stuck at a later time. - - So we can only break connections in cycles that have a Variadic or GreedyVariadic type or a default value. - - This will raise a PipelineRuntimeError if we there are cycles that can't be broken. - That is bound to happen when at least one of the inputs in a cycle is mandatory. - - If the Pipeline's graph doesn't have any cycle it will just return that graph and an empty dictionary. - - :returns: - A tuple containing: - * A copy of the Pipeline's graph without cycles - * A dictionary of Component's names and a list of all the cycles they were part of. - The cycles are a list of Component's names that create that cycle. - """ - if networkx.is_directed_acyclic_graph(self.graph): - return self.graph, {} - - temp_graph: networkx.MultiDiGraph = self.graph.copy() - # A list of all the cycles that are found in the graph, each inner list contains - # the Component names that create that cycle. - cycles: List[List[str]] = list(networkx.simple_cycles(self.graph)) - # Maps a Component name to a list of its output socket names that have been broken - edges_removed: Dict[str, List[str]] = defaultdict(list) - # This keeps track of all the cycles that a component is part of. - # Maps a Component name to a list of cycles, each inner list contains - # the Component names that create that cycle (the key will also be - # an element in each list). The last Component in each list is implicitly - # connected to the first. - components_in_cycles: Dict[str, List[List[str]]] = defaultdict(list) - - # Used to minimize the number of time we check whether the graph has any more - # cycles left to break or not. - graph_has_cycles = True - - # Iterate all the cycles to find the least amount of connections that we can remove - # to make the Pipeline graph acyclic. - # As soon as the graph is acyclic we stop breaking connections and return. - for cycle in cycles: - for comp in cycle: - components_in_cycles[comp].append(cycle) - - # Iterate this cycle, we zip the cycle with itself so that at the last iteration - # sender_comp will be the last element of cycle and receiver_comp will be the first. - # So if cycle is [1, 2, 3, 4] we would call zip([1, 2, 3, 4], [2, 3, 4, 1]). - for sender_comp, receiver_comp in zip(cycle, cycle[1:] + cycle[:1]): - # We get the key and iterate those as we want to edit the graph data while - # iterating the edges and that would raise. - # Even though the connection key set in Pipeline.connect() uses only the - # sockets name we don't have clashes since it's only used to differentiate - # multiple edges between two nodes. - edge_keys = list(temp_graph.get_edge_data(sender_comp, receiver_comp).keys()) - for edge_key in edge_keys: - edge_data = temp_graph.get_edge_data(sender_comp, receiver_comp)[edge_key] - receiver_socket = edge_data["to_socket"] - if not receiver_socket.is_variadic and receiver_socket.is_mandatory: - continue - - # We found a breakable edge - sender_socket = edge_data["from_socket"] - edges_removed[sender_comp].append(sender_socket.name) - temp_graph.remove_edge(sender_comp, receiver_comp, edge_key) - - graph_has_cycles = not networkx.is_directed_acyclic_graph(temp_graph) - if not graph_has_cycles: - # We removed all the cycles, we can stop - break - - if not graph_has_cycles: - # We removed all the cycles, nice - break - - if graph_has_cycles: - msg = "Pipeline contains a cycle that we can't execute" - raise PipelineRuntimeError(msg) - - return temp_graph, components_in_cycles - - def _connections_status( sender_node: str, receiver_node: str, sender_sockets: List[OutputSocket], receiver_sockets: List[InputSocket] ): @@ -1266,109 +839,3 @@ def _connections_status( return f"'{sender_node}':\n{sender_sockets_list}\n'{receiver_node}':\n{receiver_sockets_list}" - -def _is_lazy_variadic(c: Component) -> bool: - """ - Small utility function to check if a Component has at least a Variadic input and no GreedyVariadic input. - """ - is_variadic = any( - socket.is_variadic - for socket in c.__haystack_input__._sockets_dict.values() # type: ignore - ) - if not is_variadic: - return False - return not any( - socket.is_greedy - for socket in c.__haystack_input__._sockets_dict.values() # type: ignore - ) - - -def _has_all_inputs_with_defaults(c: Component) -> bool: - """ - Small utility function to check if a Component has all inputs with defaults. - """ - return all( - not socket.is_mandatory - for socket in c.__haystack_input__._sockets_dict.values() # type: ignore - ) - - -def _add_missing_input_defaults(name: str, comp: Component, components_inputs: Dict[str, Dict[str, Any]]): - """ - Updates the inputs with the default values for the inputs that are missing - - :param name: Name of the Component - :param comp: Instance of the Component - :param components_inputs: The current state of the inputs divided by Component name - """ - if name not in components_inputs: - components_inputs[name] = {} - - for input_socket in comp.__haystack_input__._sockets_dict.values(): # type: ignore - if input_socket.is_mandatory: - continue - - if input_socket.name not in components_inputs[name]: - components_inputs[name][input_socket.name] = input_socket.default_value - - -def _enqueue_component( - component_pair: Tuple[str, Component], - run_queue: List[Tuple[str, Component]], - waiting_queue: List[Tuple[str, Component]], -): - """ - Append a Component in the queue of Components to run if not already in it. - - Remove it from the waiting list if it's there. - - :param component_pair: Tuple of Component name and instance - :param run_queue: Queue of Components to run - :param waiting_queue: Queue of Components waiting for input - """ - if component_pair in waiting_queue: - waiting_queue.remove(component_pair) - - if component_pair not in run_queue: - run_queue.append(component_pair) - - -def _dequeue_component( - component_pair: Tuple[str, Component], - run_queue: List[Tuple[str, Component]], - waiting_queue: List[Tuple[str, Component]], -): - """ - Removes a Component both from the queue of Components to run and the waiting list. - - :param component_pair: Tuple of Component name and instance - :param run_queue: Queue of Components to run - :param waiting_queue: Queue of Components waiting for input - """ - if component_pair in waiting_queue: - waiting_queue.remove(component_pair) - - if component_pair in run_queue: - run_queue.remove(component_pair) - - -def _enqueue_waiting_component(component_pair: Tuple[str, Component], waiting_queue: List[Tuple[str, Component]]): - """ - Append a Component in the queue of Components that are waiting for inputs if not already in it. - - :param component_pair: Tuple of Component name and instance - :param waiting_queue: Queue of Components waiting for input - """ - if component_pair not in waiting_queue: - waiting_queue.append(component_pair) - - -def _dequeue_waiting_component(component_pair: Tuple[str, Component], waiting_queue: List[Tuple[str, Component]]): - """ - Removes a Component from the queue of Components that are waiting for inputs. - - :param component_pair: Tuple of Component name and instance - :param waiting_queue: Queue of Components waiting for input - """ - if component_pair in waiting_queue: - waiting_queue.remove(component_pair) From 932718fbbd92e3916d5c842bd029038e75f110de Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 11:39:41 +0100 Subject: [PATCH 08/88] test: intermediate from cycle outputs are available outside cycle --- .../pipeline/features/pipeline_run.feature | 1 + test/core/pipeline/features/test_run.py | 145 ++++++++++++++++++ 2 files changed, 146 insertions(+) diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index 398c4f3c2c..12ad1513cc 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -49,6 +49,7 @@ Feature: Pipeline running | that has a feedback loop | | created in a non-standard order that has a loop | | that has an agent with a feedback cycle | + | that passes outputs that are consumed in cycle to outside the cycle | Scenario Outline: Running a bad Pipeline Given a pipeline diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 49ac9ba018..7d5b9965ca 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -2821,3 +2821,148 @@ def run(self, replies: List[str]): ) ], ) + +@given("a pipeline that passes outputs that are consumed in cycle to outside the cycle", target_fixture="pipeline_data") +def passes_outputs_outside_cycle(): + @component + class FixedGenerator: + def __init__(self, replies): + self.replies = replies + self.idx = 0 + + @component.output_types(replies=List[str]) + def run(self, prompt: str): + if self.idx < len(self.replies): + replies = [self.replies[self.idx]] + self.idx += 1 + else: + self.idx = 0 + replies = [self.replies[self.idx]] + self.idx += 1 + + return {"replies": replies} + + @component + class AnswerBuilderWithPrompt: + @component.output_types(answers=List[GeneratedAnswer]) + def run(self, replies: List[str], query: str, prompt: Optional[str] = None) -> Dict[str, Any]: + answer = GeneratedAnswer(data=replies[0], query=query, documents=[]) + + if prompt is not None: + answer.meta["prompt"] = prompt + + + return {"answers": [answer]} + + code_prompt_template = "{{task}}" + + feedback_prompt_template = """ +Check if this code is valid and can run: {{ code[0] }} +Return "PASS" if it passes and "FAIL" if it fails. +Provide additional feedback on why it fails. + """ + + valid_response = """ +def generate_santa_sleigh(): + ''' + Returns ASCII art of Santa Claus on his sleigh with Rudolph leading the way. + ''' + # implementation goes here. + return art + """ + + code_llm = FixedGenerator(replies=["invalid code", "invalid code", valid_response]) + code_prompt = PromptBuilder(template=code_prompt_template) + + feedback_llm = FixedGenerator(replies=["FAIL", "FAIL, come on, try again.", "PASS"]) + feedback_prompt = PromptBuilder(template=feedback_prompt_template) + + routes = [ + { + "condition": "{{ 'FAIL' in replies[0] }}", + "output": "{{ replies[0] }}", + "output_name": "fail", + "output_type": str, + }, + { + "condition": "{{ 'PASS' in replies[0] }}", + "output": "{{ code }}", + "output_name": "pass", + "output_type": List[str], + }, + ] + + router = ConditionalRouter(routes=routes) + joiner = BranchJoiner(type_=str) + concatenator = OutputAdapter(template="{{code_prompt + '\n' + generated_code[0] + '\n' + feedback}}", output_type=str) + + answer_builder = AnswerBuilderWithPrompt() + + pp = Pipeline(max_runs_per_component=100) + + pp.add_component("concatenator", concatenator) + pp.add_component("code_llm", code_llm) + pp.add_component("code_prompt", code_prompt) + pp.add_component("feedback_prompt", feedback_prompt) + pp.add_component("feedback_llm", feedback_llm) + pp.add_component("router", router) + pp.add_component("joiner", joiner) + + pp.add_component("answer_builder", answer_builder) + + pp.connect("concatenator.output", "joiner.value") + pp.connect("joiner.value", "code_prompt.task") + pp.connect("code_prompt.prompt", "code_llm.prompt") + pp.connect("code_prompt.prompt", "concatenator.code_prompt") + pp.connect("code_llm.replies", "feedback_prompt.code") + pp.connect("feedback_llm.replies", "router.replies") + pp.connect("router.fail", "concatenator.feedback") + pp.connect("feedback_prompt.prompt", "feedback_llm.prompt") + pp.connect("router.pass", "answer_builder.replies") + pp.connect("code_llm.replies", "router.code") + pp.connect("code_llm.replies", "concatenator.generated_code") + pp.connect("concatenator.output", "answer_builder.prompt") + + task = "Generate code to generate christmas ascii-art" + + expected_prompt = """Generate code to generate christmas ascii-art +invalid code +FAIL +invalid code +FAIL, come on, try again.""" + return ( + pp, + [ + PipelineRunData( + inputs={"joiner": {"value": task}, "answer_builder": {"query": task}}, + expected_outputs={ + "answer_builder": {"answers": [GeneratedAnswer(data=valid_response, query=task, documents=[], meta={"prompt": expected_prompt})]} + }, + expected_run_order=[ + "joiner", + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "concatenator", + "joiner", + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "concatenator", + "joiner", + "code_prompt", + "code_llm", + "feedback_prompt", + "feedback_llm", + "router", + "answer_builder", + ], + ) + ], + ) + + From acce8cd65363740a90cef33ad9190bdfaf5bc0df Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 13:52:29 +0100 Subject: [PATCH 09/88] add tests for component checks (Claude) --- test/core/pipeline/test_component_checks.py | 584 ++++++++++++++++++++ 1 file changed, 584 insertions(+) create mode 100644 test/core/pipeline/test_component_checks.py diff --git a/test/core/pipeline/test_component_checks.py b/test/core/pipeline/test_component_checks.py new file mode 100644 index 0000000000..418d5e3814 --- /dev/null +++ b/test/core/pipeline/test_component_checks.py @@ -0,0 +1,584 @@ +import pytest +from typing import Dict, Any + +from haystack.core.pipeline.component_checks import * +from haystack.core.pipeline.component_checks import _NO_OUTPUT_PRODUCED +from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic + +@pytest.fixture +def basic_component(): + """Basic component with one mandatory and one optional input""" + return { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int), + "optional_input": InputSocket("optional_input", str, default_value="default") + }, + "output_sockets": { + "output": OutputSocket("output", int) + } + } + +@pytest.fixture +def variadic_component(): + """Component with variadic input""" + return { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "variadic_input": InputSocket("variadic_input", Variadic[int]), + "normal_input": InputSocket("normal_input", str) + }, + "output_sockets": { + "output": OutputSocket("output", int) + } + } + +@pytest.fixture +def greedy_variadic_component(): + """Component with greedy variadic input""" + return { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "greedy_input": InputSocket("greedy_input", GreedyVariadic[int]), + "normal_input": InputSocket("normal_input", str) + }, + "output_sockets": { + "output": OutputSocket("output", int) + } + } + +@pytest.fixture +def input_socket_with_sender(): + """Regular input socket with a single sender""" + socket = InputSocket("test_input", int) + socket.senders = ["component1"] + return socket + +@pytest.fixture +def variadic_socket_with_senders(): + """Variadic input socket with multiple senders""" + socket = InputSocket("test_variadic", Variadic[int]) + socket.senders = ["component1", "component2"] + return socket + +@pytest.fixture +def component_with_multiple_sockets(input_socket_with_sender, variadic_socket_with_senders): + """Component with multiple input sockets including both regular and variadic""" + return { + "instance": "mock_instance", + "input_sockets": { + "socket1": input_socket_with_sender, + "socket2": variadic_socket_with_senders, + "socket3": InputSocket("socket3", str) # No senders + } + } + +@pytest.fixture +def regular_socket(): + """Regular input socket with one sender""" + socket = InputSocket("regular", int) + socket.senders = ["component1"] + return socket + +@pytest.fixture +def lazy_variadic_socket(): + """Lazy variadic input socket with multiple senders""" + socket = InputSocket("lazy_variadic", Variadic[int]) + socket.senders = ["component1", "component2"] + return socket + +@pytest.fixture +def greedy_variadic_socket(): + """Greedy variadic input socket with multiple senders""" + socket = InputSocket("greedy_variadic", GreedyVariadic[int]) + socket.senders = ["component1", "component2", "component3"] + return socket + +@pytest.fixture +def complex_component(regular_socket, lazy_variadic_socket, greedy_variadic_socket): + """Component with all types of sockets""" + return { + "instance": "mock_instance", + "input_sockets": { + "regular": regular_socket, + "lazy_var": lazy_variadic_socket, + "greedy_var": greedy_variadic_socket + } + } + +class TestCanComponentRun: + def test_component_with_all_mandatory_inputs_and_trigger(self, basic_component): + inputs = { + "mandatory_input": [{"sender": "previous_component", "value": 42}] + } + assert can_component_run(basic_component, inputs) is True + + def test_component_missing_mandatory_input(self, basic_component): + inputs = { + "optional_input": [{"sender": "previous_component", "value": "test"}] + } + assert can_component_run(basic_component, inputs) is False + + def test_component_with_no_trigger_but_all_inputs(self, basic_component): + """ + Test case where all mandatory inputs are present with valid values, + but there is no trigger (no new input from predecessor, not first visit) + """ + # Set visits > 0 so it's not triggered by first visit + basic_component["visits"] = 1 + inputs = { + "mandatory_input": [{"sender": None, "value": 42}] # Valid input value + } + assert can_component_run(basic_component, inputs) is False + + def test_component_with_multiple_visits(self, basic_component): + basic_component["visits"] = 2 + inputs = { + "mandatory_input": [{"sender": "previous_component", "value": 42}] + } + assert can_component_run(basic_component, inputs) is True + + def test_component_with_no_inputs_first_visit(self, basic_component): + basic_component["input_sockets"] = {} # No inputs required + inputs = {} + assert can_component_run(basic_component, inputs) is True + +class TestHasAnyTrigger: + def test_trigger_from_predecessor(self, basic_component): + inputs = { + "mandatory_input": [{"sender": "previous_component", "value": 42}] + } + assert has_any_trigger(basic_component, inputs) is True + + def test_trigger_from_user_first_visit(self, basic_component): + inputs = { + "mandatory_input": [{"sender": None, "value": 42}] + } + assert has_any_trigger(basic_component, inputs) is True + + def test_no_trigger_from_user_after_first_visit(self, basic_component): + basic_component["visits"] = 1 + inputs = { + "mandatory_input": [{"sender": None, "value": 42}] + } + assert has_any_trigger(basic_component, inputs) is False + + def test_trigger_without_inputs_first_visit(self, basic_component): + basic_component["input_sockets"] = {} # No inputs + inputs = {} + assert has_any_trigger(basic_component, inputs) is True + + def test_no_trigger_without_inputs_after_first_visit(self, basic_component): + basic_component["input_sockets"] = {} + basic_component["visits"] = 1 + inputs = {} + assert has_any_trigger(basic_component, inputs) is False + +class TestAllMandatorySocketsReady: + def test_all_mandatory_sockets_filled(self, basic_component): + inputs = { + "mandatory_input": [{"sender": "previous_component", "value": 42}] + } + assert are_all_mandatory_sockets_ready(basic_component, inputs) is True + + def test_missing_mandatory_socket(self, basic_component): + inputs = { + "optional_input": [{"sender": "previous_component", "value": "test"}] + } + assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + + def test_variadic_socket_with_input(self, variadic_component): + inputs = { + "variadic_input": [{"sender": "previous_component", "value": 42}], + "normal_input": [{"sender": "previous_component", "value": "test"}] + } + assert are_all_mandatory_sockets_ready(variadic_component, inputs) is True + + def test_greedy_variadic_socket_with_partial_input(self, greedy_variadic_component): + inputs = { + "greedy_input": [{"sender": "previous_component", "value": 42}], + "normal_input": [{"sender": "previous_component", "value": "test"}] + } + assert are_all_mandatory_sockets_ready(greedy_variadic_component, inputs) is True + + def test_variadic_socket_no_input(self, variadic_component): + inputs = { + "normal_input": [{"sender": "previous_component", "value": "test"}] + } + assert are_all_mandatory_sockets_ready(variadic_component, inputs) is False + + def test_empty_inputs(self, basic_component): + inputs = {} + assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + + def test_no_mandatory_sockets(self, basic_component): + basic_component["input_sockets"] = { + "optional_1": InputSocket("optional_1", str, default_value="default1"), + "optional_2": InputSocket("optional_2", str, default_value="default2") + } + inputs = {} + assert are_all_mandatory_sockets_ready(basic_component, inputs) is True + + def test_multiple_mandatory_sockets(self, basic_component): + basic_component["input_sockets"] = { + "mandatory_1": InputSocket("mandatory_1", int), + "mandatory_2": InputSocket("mandatory_2", str), + "optional": InputSocket("optional", bool, default_value=False) + } + inputs = { + "mandatory_1": [{"sender": "comp1", "value": 42}], + "mandatory_2": [{"sender": "comp2", "value": "test"}] + } + assert are_all_mandatory_sockets_ready(basic_component, inputs) is True + + # Missing one mandatory input + inputs = { + "mandatory_1": [{"sender": "comp1", "value": 42}], + "optional": [{"sender": "comp3", "value": True}] + } + assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + +class TestPredecessorInputDetection: + def test_any_predecessors_provided_input_with_predecessor(self, component_with_multiple_sockets): + inputs = { + "socket1": [{"sender": "component1", "value": 42}], + "socket2": [{"sender": None, "value": "test"}] + } + assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is True + + def test_any_predecessors_provided_input_no_predecessor(self, component_with_multiple_sockets): + inputs = { + "socket1": [{"sender": None, "value": 42}], + "socket2": [{"sender": None, "value": "test"}] + } + assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False + + def test_any_predecessors_provided_input_with_no_output(self, component_with_multiple_sockets): + inputs = { + "socket1": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}], + "socket2": [{"sender": None, "value": "test"}] + } + assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False + + def test_any_predecessors_provided_input_empty_inputs(self, component_with_multiple_sockets): + inputs = {} + assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False + +class TestSocketValueFromPredecessor: + def test_socket_value_from_predecessor_with_valid_input(self): + socket_inputs = [{"sender": "component1", "value": 42}] + assert any_socket_value_from_predecessor_received(socket_inputs) is True + + def test_socket_value_from_predecessor_with_no_output(self): + socket_inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] + assert any_socket_value_from_predecessor_received(socket_inputs) is False + + def test_socket_value_from_predecessor_with_user_input(self): + socket_inputs = [{"sender": None, "value": 42}] + assert any_socket_value_from_predecessor_received(socket_inputs) is False + + def test_socket_value_from_predecessor_with_mixed_inputs(self): + socket_inputs = [ + {"sender": None, "value": 42}, + {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "component2", "value": 100} + ] + assert any_socket_value_from_predecessor_received(socket_inputs) is True + + def test_socket_value_from_predecessor_empty_list(self): + assert any_socket_value_from_predecessor_received([]) is False + +class TestUserInputDetection: + def test_has_user_input_with_user_input(self): + inputs = { + "socket1": [{"sender": None, "value": 42}], + "socket2": [{"sender": "component1", "value": "test"}] + } + assert has_user_input(inputs) is True + + def test_has_user_input_without_user_input(self): + inputs = { + "socket1": [{"sender": "component1", "value": 42}], + "socket2": [{"sender": "component2", "value": "test"}] + } + assert has_user_input(inputs) is False + + def test_has_user_input_empty_inputs(self): + inputs = {} + assert has_user_input(inputs) is False + + def test_has_user_input_with_no_output(self): + inputs = { + "socket1": [{"sender": None, "value": _NO_OUTPUT_PRODUCED}] + } + assert has_user_input(inputs) is True + +class TestPipelineInputCapability: + def test_cannot_receive_inputs_no_senders(self): + component = { + "input_sockets": { + "socket1": InputSocket("socket1", int), + "socket2": InputSocket("socket2", str) + } + } + assert can_not_receive_inputs_from_pipeline(component) is True + + def test_cannot_receive_inputs_with_senders(self, component_with_multiple_sockets): + assert can_not_receive_inputs_from_pipeline(component_with_multiple_sockets) is False + + def test_cannot_receive_inputs_mixed_senders(self, input_socket_with_sender): + component = { + "input_sockets": { + "socket1": input_socket_with_sender, + "socket2": InputSocket("socket2", str) # No senders + } + } + assert can_not_receive_inputs_from_pipeline(component) is False + +class TestSocketExecutionStatus: + def test_regular_socket_predecessor_executed(self, input_socket_with_sender): + socket_inputs = [{"sender": "component1", "value": 42}] + assert all_socket_predecessors_executed(input_socket_with_sender, socket_inputs) is True + + def test_regular_socket_predecessor_not_executed(self, input_socket_with_sender): + socket_inputs = [] + assert all_socket_predecessors_executed(input_socket_with_sender, socket_inputs) is False + + def test_regular_socket_with_wrong_predecessor(self, input_socket_with_sender): + socket_inputs = [{"sender": "component2", "value": 42}] + assert all_socket_predecessors_executed(input_socket_with_sender, socket_inputs) is False + + def test_variadic_socket_all_predecessors_executed(self, variadic_socket_with_senders): + socket_inputs = [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is True + + def test_variadic_socket_partial_execution(self, variadic_socket_with_senders): + socket_inputs = [{"sender": "component1", "value": 42}] + assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is False + + def test_variadic_socket_with_user_input(self, variadic_socket_with_senders): + socket_inputs = [ + {"sender": "component1", "value": 42}, + {"sender": None, "value": 43}, + {"sender": "component2", "value": 44} + ] + assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is True + + def test_variadic_socket_no_execution(self, variadic_socket_with_senders): + socket_inputs = [] + assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is False + +class TestSocketInputReceived: + def test_any_socket_input_received_with_value(self): + socket_inputs = [{"sender": "component1", "value": 42}] + assert any_socket_input_received(socket_inputs) is True + + def test_any_socket_input_received_with_no_output(self): + socket_inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] + assert any_socket_input_received(socket_inputs) is False + + def test_any_socket_input_received_mixed_inputs(self): + socket_inputs = [ + {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "component2", "value": 42} + ] + assert any_socket_input_received(socket_inputs) is True + + def test_any_socket_input_received_empty_list(self): + assert any_socket_input_received([]) is False + +class TestLazyVariadicSocket: + def test_lazy_variadic_all_inputs_received(self, variadic_socket_with_senders): + socket_inputs = [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is True + + def test_lazy_variadic_partial_inputs(self, variadic_socket_with_senders): + socket_inputs = [{"sender": "component1", "value": 42}] + assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is False + + def test_lazy_variadic_with_no_output(self, variadic_socket_with_senders): + socket_inputs = [ + {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "component2", "value": 42} + ] + assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is False + + def test_lazy_variadic_with_user_input(self, variadic_socket_with_senders): + socket_inputs = [ + {"sender": "component1", "value": 42}, + {"sender": None, "value": 43}, + {"sender": "component2", "value": 44} + ] + assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is True + + def test_lazy_variadic_empty_inputs(self, variadic_socket_with_senders): + assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, []) is False + +class TestSocketTypeDetection: + def test_is_socket_lazy_variadic_with_lazy_socket(self, lazy_variadic_socket): + assert is_socket_lazy_variadic(lazy_variadic_socket) is True + + def test_is_socket_lazy_variadic_with_greedy_socket(self, greedy_variadic_socket): + assert is_socket_lazy_variadic(greedy_variadic_socket) is False + + def test_is_socket_lazy_variadic_with_regular_socket(self, regular_socket): + assert is_socket_lazy_variadic(regular_socket) is False + +class TestSocketInputCompletion: + def test_regular_socket_complete(self, regular_socket): + inputs = [{"sender": "component1", "value": 42}] + assert has_socket_received_all_inputs(regular_socket, inputs) is True + + def test_regular_socket_incomplete(self, regular_socket): + inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] + assert has_socket_received_all_inputs(regular_socket, inputs) is False + + def test_regular_socket_no_inputs(self, regular_socket): + inputs = [] + assert has_socket_received_all_inputs(regular_socket, inputs) is False + + def test_lazy_variadic_socket_all_inputs(self, lazy_variadic_socket): + inputs = [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is True + + def test_lazy_variadic_socket_partial_inputs(self, lazy_variadic_socket): + inputs = [{"sender": "component1", "value": 42}] + assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is False + + def test_lazy_variadic_socket_with_no_output(self, lazy_variadic_socket): + inputs = [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": _NO_OUTPUT_PRODUCED} + ] + assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is False + + def test_greedy_variadic_socket_one_input(self, greedy_variadic_socket): + inputs = [{"sender": "component1", "value": 42}] + assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is True + + def test_greedy_variadic_socket_multiple_inputs(self, greedy_variadic_socket): + inputs = [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is True + + def test_greedy_variadic_socket_no_valid_inputs(self, greedy_variadic_socket): + inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] + assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is False + +class TestPredecessorExecution: + def test_all_predecessors_executed_complete(self, complex_component): + inputs = { + "regular": [{"sender": "component1", "value": 42}], + "lazy_var": [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ], + "greedy_var": [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43}, + {"sender": "component3", "value": 44} + ] + } + assert all_predecessors_executed(complex_component, inputs) is True + + def test_all_predecessors_executed_partial(self, complex_component): + inputs = { + "regular": [{"sender": "component1", "value": 42}], + "lazy_var": [{"sender": "component1", "value": 42}], # Missing component2 + "greedy_var": [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + } + assert all_predecessors_executed(complex_component, inputs) is False + + def test_all_predecessors_executed_with_user_input(self, complex_component): + inputs = { + "regular": [{"sender": "component1", "value": 42}], + "lazy_var": [ + {"sender": "component1", "value": 42}, + {"sender": None, "value": 43} # User input shouldn't affect predecessor execution + ], + "greedy_var": [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43}, + {"sender": "component3", "value": 44} + ] + } + assert all_predecessors_executed(complex_component, inputs) is False + +class TestLazyVariadicResolution: + def test_lazy_variadic_sockets_all_resolved(self, complex_component): + inputs = { + "lazy_var": [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + } + assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is True + + def test_lazy_variadic_sockets_partially_resolved(self, complex_component): + inputs = { + "lazy_var": [{"sender": "component1", "value": 42}] # Missing component2 + } + assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is False + + def test_lazy_variadic_sockets_with_no_inputs(self, complex_component): + inputs = {} + assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is False + + def test_lazy_variadic_sockets_with_predecessors_executed(self, complex_component): + inputs = { + "lazy_var": [ + {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "component2", "value": _NO_OUTPUT_PRODUCED} + ] + } + # All predecessors executed but produced no output + assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is True + +class TestGreedySocketReadiness: + def test_greedy_socket_ready(self, complex_component): + inputs = { + "greedy_var": [{"sender": "component1", "value": 42}] + } + assert is_any_greedy_socket_ready(complex_component, inputs) is True + + def test_greedy_socket_multiple_inputs_ready(self, complex_component): + inputs = { + "greedy_var": [ + {"sender": "component1", "value": 42}, + {"sender": "component2", "value": 43} + ] + } + assert is_any_greedy_socket_ready(complex_component, inputs) is True + + def test_greedy_socket_not_ready(self, complex_component): + inputs = { + "greedy_var": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] + } + assert is_any_greedy_socket_ready(complex_component, inputs) is False + + def test_greedy_socket_no_inputs(self, complex_component): + inputs = {} + assert is_any_greedy_socket_ready(complex_component, inputs) is False + + def test_greedy_socket_with_user_input(self, complex_component): + inputs = { + "greedy_var": [{"sender": None, "value": 42}] + } + assert is_any_greedy_socket_ready(complex_component, inputs) is True \ No newline at end of file From 21c78b84f1881078a2573869a700ed565506eb4b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 13:56:31 +0100 Subject: [PATCH 10/88] adapt tests for component checks (o1 review) --- test/core/pipeline/test_component_checks.py | 217 ++++++++++++++++---- 1 file changed, 178 insertions(+), 39 deletions(-) diff --git a/test/core/pipeline/test_component_checks.py b/test/core/pipeline/test_component_checks.py index 418d5e3814..3a1d0e0e19 100644 --- a/test/core/pipeline/test_component_checks.py +++ b/test/core/pipeline/test_component_checks.py @@ -5,9 +5,10 @@ from haystack.core.pipeline.component_checks import _NO_OUTPUT_PRODUCED from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic + @pytest.fixture def basic_component(): - """Basic component with one mandatory and one optional input""" + """Basic component with one mandatory and one optional input.""" return { "instance": "mock_instance", "visits": 0, @@ -22,7 +23,7 @@ def basic_component(): @pytest.fixture def variadic_component(): - """Component with variadic input""" + """Component with variadic input.""" return { "instance": "mock_instance", "visits": 0, @@ -37,7 +38,7 @@ def variadic_component(): @pytest.fixture def greedy_variadic_component(): - """Component with greedy variadic input""" + """Component with greedy variadic input.""" return { "instance": "mock_instance", "visits": 0, @@ -52,21 +53,21 @@ def greedy_variadic_component(): @pytest.fixture def input_socket_with_sender(): - """Regular input socket with a single sender""" + """Regular input socket with a single sender.""" socket = InputSocket("test_input", int) socket.senders = ["component1"] return socket @pytest.fixture def variadic_socket_with_senders(): - """Variadic input socket with multiple senders""" + """Variadic input socket with multiple senders.""" socket = InputSocket("test_variadic", Variadic[int]) socket.senders = ["component1", "component2"] return socket @pytest.fixture def component_with_multiple_sockets(input_socket_with_sender, variadic_socket_with_senders): - """Component with multiple input sockets including both regular and variadic""" + """Component with multiple input sockets including both regular and variadic.""" return { "instance": "mock_instance", "input_sockets": { @@ -78,28 +79,28 @@ def component_with_multiple_sockets(input_socket_with_sender, variadic_socket_wi @pytest.fixture def regular_socket(): - """Regular input socket with one sender""" + """Regular input socket with one sender.""" socket = InputSocket("regular", int) socket.senders = ["component1"] return socket @pytest.fixture def lazy_variadic_socket(): - """Lazy variadic input socket with multiple senders""" + """Lazy variadic input socket with multiple senders.""" socket = InputSocket("lazy_variadic", Variadic[int]) socket.senders = ["component1", "component2"] return socket @pytest.fixture def greedy_variadic_socket(): - """Greedy variadic input socket with multiple senders""" + """Greedy variadic input socket with multiple senders.""" socket = InputSocket("greedy_variadic", GreedyVariadic[int]) socket.senders = ["component1", "component2", "component3"] return socket @pytest.fixture def complex_component(regular_socket, lazy_variadic_socket, greedy_variadic_socket): - """Component with all types of sockets""" + """Component with all types of sockets.""" return { "instance": "mock_instance", "input_sockets": { @@ -109,14 +110,17 @@ def complex_component(regular_socket, lazy_variadic_socket, greedy_variadic_sock } } + class TestCanComponentRun: def test_component_with_all_mandatory_inputs_and_trigger(self, basic_component): + """Checks that the component runs if all mandatory inputs are received and triggered.""" inputs = { "mandatory_input": [{"sender": "previous_component", "value": 42}] } assert can_component_run(basic_component, inputs) is True def test_component_missing_mandatory_input(self, basic_component): + """Checks that the component won't run if mandatory inputs are missing.""" inputs = { "optional_input": [{"sender": "previous_component", "value": "test"}] } @@ -125,16 +129,16 @@ def test_component_missing_mandatory_input(self, basic_component): def test_component_with_no_trigger_but_all_inputs(self, basic_component): """ Test case where all mandatory inputs are present with valid values, - but there is no trigger (no new input from predecessor, not first visit) + but there is no trigger (no new input from predecessor, not first visit). """ - # Set visits > 0 so it's not triggered by first visit basic_component["visits"] = 1 inputs = { - "mandatory_input": [{"sender": None, "value": 42}] # Valid input value + "mandatory_input": [{"sender": None, "value": 42}] } assert can_component_run(basic_component, inputs) is False def test_component_with_multiple_visits(self, basic_component): + """Checks that a component can still be triggered on subsequent visits by a predecessor.""" basic_component["visits"] = 2 inputs = { "mandatory_input": [{"sender": "previous_component", "value": 42}] @@ -142,24 +146,43 @@ def test_component_with_multiple_visits(self, basic_component): assert can_component_run(basic_component, inputs) is True def test_component_with_no_inputs_first_visit(self, basic_component): - basic_component["input_sockets"] = {} # No inputs required + """Checks that a component with no input sockets can be triggered on its first visit.""" + basic_component["input_sockets"] = {} inputs = {} assert can_component_run(basic_component, inputs) is True + def test_component_triggered_on_second_visit_with_new_input(self, basic_component): + """ + Tests that a second visit is triggered if new predecessor input arrives + (i.e. visits > 0, but a valid new input from a predecessor is provided). + """ + # First, simulate that the component has already run once. + basic_component["visits"] = 1 + + # Now a predecessor provides a new input; this should re-trigger execution. + inputs = { + "mandatory_input": [{"sender": "previous_component", "value": 99}] + } + assert can_component_run(basic_component, inputs) is True + + class TestHasAnyTrigger: def test_trigger_from_predecessor(self, basic_component): + """Ensures that new data from a predecessor can trigger a component.""" inputs = { "mandatory_input": [{"sender": "previous_component", "value": 42}] } assert has_any_trigger(basic_component, inputs) is True def test_trigger_from_user_first_visit(self, basic_component): + """Checks that user input (sender=None) triggers the component on the first visit.""" inputs = { "mandatory_input": [{"sender": None, "value": 42}] } assert has_any_trigger(basic_component, inputs) is True def test_no_trigger_from_user_after_first_visit(self, basic_component): + """Checks that user input no longer triggers the component after the first visit.""" basic_component["visits"] = 1 inputs = { "mandatory_input": [{"sender": None, "value": 42}] @@ -167,30 +190,36 @@ def test_no_trigger_from_user_after_first_visit(self, basic_component): assert has_any_trigger(basic_component, inputs) is False def test_trigger_without_inputs_first_visit(self, basic_component): - basic_component["input_sockets"] = {} # No inputs + """Checks that a component with no inputs is triggered on the first visit.""" + basic_component["input_sockets"] = {} inputs = {} assert has_any_trigger(basic_component, inputs) is True def test_no_trigger_without_inputs_after_first_visit(self, basic_component): + """Checks that on subsequent visits, no inputs means no trigger.""" basic_component["input_sockets"] = {} basic_component["visits"] = 1 inputs = {} assert has_any_trigger(basic_component, inputs) is False + class TestAllMandatorySocketsReady: def test_all_mandatory_sockets_filled(self, basic_component): + """Checks that all mandatory sockets are ready when they have valid input.""" inputs = { "mandatory_input": [{"sender": "previous_component", "value": 42}] } assert are_all_mandatory_sockets_ready(basic_component, inputs) is True def test_missing_mandatory_socket(self, basic_component): + """Ensures that if a mandatory socket is missing, the component is not ready.""" inputs = { "optional_input": [{"sender": "previous_component", "value": "test"}] } assert are_all_mandatory_sockets_ready(basic_component, inputs) is False def test_variadic_socket_with_input(self, variadic_component): + """Verifies that a variadic socket is considered filled if it has at least one input.""" inputs = { "variadic_input": [{"sender": "previous_component", "value": 42}], "normal_input": [{"sender": "previous_component", "value": "test"}] @@ -198,6 +227,7 @@ def test_variadic_socket_with_input(self, variadic_component): assert are_all_mandatory_sockets_ready(variadic_component, inputs) is True def test_greedy_variadic_socket_with_partial_input(self, greedy_variadic_component): + """Greedy variadic sockets are ready with at least one valid input.""" inputs = { "greedy_input": [{"sender": "previous_component", "value": 42}], "normal_input": [{"sender": "previous_component", "value": "test"}] @@ -205,16 +235,19 @@ def test_greedy_variadic_socket_with_partial_input(self, greedy_variadic_compone assert are_all_mandatory_sockets_ready(greedy_variadic_component, inputs) is True def test_variadic_socket_no_input(self, variadic_component): + """A variadic socket is not filled if it has zero valid inputs.""" inputs = { "normal_input": [{"sender": "previous_component", "value": "test"}] } assert are_all_mandatory_sockets_ready(variadic_component, inputs) is False def test_empty_inputs(self, basic_component): + """Checks that if there are no inputs at all, mandatory sockets are not ready.""" inputs = {} assert are_all_mandatory_sockets_ready(basic_component, inputs) is False def test_no_mandatory_sockets(self, basic_component): + """Ensures that if there are no mandatory sockets, the component is considered ready.""" basic_component["input_sockets"] = { "optional_1": InputSocket("optional_1", str, default_value="default1"), "optional_2": InputSocket("optional_2", str, default_value="default2") @@ -223,6 +256,7 @@ def test_no_mandatory_sockets(self, basic_component): assert are_all_mandatory_sockets_ready(basic_component, inputs) is True def test_multiple_mandatory_sockets(self, basic_component): + """Checks readiness when multiple mandatory sockets are defined.""" basic_component["input_sockets"] = { "mandatory_1": InputSocket("mandatory_1", int), "mandatory_2": InputSocket("mandatory_2", str), @@ -241,8 +275,12 @@ def test_multiple_mandatory_sockets(self, basic_component): } assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + class TestPredecessorInputDetection: def test_any_predecessors_provided_input_with_predecessor(self, component_with_multiple_sockets): + """ + Tests detection of predecessor input when a valid predecessor sends data. + """ inputs = { "socket1": [{"sender": "component1", "value": 42}], "socket2": [{"sender": None, "value": "test"}] @@ -250,6 +288,9 @@ def test_any_predecessors_provided_input_with_predecessor(self, component_with_m assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is True def test_any_predecessors_provided_input_no_predecessor(self, component_with_multiple_sockets): + """ + Checks that no predecessor inputs are detected if all senders are None (user inputs). + """ inputs = { "socket1": [{"sender": None, "value": 42}], "socket2": [{"sender": None, "value": "test"}] @@ -257,6 +298,9 @@ def test_any_predecessors_provided_input_no_predecessor(self, component_with_mul assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False def test_any_predecessors_provided_input_with_no_output(self, component_with_multiple_sockets): + """ + Ensures that _NO_OUTPUT_PRODUCED from a predecessor is ignored in the predecessor detection. + """ inputs = { "socket1": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}], "socket2": [{"sender": None, "value": "test"}] @@ -264,35 +308,58 @@ def test_any_predecessors_provided_input_with_no_output(self, component_with_mul assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False def test_any_predecessors_provided_input_empty_inputs(self, component_with_multiple_sockets): + """Ensures that empty inputs dictionary returns False.""" inputs = {} assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False -class TestSocketValueFromPredecessor: - def test_socket_value_from_predecessor_with_valid_input(self): - socket_inputs = [{"sender": "component1", "value": 42}] - assert any_socket_value_from_predecessor_received(socket_inputs) is True - def test_socket_value_from_predecessor_with_no_output(self): - socket_inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] - assert any_socket_value_from_predecessor_received(socket_inputs) is False - - def test_socket_value_from_predecessor_with_user_input(self): - socket_inputs = [{"sender": None, "value": 42}] - assert any_socket_value_from_predecessor_received(socket_inputs) is False - - def test_socket_value_from_predecessor_with_mixed_inputs(self): - socket_inputs = [ - {"sender": None, "value": 42}, - {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, - {"sender": "component2", "value": 100} - ] - assert any_socket_value_from_predecessor_received(socket_inputs) is True +class TestSocketValueFromPredecessor: + """ + Tests for `any_socket_value_from_predecessor_received`, verifying whether + any predecessor component provided valid output to a socket. + """ + + @pytest.mark.parametrize( + "socket_inputs, expected_result", + [ + pytest.param( + [{"sender": "component1", "value": 42}], + True, + id="valid_input" + ), + pytest.param( + [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}], + False, + id="no_output" + ), + pytest.param( + [{"sender": None, "value": 42}], + False, + id="user_input" + ), + pytest.param( + [ + {"sender": None, "value": 42}, + {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "component2", "value": 100} + ], + True, + id="mixed_inputs" + ), + pytest.param([], False, id="empty_list"), + ], + ) + def test_any_socket_value_from_predecessor_received(self, socket_inputs, expected_result): + """ + Parametrized test to check whether any valid predecessor input + exists in a list of socket inputs. + """ + assert any_socket_value_from_predecessor_received(socket_inputs) == expected_result - def test_socket_value_from_predecessor_empty_list(self): - assert any_socket_value_from_predecessor_received([]) is False class TestUserInputDetection: def test_has_user_input_with_user_input(self): + """Checks that having a sender=None input means user input is present.""" inputs = { "socket1": [{"sender": None, "value": 42}], "socket2": [{"sender": "component1", "value": "test"}] @@ -300,6 +367,7 @@ def test_has_user_input_with_user_input(self): assert has_user_input(inputs) is True def test_has_user_input_without_user_input(self): + """Ensures that if all senders are component-based, there's no user input.""" inputs = { "socket1": [{"sender": "component1", "value": 42}], "socket2": [{"sender": "component2", "value": "test"}] @@ -307,17 +375,24 @@ def test_has_user_input_without_user_input(self): assert has_user_input(inputs) is False def test_has_user_input_empty_inputs(self): + """Checks that an empty inputs dict has no user input.""" inputs = {} assert has_user_input(inputs) is False def test_has_user_input_with_no_output(self): + """ + Even if the input value is _NO_OUTPUT_PRODUCED, if sender=None + it still counts as user input being provided. + """ inputs = { "socket1": [{"sender": None, "value": _NO_OUTPUT_PRODUCED}] } assert has_user_input(inputs) is True + class TestPipelineInputCapability: def test_cannot_receive_inputs_no_senders(self): + """Checks that a component with zero senders for each socket cannot receive pipeline inputs.""" component = { "input_sockets": { "socket1": InputSocket("socket1", int), @@ -327,9 +402,11 @@ def test_cannot_receive_inputs_no_senders(self): assert can_not_receive_inputs_from_pipeline(component) is True def test_cannot_receive_inputs_with_senders(self, component_with_multiple_sockets): + """If at least one socket has a sender, the component can receive pipeline inputs.""" assert can_not_receive_inputs_from_pipeline(component_with_multiple_sockets) is False def test_cannot_receive_inputs_mixed_senders(self, input_socket_with_sender): + """A single socket with a sender means the component can receive pipeline inputs.""" component = { "input_sockets": { "socket1": input_socket_with_sender, @@ -338,20 +415,25 @@ def test_cannot_receive_inputs_mixed_senders(self, input_socket_with_sender): } assert can_not_receive_inputs_from_pipeline(component) is False + class TestSocketExecutionStatus: def test_regular_socket_predecessor_executed(self, input_socket_with_sender): + """Verifies that if the correct sender provides a value, the socket is marked as executed.""" socket_inputs = [{"sender": "component1", "value": 42}] assert all_socket_predecessors_executed(input_socket_with_sender, socket_inputs) is True def test_regular_socket_predecessor_not_executed(self, input_socket_with_sender): + """If there are no inputs, the predecessor is not considered executed.""" socket_inputs = [] assert all_socket_predecessors_executed(input_socket_with_sender, socket_inputs) is False def test_regular_socket_with_wrong_predecessor(self, input_socket_with_sender): + """Checks that a mismatch in sender means the socket is not yet executed.""" socket_inputs = [{"sender": "component2", "value": 42}] assert all_socket_predecessors_executed(input_socket_with_sender, socket_inputs) is False def test_variadic_socket_all_predecessors_executed(self, variadic_socket_with_senders): + """Variadic socket is executed only if all senders have produced at least one valid result.""" socket_inputs = [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": 43} @@ -359,10 +441,15 @@ def test_variadic_socket_all_predecessors_executed(self, variadic_socket_with_se assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is True def test_variadic_socket_partial_execution(self, variadic_socket_with_senders): + """If only one of multiple senders produced an output, not all predecessors are executed.""" socket_inputs = [{"sender": "component1", "value": 42}] assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is False def test_variadic_socket_with_user_input(self, variadic_socket_with_senders): + """ + User input (sender=None) doesn't block the socket from being 'executed' if + all named predecessors have also produced outputs. + """ socket_inputs = [ {"sender": "component1", "value": 42}, {"sender": None, "value": 43}, @@ -371,19 +458,24 @@ def test_variadic_socket_with_user_input(self, variadic_socket_with_senders): assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is True def test_variadic_socket_no_execution(self, variadic_socket_with_senders): + """Empty inputs means no predecessor has executed.""" socket_inputs = [] assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is False + class TestSocketInputReceived: def test_any_socket_input_received_with_value(self): + """Checks that if there's a non-_NO_OUTPUT_PRODUCED value, the socket is marked as having input.""" socket_inputs = [{"sender": "component1", "value": 42}] assert any_socket_input_received(socket_inputs) is True def test_any_socket_input_received_with_no_output(self): + """If all inputs are _NO_OUTPUT_PRODUCED, the socket has no effective input.""" socket_inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] assert any_socket_input_received(socket_inputs) is False def test_any_socket_input_received_mixed_inputs(self): + """A single valid input among many is enough to consider the socket as having input.""" socket_inputs = [ {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, {"sender": "component2", "value": 42} @@ -391,10 +483,13 @@ def test_any_socket_input_received_mixed_inputs(self): assert any_socket_input_received(socket_inputs) is True def test_any_socket_input_received_empty_list(self): + """Empty list: no input received.""" assert any_socket_input_received([]) is False + class TestLazyVariadicSocket: def test_lazy_variadic_all_inputs_received(self, variadic_socket_with_senders): + """Lazy variadic socket is ready only if all named senders provided outputs.""" socket_inputs = [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": 43} @@ -402,10 +497,12 @@ def test_lazy_variadic_all_inputs_received(self, variadic_socket_with_senders): assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is True def test_lazy_variadic_partial_inputs(self, variadic_socket_with_senders): + """Partial inputs from only some senders is insufficient for a lazy variadic socket.""" socket_inputs = [{"sender": "component1", "value": 42}] assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is False def test_lazy_variadic_with_no_output(self, variadic_socket_with_senders): + """_NO_OUTPUT_PRODUCED from a sender doesn't count as valid input, so it's not fully received.""" socket_inputs = [ {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, {"sender": "component2", "value": 42} @@ -413,6 +510,10 @@ def test_lazy_variadic_with_no_output(self, variadic_socket_with_senders): assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is False def test_lazy_variadic_with_user_input(self, variadic_socket_with_senders): + """ + User input doesn't block a lazy variadic socket, as long as all named senders + also provided outputs. + """ socket_inputs = [ {"sender": "component1", "value": 42}, {"sender": None, "value": 43}, @@ -421,32 +522,42 @@ def test_lazy_variadic_with_user_input(self, variadic_socket_with_senders): assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is True def test_lazy_variadic_empty_inputs(self, variadic_socket_with_senders): + """No inputs at all means the lazy variadic socket hasn't received everything yet.""" assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, []) is False + class TestSocketTypeDetection: def test_is_socket_lazy_variadic_with_lazy_socket(self, lazy_variadic_socket): + """Ensures that a non-greedy variadic socket is detected as lazy.""" assert is_socket_lazy_variadic(lazy_variadic_socket) is True def test_is_socket_lazy_variadic_with_greedy_socket(self, greedy_variadic_socket): + """Greedy variadic sockets should not be marked as lazy.""" assert is_socket_lazy_variadic(greedy_variadic_socket) is False def test_is_socket_lazy_variadic_with_regular_socket(self, regular_socket): + """Regular sockets are not variadic at all.""" assert is_socket_lazy_variadic(regular_socket) is False + class TestSocketInputCompletion: def test_regular_socket_complete(self, regular_socket): + """A single valid input marks a regular socket as complete.""" inputs = [{"sender": "component1", "value": 42}] assert has_socket_received_all_inputs(regular_socket, inputs) is True def test_regular_socket_incomplete(self, regular_socket): + """_NO_OUTPUT_PRODUCED means the socket is not complete.""" inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] assert has_socket_received_all_inputs(regular_socket, inputs) is False def test_regular_socket_no_inputs(self, regular_socket): + """No inputs at all means the socket is incomplete.""" inputs = [] assert has_socket_received_all_inputs(regular_socket, inputs) is False def test_lazy_variadic_socket_all_inputs(self, lazy_variadic_socket): + """Lazy variadic socket is complete only if all senders have produced valid outputs.""" inputs = [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": 43} @@ -454,10 +565,12 @@ def test_lazy_variadic_socket_all_inputs(self, lazy_variadic_socket): assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is True def test_lazy_variadic_socket_partial_inputs(self, lazy_variadic_socket): + """Partial coverage of senders is insufficient for lazy variadic sockets.""" inputs = [{"sender": "component1", "value": 42}] assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is False def test_lazy_variadic_socket_with_no_output(self, lazy_variadic_socket): + """A sender that produces _NO_OUTPUT_PRODUCED does not fulfill the lazy socket requirement.""" inputs = [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": _NO_OUTPUT_PRODUCED} @@ -465,10 +578,12 @@ def test_lazy_variadic_socket_with_no_output(self, lazy_variadic_socket): assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is False def test_greedy_variadic_socket_one_input(self, greedy_variadic_socket): + """A greedy variadic socket is complete if it has at least one valid input.""" inputs = [{"sender": "component1", "value": 42}] assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is True def test_greedy_variadic_socket_multiple_inputs(self, greedy_variadic_socket): + """A greedy variadic socket with multiple inputs remains complete as soon as one is valid.""" inputs = [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": 43} @@ -476,11 +591,17 @@ def test_greedy_variadic_socket_multiple_inputs(self, greedy_variadic_socket): assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is True def test_greedy_variadic_socket_no_valid_inputs(self, greedy_variadic_socket): + """All _NO_OUTPUT_PRODUCED means the greedy socket is not complete.""" inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is False + class TestPredecessorExecution: def test_all_predecessors_executed_complete(self, complex_component): + """ + Checks that if all named senders produce valid outputs for each socket, + then all predecessors are considered executed. + """ inputs = { "regular": [{"sender": "component1", "value": 42}], "lazy_var": [ @@ -496,6 +617,7 @@ def test_all_predecessors_executed_complete(self, complex_component): assert all_predecessors_executed(complex_component, inputs) is True def test_all_predecessors_executed_partial(self, complex_component): + """If a lazy socket is missing one predecessor, not all predecessors are executed.""" inputs = { "regular": [{"sender": "component1", "value": 42}], "lazy_var": [{"sender": "component1", "value": 42}], # Missing component2 @@ -507,11 +629,15 @@ def test_all_predecessors_executed_partial(self, complex_component): assert all_predecessors_executed(complex_component, inputs) is False def test_all_predecessors_executed_with_user_input(self, complex_component): + """ + User input shouldn't affect predecessor execution for the lazy socket: + we still need all named senders to produce output. + """ inputs = { "regular": [{"sender": "component1", "value": 42}], "lazy_var": [ {"sender": "component1", "value": 42}, - {"sender": None, "value": 43} # User input shouldn't affect predecessor execution + {"sender": None, "value": 43} ], "greedy_var": [ {"sender": "component1", "value": 42}, @@ -521,8 +647,10 @@ def test_all_predecessors_executed_with_user_input(self, complex_component): } assert all_predecessors_executed(complex_component, inputs) is False + class TestLazyVariadicResolution: def test_lazy_variadic_sockets_all_resolved(self, complex_component): + """Checks that lazy variadic sockets are resolved when all inputs have arrived.""" inputs = { "lazy_var": [ {"sender": "component1", "value": 42}, @@ -532,33 +660,41 @@ def test_lazy_variadic_sockets_all_resolved(self, complex_component): assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is True def test_lazy_variadic_sockets_partially_resolved(self, complex_component): + """Missing some sender outputs means lazy variadic sockets are not resolved.""" inputs = { "lazy_var": [{"sender": "component1", "value": 42}] # Missing component2 } assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is False def test_lazy_variadic_sockets_with_no_inputs(self, complex_component): + """No inputs: lazy variadic socket is not resolved.""" inputs = {} assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is False def test_lazy_variadic_sockets_with_predecessors_executed(self, complex_component): + """ + Ensures that if all predecessors have executed (but produced no output), + the lazy variadic socket is still considered resolved. + """ inputs = { "lazy_var": [ {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, {"sender": "component2", "value": _NO_OUTPUT_PRODUCED} ] } - # All predecessors executed but produced no output assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is True + class TestGreedySocketReadiness: def test_greedy_socket_ready(self, complex_component): + """A single valid input is enough for a greedy variadic socket to be considered ready.""" inputs = { "greedy_var": [{"sender": "component1", "value": 42}] } assert is_any_greedy_socket_ready(complex_component, inputs) is True def test_greedy_socket_multiple_inputs_ready(self, complex_component): + """Multiple valid inputs on a greedy socket is also fine—it's still ready.""" inputs = { "greedy_var": [ {"sender": "component1", "value": 42}, @@ -568,17 +704,20 @@ def test_greedy_socket_multiple_inputs_ready(self, complex_component): assert is_any_greedy_socket_ready(complex_component, inputs) is True def test_greedy_socket_not_ready(self, complex_component): + """If the only input is _NO_OUTPUT_PRODUCED, the greedy socket isn't ready.""" inputs = { "greedy_var": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] } assert is_any_greedy_socket_ready(complex_component, inputs) is False def test_greedy_socket_no_inputs(self, complex_component): + """No inputs at all: the greedy socket is not ready.""" inputs = {} assert is_any_greedy_socket_ready(complex_component, inputs) is False def test_greedy_socket_with_user_input(self, complex_component): + """User input can also trigger readiness for a greedy variadic socket.""" inputs = { "greedy_var": [{"sender": None, "value": 42}] } - assert is_any_greedy_socket_ready(complex_component, inputs) is True \ No newline at end of file + assert is_any_greedy_socket_ready(complex_component, inputs) is True From cf23b325dd6e676d5b41fa87c8350bfd10c99ccc Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 13:59:49 +0100 Subject: [PATCH 11/88] chore: format --- haystack/core/component/component.py | 6 +- haystack/core/pipeline/base.py | 2 +- haystack/core/pipeline/component_checks.py | 37 ++- haystack/core/pipeline/pipeline.py | 39 ++- haystack/core/pipeline/utils.py | 1 + pyproject.toml | 2 +- test/components/audio/test_whisper_local.py | 12 +- .../converters/test_docx_file_to_document.py | 6 +- .../test_openai_document_embedder.py | 6 +- .../embedders/test_openai_text_embedder.py | 6 +- .../joiners/test_document_joiner.py | 6 +- .../routers/test_conditional_router.py | 6 +- test/core/pipeline/features/test_run.py | 20 +- test/core/pipeline/test_component_checks.py | 238 ++++++------------ 14 files changed, 165 insertions(+), 222 deletions(-) diff --git a/haystack/core/component/component.py b/haystack/core/component/component.py index d77fd77593..fe6902b2f1 100644 --- a/haystack/core/component/component.py +++ b/haystack/core/component/component.py @@ -268,9 +268,9 @@ def __call__(cls, *args, **kwargs): try: pre_init_hook.in_progress = True named_positional_args = ComponentMeta._positional_to_kwargs(cls, args) - assert set(named_positional_args.keys()).intersection(kwargs.keys()) == set(), ( - "positional and keyword arguments overlap" - ) + assert ( + set(named_positional_args.keys()).intersection(kwargs.keys()) == set() + ), "positional and keyword arguments overlap" kwargs.update(named_positional_args) pre_init_hook.callback(cls, kwargs) instance = super().__call__(**kwargs) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 1bfbf42f6f..730cd888dd 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -815,6 +815,7 @@ def _find_receivers_from(self, component_name: str) -> List[Tuple[str, OutputSoc res.append((receiver_name, sender_socket, receiver_socket)) return res + def _connections_status( sender_node: str, receiver_node: str, sender_sockets: List[OutputSocket], receiver_sockets: List[InputSocket] ): @@ -838,4 +839,3 @@ def _connections_status( receiver_sockets_list = "\n".join(receiver_sockets_entries) return f"'{sender_node}':\n{sender_sockets_list}\n'{receiver_node}':\n{receiver_sockets_list}" - diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index 3079878ad3..92a99d89b2 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -8,6 +8,7 @@ _NO_OUTPUT_PRODUCED = _empty + def can_component_run(component: Dict, inputs: Dict) -> bool: """ Checks if the component can run, given the current state of its inputs. @@ -22,6 +23,7 @@ def can_component_run(component: Dict, inputs: Dict) -> bool: return received_all_mandatory_inputs and received_trigger + def has_any_trigger(component: Dict, inputs: Dict) -> bool: """ Checks if a component was triggered to execute. @@ -45,6 +47,7 @@ def has_any_trigger(component: Dict, inputs: Dict) -> bool: return trigger_from_predecessor or trigger_from_user or trigger_without_inputs + def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: """ Checks if all mandatory sockets of a component have enough inputs for the component to execute. @@ -64,6 +67,7 @@ def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: return filled_mandatory_sockets == expected_mandatory_sockets + def any_predecessors_provided_input(component: Dict, inputs: Dict) -> bool: """ Checks if a component received inputs from any predecessors. @@ -71,7 +75,11 @@ def any_predecessors_provided_input(component: Dict, inputs: Dict) -> bool: :param component: Component metadata and the component instance. :param inputs: Inputs for the component. """ - return any(any_socket_value_from_predecessor_received(inputs.get(socket_name, [])) for socket_name in component["input_sockets"].keys()) + return any( + any_socket_value_from_predecessor_received(inputs.get(socket_name, [])) + for socket_name in component["input_sockets"].keys() + ) + def any_socket_value_from_predecessor_received(socket_inputs: List[Dict[str, Any]]) -> bool: """ @@ -81,6 +89,7 @@ def any_socket_value_from_predecessor_received(socket_inputs: List[Dict[str, Any # When sender is None, the input was provided from outside the pipeline. return any(inp["value"] != _NO_OUTPUT_PRODUCED and inp["sender"] is not None for inp in socket_inputs) + def has_user_input(inputs: Dict) -> bool: """ Checks if a component has received input from outside the pipeline (e.g. user input). @@ -88,6 +97,7 @@ def has_user_input(inputs: Dict) -> bool: """ return any(inp for socket in inputs.values() for inp in socket if inp["sender"] is None) + def can_not_receive_inputs_from_pipeline(component: Dict) -> bool: """ Checks if a component can not receive inputs from any other components in the pipeline. @@ -95,6 +105,7 @@ def can_not_receive_inputs_from_pipeline(component: Dict) -> bool: """ return all([len(sock.senders) == 0 for sock in component["input_sockets"].values()]) + def all_socket_predecessors_executed(socket: InputSocket, socket_inputs: List[Dict]) -> bool: """ Checks if all components connecting to an InputSocket have executed. @@ -106,6 +117,7 @@ def all_socket_predecessors_executed(socket: InputSocket, socket_inputs: List[Di return expected_senders == executed_senders + def any_socket_input_received(socket_inputs: List[Dict]) -> bool: """ Checks if a socket has received any input from any other components in the pipeline or from outside the pipeline. @@ -113,6 +125,7 @@ def any_socket_input_received(socket_inputs: List[Dict]) -> bool: """ return any(inp["value"] != _NO_OUTPUT_PRODUCED for inp in socket_inputs) + def has_lazy_variadic_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict]) -> bool: """ Checks if a lazy variadic socket has received all expected inputs from other components in the pipeline. @@ -126,6 +139,7 @@ def has_lazy_variadic_socket_received_all_inputs(socket: InputSocket, socket_inp return expected_senders == actual_senders + def is_socket_lazy_variadic(socket: InputSocket) -> bool: """ Checks if an InputSocket is a lazy variadic socket. @@ -133,6 +147,7 @@ def is_socket_lazy_variadic(socket: InputSocket) -> bool: """ return socket.is_variadic and not socket.is_greedy + def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict]) -> bool: """ Checks if a socket has received all expected inputs. @@ -145,14 +160,13 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict # The socket is greedy variadic and at least one input was produced, it is complete. if ( - socket.is_variadic - and socket.is_greedy - and len(socket_inputs) > 0 - and any([sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs]) + socket.is_variadic + and socket.is_greedy + and len(socket_inputs) > 0 + and any([sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs]) ): return True - # The socket is lazy variadic and all expected inputs were produced. if is_socket_lazy_variadic(socket) and has_lazy_variadic_socket_received_all_inputs(socket, socket_inputs): return True @@ -163,6 +177,7 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict return False + def all_predecessors_executed(component: Dict, inputs: Dict) -> bool: """ Checks if all predecessors of a component have executed. @@ -170,10 +185,11 @@ def all_predecessors_executed(component: Dict, inputs: Dict) -> bool: :param inputs: Inputs for the component. """ return all( - all_socket_predecessors_executed(socket, inputs.get(socket_name, [])) - for socket_name, socket in component["input_sockets"].items() + all_socket_predecessors_executed(socket, inputs.get(socket_name, [])) + for socket_name, socket in component["input_sockets"].items() ) + def are_all_lazy_variadic_sockets_resolved(component: Dict, inputs: Dict) -> bool: """ Checks if the final state for all lazy variadic sockets of a component is resolved. @@ -185,13 +201,14 @@ def are_all_lazy_variadic_sockets_resolved(component: Dict, inputs: Dict) -> boo if is_socket_lazy_variadic(socket): socket_inputs = inputs.get(socket_name, []) if not ( - has_lazy_variadic_socket_received_all_inputs(socket, socket_inputs) - or all_socket_predecessors_executed(socket, socket_inputs) + has_lazy_variadic_socket_received_all_inputs(socket, socket_inputs) + or all_socket_predecessors_executed(socket, socket_inputs) ): return False return True + def is_any_greedy_socket_ready(component: Dict, inputs: Dict) -> bool: """ Checks if the component has any greedy socket that is ready to run. diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index fdfc90d46a..7e378ba1a8 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -10,15 +10,23 @@ from haystack.core.component import Component from haystack.core.errors import PipelineMaxComponentRuns, PipelineRuntimeError from haystack.core.pipeline.base import PipelineBase -from haystack.core.pipeline.component_checks import can_component_run, is_any_greedy_socket_ready, \ - is_socket_lazy_variadic -from haystack.core.pipeline.component_checks import all_predecessors_executed, are_all_lazy_variadic_sockets_resolved, _NO_OUTPUT_PRODUCED +from haystack.core.pipeline.component_checks import ( + can_component_run, + is_any_greedy_socket_ready, + is_socket_lazy_variadic, +) +from haystack.core.pipeline.component_checks import ( + all_predecessors_executed, + are_all_lazy_variadic_sockets_resolved, + _NO_OUTPUT_PRODUCED, +) from haystack.core.pipeline.utils import FIFOPriorityQueue from haystack.telemetry import pipeline_running logger = logging.getLogger(__name__) + class ComponentPriority(IntEnum): HIGHEST = 1 READY = 2 @@ -26,6 +34,7 @@ class ComponentPriority(IntEnum): DEFER_LAST = 4 BLOCKED = 5 + class Pipeline(PipelineBase): """ Synchronous version of the orchestration engine. @@ -72,9 +81,7 @@ def _run_component( parent_span=parent_span, ) as span: component_inputs, inputs = self._consume_component_inputs( - component_name=component_name, - component=component, - inputs=inputs + component_name=component_name, component=component, inputs=inputs ) # We deepcopy the inputs otherwise we might lose that information # when we delete them in case they're sent to other Components @@ -125,9 +132,9 @@ def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict # We prune all inputs except for those that were provided from outside the pipeline (e.g. user inputs). pruned_inputs = { socket_name: [ - sock for sock in socket if sock["sender"] is None - and not socket_name in greedy_inputs_to_remove - ] for socket_name, socket in component_inputs.items() + sock for sock in socket if sock["sender"] is None and not socket_name in greedy_inputs_to_remove + ] + for socket_name, socket in component_inputs.items() } pruned_inputs = {socket_name: socket for socket_name, socket in pruned_inputs.items() if len(socket) > 0} @@ -188,7 +195,9 @@ def _calculate_priority(component: Dict, inputs: Dict) -> ComponentPriority: def _get_component_with_graph_metadata(self, component_name: str) -> Dict[str, Any]: return self.graph.nodes[component_name] - def _get_next_runnable_component(self, priority_queue: FIFOPriorityQueue) -> Union[Tuple[Component, str, Dict], None]: + def _get_next_runnable_component( + self, priority_queue: FIFOPriorityQueue + ) -> Union[Tuple[Component, str, Dict], None]: """ Returns the next runnable component alongside its metadata from the priority queue. :param priority_queue: Priority queue of component names. @@ -209,7 +218,9 @@ def _get_next_runnable_component(self, priority_queue: FIFOPriorityQueue) -> Uni return None @staticmethod - def _write_component_outputs(component_name, component_outputs, inputs, receivers, include_outputs_from) -> Tuple[Dict, Dict]: + def _write_component_outputs( + component_name, component_outputs, inputs, receivers, include_outputs_from + ) -> Tuple[Dict, Dict]: """ Distributes the outputs of a component to the input sockets that it is connected to. :param component_name: The name of the component. @@ -252,7 +263,9 @@ def _write_component_outputs(component_name, component_outputs, inputs, receiver return pruned_outputs, inputs @staticmethod - def _merge_component_and_pipeline_outputs(component_name: str, component_outputs: Dict, pipeline_outputs: Dict) -> Dict: + def _merge_component_and_pipeline_outputs( + component_name: str, component_outputs: Dict, pipeline_outputs: Dict + ) -> Dict: """ Merges the outputs of a component with the current pipeline outputs. :param component_name: The name of the component. @@ -425,7 +438,7 @@ def run( # noqa: PLR0915, PLR0912 component_outputs=component_outputs, inputs=inputs, receivers=cached_receivers[component_name], - include_outputs_from=include_outputs_from + include_outputs_from=include_outputs_from, ) # TODO check original logic in pipeline, it looks like we don't want to override existing outputs # e.g. for cycles but the tests check if intermediate outputs from components in cycles are overwritten diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 5950296ab3..ec337806d8 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -22,6 +22,7 @@ def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: return (split_str[0], split_str[1]) return connection, None + class FIFOPriorityQueue: """ A priority queue that maintains FIFO order for items of equal priority. diff --git a/pyproject.toml b/pyproject.toml index 73031b8130..85750257c5 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -88,7 +88,7 @@ extra-dependencies = [ "huggingface_hub>=0.27.0", # Hugging Face API Generators and Embedders "sentence-transformers>=3.0.0", # SentenceTransformersTextEmbedder and SentenceTransformersDocumentEmbedder "langdetect", # TextLanguageRouter and DocumentLanguageClassifier - "openai-whisper>=20231106", # LocalWhisperTranscriber + #"openai-whisper>=20231106", # LocalWhisperTranscriber "arrow>=1.3.0", # Jinja2TimeExtension # NamedEntityExtractor diff --git a/test/components/audio/test_whisper_local.py b/test/components/audio/test_whisper_local.py index 394a9c4000..28463c4ce6 100644 --- a/test/components/audio/test_whisper_local.py +++ b/test/components/audio/test_whisper_local.py @@ -190,14 +190,14 @@ def test_whisper_local_transcriber(self, test_files_path): docs = output["documents"] assert len(docs) == 3 - assert all(word in docs[0].content.strip().lower() for word in {"content", "the", "document"}), ( - f"Expected words not found in: {docs[0].content.strip().lower()}" - ) + assert all( + word in docs[0].content.strip().lower() for word in {"content", "the", "document"} + ), f"Expected words not found in: {docs[0].content.strip().lower()}" assert test_files_path / "audio" / "this is the content of the document.wav" == docs[0].meta["audio_file"] - assert all(word in docs[1].content.strip().lower() for word in {"context", "answer"}), ( - f"Expected words not found in: {docs[1].content.strip().lower()}" - ) + assert all( + word in docs[1].content.strip().lower() for word in {"context", "answer"} + ), f"Expected words not found in: {docs[1].content.strip().lower()}" path = test_files_path / "audio" / "the context for this answer is here.wav" assert path.absolute() == docs[1].meta["audio_file"] diff --git a/test/components/converters/test_docx_file_to_document.py b/test/components/converters/test_docx_file_to_document.py index c013759938..9b4ee3fe60 100644 --- a/test/components/converters/test_docx_file_to_document.py +++ b/test/components/converters/test_docx_file_to_document.py @@ -176,9 +176,9 @@ def test_run_with_table(self, test_files_path): table_index = next(i for i, part in enumerate(content_parts) if "| This | Is | Just a |" in part) # check that natural order of the document is preserved assert any("Donald Trump" in part for part in content_parts[:table_index]), "Text before table not found" - assert any("Now we are in Page 2" in part for part in content_parts[table_index + 1 :]), ( - "Text after table not found" - ) + assert any( + "Now we are in Page 2" in part for part in content_parts[table_index + 1 :] + ), "Text after table not found" def test_run_with_store_full_path_false(self, test_files_path): """ diff --git a/test/components/embedders/test_openai_document_embedder.py b/test/components/embedders/test_openai_document_embedder.py index 7d43bcfa83..87ed6afbb6 100644 --- a/test/components/embedders/test_openai_document_embedder.py +++ b/test/components/embedders/test_openai_document_embedder.py @@ -251,8 +251,8 @@ def test_run(self): assert len(doc.embedding) == 1536 assert all(isinstance(x, float) for x in doc.embedding) - assert "text" in result["meta"]["model"] and "ada" in result["meta"]["model"], ( - "The model name does not contain 'text' and 'ada'" - ) + assert ( + "text" in result["meta"]["model"] and "ada" in result["meta"]["model"] + ), "The model name does not contain 'text' and 'ada'" assert result["meta"]["usage"] == {"prompt_tokens": 15, "total_tokens": 15}, "Usage information does not match" diff --git a/test/components/embedders/test_openai_text_embedder.py b/test/components/embedders/test_openai_text_embedder.py index 695e6351f0..31a0360555 100644 --- a/test/components/embedders/test_openai_text_embedder.py +++ b/test/components/embedders/test_openai_text_embedder.py @@ -130,8 +130,8 @@ def test_run(self): assert len(result["embedding"]) == 1536 assert all(isinstance(x, float) for x in result["embedding"]) - assert "text" in result["meta"]["model"] and "ada" in result["meta"]["model"], ( - "The model name does not contain 'text' and 'ada'" - ) + assert ( + "text" in result["meta"]["model"] and "ada" in result["meta"]["model"] + ), "The model name does not contain 'text' and 'ada'" assert result["meta"]["usage"] == {"prompt_tokens": 6, "total_tokens": 6}, "Usage information does not match" diff --git a/test/components/joiners/test_document_joiner.py b/test/components/joiners/test_document_joiner.py index 8160fdc48a..6cc4f5f9e0 100644 --- a/test/components/joiners/test_document_joiner.py +++ b/test/components/joiners/test_document_joiner.py @@ -302,6 +302,6 @@ def test_test_score_norm_with_rrf(self): for i in range(len(join_results["documents"]) - 1) ) - assert is_sorted, ( - "Documents are not sorted in descending order by score, there is an issue with rff ranking" - ) + assert ( + is_sorted + ), "Documents are not sorted in descending order by score, there is an issue with rff ranking" diff --git a/test/components/routers/test_conditional_router.py b/test/components/routers/test_conditional_router.py index 478e62d5bf..66d941b645 100644 --- a/test/components/routers/test_conditional_router.py +++ b/test/components/routers/test_conditional_router.py @@ -436,9 +436,9 @@ def test_router_with_optional_parameters(self): # Test pipeline without path parameter result = pipe.run(data={"router": {"question": "What?"}}) - assert result["router"] == {"fallback": "What?"}, ( - "Default route should work in pipeline when 'path' is not provided" - ) + assert result["router"] == { + "fallback": "What?" + }, "Default route should work in pipeline when 'path' is not provided" # Test pipeline with path parameter result = pipe.run(data={"router": {"question": "What?", "path": "followup_short"}}) diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 7d5b9965ca..fa46ffa2f0 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -10,7 +10,7 @@ from haystack.dataclasses import ChatMessage, GeneratedAnswer from haystack.components.routers import ConditionalRouter from haystack.components.builders import PromptBuilder, AnswerBuilder, ChatPromptBuilder -from haystack.components.converters.output_adapter import OutputAdapter +from haystack.components.converters.output_adapter import OutputAdapter from haystack.components.preprocessors import DocumentCleaner from haystack.components.retrievers.in_memory import InMemoryBM25Retriever from haystack.document_stores.in_memory import InMemoryDocumentStore @@ -1809,6 +1809,7 @@ def run(self, create_document: bool = False): ], ) + @given( "a pipeline that has a variadic component that receives partial inputs in a different order", target_fixture="pipeline_data", @@ -2284,6 +2285,7 @@ def that_has_a_string_variadic_component(): ], ) + @given("a pipeline that is an agent that can use RAG", target_fixture="pipeline_data") def an_agent_that_can_use_RAG(): @component @@ -2822,6 +2824,7 @@ def run(self, replies: List[str]): ], ) + @given("a pipeline that passes outputs that are consumed in cycle to outside the cycle", target_fixture="pipeline_data") def passes_outputs_outside_cycle(): @component @@ -2851,7 +2854,6 @@ def run(self, replies: List[str], query: str, prompt: Optional[str] = None) -> D if prompt is not None: answer.meta["prompt"] = prompt - return {"answers": [answer]} code_prompt_template = "{{task}}" @@ -2894,7 +2896,9 @@ def generate_santa_sleigh(): router = ConditionalRouter(routes=routes) joiner = BranchJoiner(type_=str) - concatenator = OutputAdapter(template="{{code_prompt + '\n' + generated_code[0] + '\n' + feedback}}", output_type=str) + concatenator = OutputAdapter( + template="{{code_prompt + '\n' + generated_code[0] + '\n' + feedback}}", output_type=str + ) answer_builder = AnswerBuilderWithPrompt() @@ -2936,7 +2940,13 @@ def generate_santa_sleigh(): PipelineRunData( inputs={"joiner": {"value": task}, "answer_builder": {"query": task}}, expected_outputs={ - "answer_builder": {"answers": [GeneratedAnswer(data=valid_response, query=task, documents=[], meta={"prompt": expected_prompt})]} + "answer_builder": { + "answers": [ + GeneratedAnswer( + data=valid_response, query=task, documents=[], meta={"prompt": expected_prompt} + ) + ] + } }, expected_run_order=[ "joiner", @@ -2964,5 +2974,3 @@ def generate_santa_sleigh(): ) ], ) - - diff --git a/test/core/pipeline/test_component_checks.py b/test/core/pipeline/test_component_checks.py index 3a1d0e0e19..c6ce9f2ba9 100644 --- a/test/core/pipeline/test_component_checks.py +++ b/test/core/pipeline/test_component_checks.py @@ -14,13 +14,12 @@ def basic_component(): "visits": 0, "input_sockets": { "mandatory_input": InputSocket("mandatory_input", int), - "optional_input": InputSocket("optional_input", str, default_value="default") + "optional_input": InputSocket("optional_input", str, default_value="default"), }, - "output_sockets": { - "output": OutputSocket("output", int) - } + "output_sockets": {"output": OutputSocket("output", int)}, } + @pytest.fixture def variadic_component(): """Component with variadic input.""" @@ -29,13 +28,12 @@ def variadic_component(): "visits": 0, "input_sockets": { "variadic_input": InputSocket("variadic_input", Variadic[int]), - "normal_input": InputSocket("normal_input", str) + "normal_input": InputSocket("normal_input", str), }, - "output_sockets": { - "output": OutputSocket("output", int) - } + "output_sockets": {"output": OutputSocket("output", int)}, } + @pytest.fixture def greedy_variadic_component(): """Component with greedy variadic input.""" @@ -44,13 +42,12 @@ def greedy_variadic_component(): "visits": 0, "input_sockets": { "greedy_input": InputSocket("greedy_input", GreedyVariadic[int]), - "normal_input": InputSocket("normal_input", str) + "normal_input": InputSocket("normal_input", str), }, - "output_sockets": { - "output": OutputSocket("output", int) - } + "output_sockets": {"output": OutputSocket("output", int)}, } + @pytest.fixture def input_socket_with_sender(): """Regular input socket with a single sender.""" @@ -58,6 +55,7 @@ def input_socket_with_sender(): socket.senders = ["component1"] return socket + @pytest.fixture def variadic_socket_with_senders(): """Variadic input socket with multiple senders.""" @@ -65,6 +63,7 @@ def variadic_socket_with_senders(): socket.senders = ["component1", "component2"] return socket + @pytest.fixture def component_with_multiple_sockets(input_socket_with_sender, variadic_socket_with_senders): """Component with multiple input sockets including both regular and variadic.""" @@ -73,10 +72,11 @@ def component_with_multiple_sockets(input_socket_with_sender, variadic_socket_wi "input_sockets": { "socket1": input_socket_with_sender, "socket2": variadic_socket_with_senders, - "socket3": InputSocket("socket3", str) # No senders - } + "socket3": InputSocket("socket3", str), # No senders + }, } + @pytest.fixture def regular_socket(): """Regular input socket with one sender.""" @@ -84,6 +84,7 @@ def regular_socket(): socket.senders = ["component1"] return socket + @pytest.fixture def lazy_variadic_socket(): """Lazy variadic input socket with multiple senders.""" @@ -91,6 +92,7 @@ def lazy_variadic_socket(): socket.senders = ["component1", "component2"] return socket + @pytest.fixture def greedy_variadic_socket(): """Greedy variadic input socket with multiple senders.""" @@ -98,6 +100,7 @@ def greedy_variadic_socket(): socket.senders = ["component1", "component2", "component3"] return socket + @pytest.fixture def complex_component(regular_socket, lazy_variadic_socket, greedy_variadic_socket): """Component with all types of sockets.""" @@ -106,24 +109,20 @@ def complex_component(regular_socket, lazy_variadic_socket, greedy_variadic_sock "input_sockets": { "regular": regular_socket, "lazy_var": lazy_variadic_socket, - "greedy_var": greedy_variadic_socket - } + "greedy_var": greedy_variadic_socket, + }, } class TestCanComponentRun: def test_component_with_all_mandatory_inputs_and_trigger(self, basic_component): """Checks that the component runs if all mandatory inputs are received and triggered.""" - inputs = { - "mandatory_input": [{"sender": "previous_component", "value": 42}] - } + inputs = {"mandatory_input": [{"sender": "previous_component", "value": 42}]} assert can_component_run(basic_component, inputs) is True def test_component_missing_mandatory_input(self, basic_component): """Checks that the component won't run if mandatory inputs are missing.""" - inputs = { - "optional_input": [{"sender": "previous_component", "value": "test"}] - } + inputs = {"optional_input": [{"sender": "previous_component", "value": "test"}]} assert can_component_run(basic_component, inputs) is False def test_component_with_no_trigger_but_all_inputs(self, basic_component): @@ -132,17 +131,13 @@ def test_component_with_no_trigger_but_all_inputs(self, basic_component): but there is no trigger (no new input from predecessor, not first visit). """ basic_component["visits"] = 1 - inputs = { - "mandatory_input": [{"sender": None, "value": 42}] - } + inputs = {"mandatory_input": [{"sender": None, "value": 42}]} assert can_component_run(basic_component, inputs) is False def test_component_with_multiple_visits(self, basic_component): """Checks that a component can still be triggered on subsequent visits by a predecessor.""" basic_component["visits"] = 2 - inputs = { - "mandatory_input": [{"sender": "previous_component", "value": 42}] - } + inputs = {"mandatory_input": [{"sender": "previous_component", "value": 42}]} assert can_component_run(basic_component, inputs) is True def test_component_with_no_inputs_first_visit(self, basic_component): @@ -160,33 +155,25 @@ def test_component_triggered_on_second_visit_with_new_input(self, basic_componen basic_component["visits"] = 1 # Now a predecessor provides a new input; this should re-trigger execution. - inputs = { - "mandatory_input": [{"sender": "previous_component", "value": 99}] - } + inputs = {"mandatory_input": [{"sender": "previous_component", "value": 99}]} assert can_component_run(basic_component, inputs) is True class TestHasAnyTrigger: def test_trigger_from_predecessor(self, basic_component): """Ensures that new data from a predecessor can trigger a component.""" - inputs = { - "mandatory_input": [{"sender": "previous_component", "value": 42}] - } + inputs = {"mandatory_input": [{"sender": "previous_component", "value": 42}]} assert has_any_trigger(basic_component, inputs) is True def test_trigger_from_user_first_visit(self, basic_component): """Checks that user input (sender=None) triggers the component on the first visit.""" - inputs = { - "mandatory_input": [{"sender": None, "value": 42}] - } + inputs = {"mandatory_input": [{"sender": None, "value": 42}]} assert has_any_trigger(basic_component, inputs) is True def test_no_trigger_from_user_after_first_visit(self, basic_component): """Checks that user input no longer triggers the component after the first visit.""" basic_component["visits"] = 1 - inputs = { - "mandatory_input": [{"sender": None, "value": 42}] - } + inputs = {"mandatory_input": [{"sender": None, "value": 42}]} assert has_any_trigger(basic_component, inputs) is False def test_trigger_without_inputs_first_visit(self, basic_component): @@ -206,23 +193,19 @@ def test_no_trigger_without_inputs_after_first_visit(self, basic_component): class TestAllMandatorySocketsReady: def test_all_mandatory_sockets_filled(self, basic_component): """Checks that all mandatory sockets are ready when they have valid input.""" - inputs = { - "mandatory_input": [{"sender": "previous_component", "value": 42}] - } + inputs = {"mandatory_input": [{"sender": "previous_component", "value": 42}]} assert are_all_mandatory_sockets_ready(basic_component, inputs) is True def test_missing_mandatory_socket(self, basic_component): """Ensures that if a mandatory socket is missing, the component is not ready.""" - inputs = { - "optional_input": [{"sender": "previous_component", "value": "test"}] - } + inputs = {"optional_input": [{"sender": "previous_component", "value": "test"}]} assert are_all_mandatory_sockets_ready(basic_component, inputs) is False def test_variadic_socket_with_input(self, variadic_component): """Verifies that a variadic socket is considered filled if it has at least one input.""" inputs = { "variadic_input": [{"sender": "previous_component", "value": 42}], - "normal_input": [{"sender": "previous_component", "value": "test"}] + "normal_input": [{"sender": "previous_component", "value": "test"}], } assert are_all_mandatory_sockets_ready(variadic_component, inputs) is True @@ -230,15 +213,13 @@ def test_greedy_variadic_socket_with_partial_input(self, greedy_variadic_compone """Greedy variadic sockets are ready with at least one valid input.""" inputs = { "greedy_input": [{"sender": "previous_component", "value": 42}], - "normal_input": [{"sender": "previous_component", "value": "test"}] + "normal_input": [{"sender": "previous_component", "value": "test"}], } assert are_all_mandatory_sockets_ready(greedy_variadic_component, inputs) is True def test_variadic_socket_no_input(self, variadic_component): """A variadic socket is not filled if it has zero valid inputs.""" - inputs = { - "normal_input": [{"sender": "previous_component", "value": "test"}] - } + inputs = {"normal_input": [{"sender": "previous_component", "value": "test"}]} assert are_all_mandatory_sockets_ready(variadic_component, inputs) is False def test_empty_inputs(self, basic_component): @@ -250,7 +231,7 @@ def test_no_mandatory_sockets(self, basic_component): """Ensures that if there are no mandatory sockets, the component is considered ready.""" basic_component["input_sockets"] = { "optional_1": InputSocket("optional_1", str, default_value="default1"), - "optional_2": InputSocket("optional_2", str, default_value="default2") + "optional_2": InputSocket("optional_2", str, default_value="default2"), } inputs = {} assert are_all_mandatory_sockets_ready(basic_component, inputs) is True @@ -260,19 +241,16 @@ def test_multiple_mandatory_sockets(self, basic_component): basic_component["input_sockets"] = { "mandatory_1": InputSocket("mandatory_1", int), "mandatory_2": InputSocket("mandatory_2", str), - "optional": InputSocket("optional", bool, default_value=False) + "optional": InputSocket("optional", bool, default_value=False), } inputs = { "mandatory_1": [{"sender": "comp1", "value": 42}], - "mandatory_2": [{"sender": "comp2", "value": "test"}] + "mandatory_2": [{"sender": "comp2", "value": "test"}], } assert are_all_mandatory_sockets_ready(basic_component, inputs) is True # Missing one mandatory input - inputs = { - "mandatory_1": [{"sender": "comp1", "value": 42}], - "optional": [{"sender": "comp3", "value": True}] - } + inputs = {"mandatory_1": [{"sender": "comp1", "value": 42}], "optional": [{"sender": "comp3", "value": True}]} assert are_all_mandatory_sockets_ready(basic_component, inputs) is False @@ -281,20 +259,14 @@ def test_any_predecessors_provided_input_with_predecessor(self, component_with_m """ Tests detection of predecessor input when a valid predecessor sends data. """ - inputs = { - "socket1": [{"sender": "component1", "value": 42}], - "socket2": [{"sender": None, "value": "test"}] - } + inputs = {"socket1": [{"sender": "component1", "value": 42}], "socket2": [{"sender": None, "value": "test"}]} assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is True def test_any_predecessors_provided_input_no_predecessor(self, component_with_multiple_sockets): """ Checks that no predecessor inputs are detected if all senders are None (user inputs). """ - inputs = { - "socket1": [{"sender": None, "value": 42}], - "socket2": [{"sender": None, "value": "test"}] - } + inputs = {"socket1": [{"sender": None, "value": 42}], "socket2": [{"sender": None, "value": "test"}]} assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False def test_any_predecessors_provided_input_with_no_output(self, component_with_multiple_sockets): @@ -303,7 +275,7 @@ def test_any_predecessors_provided_input_with_no_output(self, component_with_mul """ inputs = { "socket1": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}], - "socket2": [{"sender": None, "value": "test"}] + "socket2": [{"sender": None, "value": "test"}], } assert any_predecessors_provided_input(component_with_multiple_sockets, inputs) is False @@ -322,29 +294,17 @@ class TestSocketValueFromPredecessor: @pytest.mark.parametrize( "socket_inputs, expected_result", [ - pytest.param( - [{"sender": "component1", "value": 42}], - True, - id="valid_input" - ), - pytest.param( - [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}], - False, - id="no_output" - ), - pytest.param( - [{"sender": None, "value": 42}], - False, - id="user_input" - ), + pytest.param([{"sender": "component1", "value": 42}], True, id="valid_input"), + pytest.param([{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}], False, id="no_output"), + pytest.param([{"sender": None, "value": 42}], False, id="user_input"), pytest.param( [ {"sender": None, "value": 42}, {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, - {"sender": "component2", "value": 100} + {"sender": "component2", "value": 100}, ], True, - id="mixed_inputs" + id="mixed_inputs", ), pytest.param([], False, id="empty_list"), ], @@ -360,17 +320,14 @@ def test_any_socket_value_from_predecessor_received(self, socket_inputs, expecte class TestUserInputDetection: def test_has_user_input_with_user_input(self): """Checks that having a sender=None input means user input is present.""" - inputs = { - "socket1": [{"sender": None, "value": 42}], - "socket2": [{"sender": "component1", "value": "test"}] - } + inputs = {"socket1": [{"sender": None, "value": 42}], "socket2": [{"sender": "component1", "value": "test"}]} assert has_user_input(inputs) is True def test_has_user_input_without_user_input(self): """Ensures that if all senders are component-based, there's no user input.""" inputs = { "socket1": [{"sender": "component1", "value": 42}], - "socket2": [{"sender": "component2", "value": "test"}] + "socket2": [{"sender": "component2", "value": "test"}], } assert has_user_input(inputs) is False @@ -384,21 +341,14 @@ def test_has_user_input_with_no_output(self): Even if the input value is _NO_OUTPUT_PRODUCED, if sender=None it still counts as user input being provided. """ - inputs = { - "socket1": [{"sender": None, "value": _NO_OUTPUT_PRODUCED}] - } + inputs = {"socket1": [{"sender": None, "value": _NO_OUTPUT_PRODUCED}]} assert has_user_input(inputs) is True class TestPipelineInputCapability: def test_cannot_receive_inputs_no_senders(self): """Checks that a component with zero senders for each socket cannot receive pipeline inputs.""" - component = { - "input_sockets": { - "socket1": InputSocket("socket1", int), - "socket2": InputSocket("socket2", str) - } - } + component = {"input_sockets": {"socket1": InputSocket("socket1", int), "socket2": InputSocket("socket2", str)}} assert can_not_receive_inputs_from_pipeline(component) is True def test_cannot_receive_inputs_with_senders(self, component_with_multiple_sockets): @@ -410,7 +360,7 @@ def test_cannot_receive_inputs_mixed_senders(self, input_socket_with_sender): component = { "input_sockets": { "socket1": input_socket_with_sender, - "socket2": InputSocket("socket2", str) # No senders + "socket2": InputSocket("socket2", str), # No senders } } assert can_not_receive_inputs_from_pipeline(component) is False @@ -434,10 +384,7 @@ def test_regular_socket_with_wrong_predecessor(self, input_socket_with_sender): def test_variadic_socket_all_predecessors_executed(self, variadic_socket_with_senders): """Variadic socket is executed only if all senders have produced at least one valid result.""" - socket_inputs = [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] + socket_inputs = [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}] assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is True def test_variadic_socket_partial_execution(self, variadic_socket_with_senders): @@ -453,7 +400,7 @@ def test_variadic_socket_with_user_input(self, variadic_socket_with_senders): socket_inputs = [ {"sender": "component1", "value": 42}, {"sender": None, "value": 43}, - {"sender": "component2", "value": 44} + {"sender": "component2", "value": 44}, ] assert all_socket_predecessors_executed(variadic_socket_with_senders, socket_inputs) is True @@ -476,10 +423,7 @@ def test_any_socket_input_received_with_no_output(self): def test_any_socket_input_received_mixed_inputs(self): """A single valid input among many is enough to consider the socket as having input.""" - socket_inputs = [ - {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, - {"sender": "component2", "value": 42} - ] + socket_inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, {"sender": "component2", "value": 42}] assert any_socket_input_received(socket_inputs) is True def test_any_socket_input_received_empty_list(self): @@ -490,10 +434,7 @@ def test_any_socket_input_received_empty_list(self): class TestLazyVariadicSocket: def test_lazy_variadic_all_inputs_received(self, variadic_socket_with_senders): """Lazy variadic socket is ready only if all named senders provided outputs.""" - socket_inputs = [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] + socket_inputs = [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}] assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is True def test_lazy_variadic_partial_inputs(self, variadic_socket_with_senders): @@ -503,10 +444,7 @@ def test_lazy_variadic_partial_inputs(self, variadic_socket_with_senders): def test_lazy_variadic_with_no_output(self, variadic_socket_with_senders): """_NO_OUTPUT_PRODUCED from a sender doesn't count as valid input, so it's not fully received.""" - socket_inputs = [ - {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, - {"sender": "component2", "value": 42} - ] + socket_inputs = [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, {"sender": "component2", "value": 42}] assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is False def test_lazy_variadic_with_user_input(self, variadic_socket_with_senders): @@ -517,7 +455,7 @@ def test_lazy_variadic_with_user_input(self, variadic_socket_with_senders): socket_inputs = [ {"sender": "component1", "value": 42}, {"sender": None, "value": 43}, - {"sender": "component2", "value": 44} + {"sender": "component2", "value": 44}, ] assert has_lazy_variadic_socket_received_all_inputs(variadic_socket_with_senders, socket_inputs) is True @@ -558,10 +496,7 @@ def test_regular_socket_no_inputs(self, regular_socket): def test_lazy_variadic_socket_all_inputs(self, lazy_variadic_socket): """Lazy variadic socket is complete only if all senders have produced valid outputs.""" - inputs = [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] + inputs = [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}] assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is True def test_lazy_variadic_socket_partial_inputs(self, lazy_variadic_socket): @@ -571,10 +506,7 @@ def test_lazy_variadic_socket_partial_inputs(self, lazy_variadic_socket): def test_lazy_variadic_socket_with_no_output(self, lazy_variadic_socket): """A sender that produces _NO_OUTPUT_PRODUCED does not fulfill the lazy socket requirement.""" - inputs = [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": _NO_OUTPUT_PRODUCED} - ] + inputs = [{"sender": "component1", "value": 42}, {"sender": "component2", "value": _NO_OUTPUT_PRODUCED}] assert has_socket_received_all_inputs(lazy_variadic_socket, inputs) is False def test_greedy_variadic_socket_one_input(self, greedy_variadic_socket): @@ -584,10 +516,7 @@ def test_greedy_variadic_socket_one_input(self, greedy_variadic_socket): def test_greedy_variadic_socket_multiple_inputs(self, greedy_variadic_socket): """A greedy variadic socket with multiple inputs remains complete as soon as one is valid.""" - inputs = [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] + inputs = [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}] assert has_socket_received_all_inputs(greedy_variadic_socket, inputs) is True def test_greedy_variadic_socket_no_valid_inputs(self, greedy_variadic_socket): @@ -604,15 +533,12 @@ def test_all_predecessors_executed_complete(self, complex_component): """ inputs = { "regular": [{"sender": "component1", "value": 42}], - "lazy_var": [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ], + "lazy_var": [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}], "greedy_var": [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}, - {"sender": "component3", "value": 44} - ] + {"sender": "component3", "value": 44}, + ], } assert all_predecessors_executed(complex_component, inputs) is True @@ -621,10 +547,7 @@ def test_all_predecessors_executed_partial(self, complex_component): inputs = { "regular": [{"sender": "component1", "value": 42}], "lazy_var": [{"sender": "component1", "value": 42}], # Missing component2 - "greedy_var": [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] + "greedy_var": [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}], } assert all_predecessors_executed(complex_component, inputs) is False @@ -635,15 +558,12 @@ def test_all_predecessors_executed_with_user_input(self, complex_component): """ inputs = { "regular": [{"sender": "component1", "value": 42}], - "lazy_var": [ - {"sender": "component1", "value": 42}, - {"sender": None, "value": 43} - ], + "lazy_var": [{"sender": "component1", "value": 42}, {"sender": None, "value": 43}], "greedy_var": [ {"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}, - {"sender": "component3", "value": 44} - ] + {"sender": "component3", "value": 44}, + ], } assert all_predecessors_executed(complex_component, inputs) is False @@ -651,12 +571,7 @@ def test_all_predecessors_executed_with_user_input(self, complex_component): class TestLazyVariadicResolution: def test_lazy_variadic_sockets_all_resolved(self, complex_component): """Checks that lazy variadic sockets are resolved when all inputs have arrived.""" - inputs = { - "lazy_var": [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] - } + inputs = {"lazy_var": [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}]} assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is True def test_lazy_variadic_sockets_partially_resolved(self, complex_component): @@ -679,7 +594,7 @@ def test_lazy_variadic_sockets_with_predecessors_executed(self, complex_componen inputs = { "lazy_var": [ {"sender": "component1", "value": _NO_OUTPUT_PRODUCED}, - {"sender": "component2", "value": _NO_OUTPUT_PRODUCED} + {"sender": "component2", "value": _NO_OUTPUT_PRODUCED}, ] } assert are_all_lazy_variadic_sockets_resolved(complex_component, inputs) is True @@ -688,26 +603,17 @@ def test_lazy_variadic_sockets_with_predecessors_executed(self, complex_componen class TestGreedySocketReadiness: def test_greedy_socket_ready(self, complex_component): """A single valid input is enough for a greedy variadic socket to be considered ready.""" - inputs = { - "greedy_var": [{"sender": "component1", "value": 42}] - } + inputs = {"greedy_var": [{"sender": "component1", "value": 42}]} assert is_any_greedy_socket_ready(complex_component, inputs) is True def test_greedy_socket_multiple_inputs_ready(self, complex_component): """Multiple valid inputs on a greedy socket is also fine—it's still ready.""" - inputs = { - "greedy_var": [ - {"sender": "component1", "value": 42}, - {"sender": "component2", "value": 43} - ] - } + inputs = {"greedy_var": [{"sender": "component1", "value": 42}, {"sender": "component2", "value": 43}]} assert is_any_greedy_socket_ready(complex_component, inputs) is True def test_greedy_socket_not_ready(self, complex_component): """If the only input is _NO_OUTPUT_PRODUCED, the greedy socket isn't ready.""" - inputs = { - "greedy_var": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}] - } + inputs = {"greedy_var": [{"sender": "component1", "value": _NO_OUTPUT_PRODUCED}]} assert is_any_greedy_socket_ready(complex_component, inputs) is False def test_greedy_socket_no_inputs(self, complex_component): @@ -717,7 +623,5 @@ def test_greedy_socket_no_inputs(self, complex_component): def test_greedy_socket_with_user_input(self, complex_component): """User input can also trigger readiness for a greedy variadic socket.""" - inputs = { - "greedy_var": [{"sender": None, "value": 42}] - } + inputs = {"greedy_var": [{"sender": None, "value": 42}]} assert is_any_greedy_socket_ready(complex_component, inputs) is True From 54d4f2c801c380610980394082fb0c59290ef6b2 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 14:30:07 +0100 Subject: [PATCH 12/88] remove tests that aren't needed anymore --- test/core/pipeline/test_pipeline.py | 476 +--------------------------- 1 file changed, 1 insertion(+), 475 deletions(-) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 1cd57a5b5b..c967cf59e6 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -14,14 +14,7 @@ from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic, _empty from haystack.core.errors import DeserializationError, PipelineConnectError, PipelineDrawingError, PipelineError from haystack.core.pipeline import Pipeline, PredefinedPipeline -from haystack.core.pipeline.base import ( - _add_missing_input_defaults, - _enqueue_component, - _dequeue_component, - _enqueue_waiting_component, - _dequeue_waiting_component, - _is_lazy_variadic, -) + from haystack.core.serialization import DeserializationCallbacks from haystack.testing.factory import component_class from haystack.testing.sample_components import AddFixedValue, Double, Greet @@ -788,27 +781,6 @@ def test__init_graph(self): for node in pipe.graph.nodes: assert pipe.graph.nodes[node]["visits"] == 0 - def test__normalize_varidiac_input_data(self): - pipe = Pipeline() - template = """ - Answer the following questions: - {{ questions | join("\n") }} - """ - pipe.add_component("prompt_builder", PromptBuilder(template=template)) - pipe.add_component("branch_joiner", BranchJoiner(type_=int)) - questions = ["What is the capital of Italy?", "What is the capital of France?"] - data = { - "prompt_builder": {"questions": questions}, - "branch_joiner": {"value": 1}, - "not_a_component": "some input data", - } - res = pipe._normalize_varidiac_input_data(data) - assert res == { - "prompt_builder": {"questions": ["What is the capital of Italy?", "What is the capital of France?"]}, - "branch_joiner": {"value": [1]}, - "not_a_component": "some input data", - } - def test__prepare_component_input_data(self): MockComponent = component_class("MockComponent", input_types={"x": List[str], "y": str}) pipe = Pipeline() @@ -1074,452 +1046,6 @@ def test__run_component_with_variadic_input(self): pipe._run_component("document_joiner", inputs) assert inputs == {"docs": []} - def test__component_has_enough_inputs_to_run(self): - sentence_builder = component_class("SentenceBuilder", input_types={"words": List[str]})() - pipe = Pipeline() - pipe.add_component("sentence_builder", sentence_builder) - - assert not pipe._component_has_enough_inputs_to_run("sentence_builder", {}) - assert not pipe._component_has_enough_inputs_to_run( - "sentence_builder", {"sentence_builder": {"wrong_input_name": "blah blah"}} - ) - assert pipe._component_has_enough_inputs_to_run( - "sentence_builder", {"sentence_builder": {"words": ["blah blah"]}} - ) - - def test__find_components_that_will_receive_no_input(self): - sentence_builder = component_class( - "SentenceBuilder", input_types={"words": List[str]}, output_types={"text": str} - )() - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - conditional_document_builder = component_class( - "ConditionalDocumentBuilder", output_types={"doc": Document, "noop": None} - )() - - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - pipe = Pipeline() - pipe.add_component("sentence_builder", sentence_builder) - pipe.add_component("document_builder", document_builder) - pipe.add_component("document_joiner", document_joiner) - pipe.add_component("conditional_document_builder", conditional_document_builder) - pipe.connect("sentence_builder.text", "document_builder.text") - pipe.connect("document_builder.doc", "document_joiner.docs") - pipe.connect("conditional_document_builder.doc", "document_joiner.docs") - - res = pipe._find_components_that_will_receive_no_input("sentence_builder", {}, {}) - assert res == {("document_builder", document_builder), ("document_joiner", document_joiner)} - - res = pipe._find_components_that_will_receive_no_input("sentence_builder", {"text": "some text"}, {}) - assert res == set() - - res = pipe._find_components_that_will_receive_no_input("conditional_document_builder", {"noop": None}, {}) - assert res == {("document_joiner", document_joiner)} - - res = pipe._find_components_that_will_receive_no_input( - "conditional_document_builder", {"noop": None}, {"document_joiner": {"docs": []}} - ) - assert res == {("document_joiner", document_joiner)} - - res = pipe._find_components_that_will_receive_no_input( - "conditional_document_builder", {"noop": None}, {"document_joiner": {"docs": [Document("some text")]}} - ) - assert res == set() - - multiple_outputs = component_class("MultipleOutputs", output_types={"first": int, "second": int})() - - def custom_init(self): - component.set_input_type(self, "first", Optional[int], 1) - component.set_input_type(self, "second", Optional[int], 2) - - multiple_optional_inputs = component_class("MultipleOptionalInputs", extra_fields={"__init__": custom_init})() - - pipe = Pipeline() - pipe.add_component("multiple_outputs", multiple_outputs) - pipe.add_component("multiple_optional_inputs", multiple_optional_inputs) - pipe.connect("multiple_outputs.second", "multiple_optional_inputs.first") - - res = pipe._find_components_that_will_receive_no_input("multiple_outputs", {"first": 1}, {}) - assert res == {("multiple_optional_inputs", multiple_optional_inputs)} - - res = pipe._find_components_that_will_receive_no_input( - "multiple_outputs", {"first": 1}, {"multiple_optional_inputs": {"second": 200}} - ) - assert res == set() - - res = pipe._find_components_that_will_receive_no_input("multiple_outputs", {"second": 1}, {}) - assert res == set() - - def test__distribute_output(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document, "another_doc": Document} - )() - document_cleaner = component_class( - "DocumentCleaner", input_types={"doc": Document}, output_types={"cleaned_doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - pipe = Pipeline() - pipe.add_component("document_builder", document_builder) - pipe.add_component("document_cleaner", document_cleaner) - pipe.add_component("document_joiner", document_joiner) - pipe.connect("document_builder.doc", "document_cleaner.doc") - pipe.connect("document_builder.another_doc", "document_joiner.docs") - - inputs = {"document_builder": {"text": "some text"}} - run_queue = [] - waiting_queue = [("document_joiner", document_joiner)] - receivers = [ - ( - "document_cleaner", - OutputSocket("doc", Document, ["document_cleaner"]), - InputSocket("doc", Document, _empty, ["document_builder"]), - ), - ( - "document_joiner", - OutputSocket("another_doc", Document, ["document_joiner"]), - InputSocket("docs", Variadic[Document], _empty, ["document_builder"]), - ), - ] - res = pipe._distribute_output( - receivers, {"doc": Document("some text"), "another_doc": Document()}, inputs, run_queue, waiting_queue - ) - - assert res == {} - assert inputs == { - "document_builder": {"text": "some text"}, - "document_cleaner": {"doc": Document("some text")}, - "document_joiner": {"docs": [Document()]}, - } - assert run_queue == [("document_cleaner", document_cleaner)] - assert waiting_queue == [("document_joiner", document_joiner)] - - def test__find_next_runnable_component(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - pipe = Pipeline() - components_inputs = {"document_builder": {"text": "some text"}} - waiting_queue = [("document_builder", document_builder)] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - assert pair == ("document_builder", document_builder) - - def test__find_next_runnable_component_without_component_inputs(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - pipe = Pipeline() - components_inputs = {} - waiting_queue = [("document_builder", document_builder)] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - assert pair == ("document_builder", document_builder) - - def test__find_next_runnable_component_with_component_with_only_variadic_non_greedy_input(self): - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - pipe = Pipeline() - components_inputs = {} - waiting_queue = [("document_joiner", document_joiner)] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - assert pair == ("document_joiner", document_joiner) - - def test__find_next_runnable_component_with_component_with_only_default_input(self): - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - - pipe = Pipeline() - components_inputs = {} - waiting_queue = [("prompt_builder", prompt_builder)] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - - assert pair == ("prompt_builder", prompt_builder) - - def test__find_next_runnable_component_with_component_with_variadic_non_greedy_and_default_input(self): - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - - pipe = Pipeline() - components_inputs = {} - waiting_queue = [("prompt_builder", prompt_builder), ("document_joiner", document_joiner)] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - - assert pair == ("document_joiner", document_joiner) - - def test__find_next_runnable_component_with_different_components_inputs(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - - pipe = Pipeline() - components_inputs = {"document_builder": {"text": "some text"}} - waiting_queue = [ - ("prompt_builder", prompt_builder), - ("document_builder", document_builder), - ("document_joiner", document_joiner), - ] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - - assert pair == ("document_builder", document_builder) - - def test__find_next_runnable_component_with_different_components_without_any_input(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - - pipe = Pipeline() - components_inputs = {} - waiting_queue = [ - ("prompt_builder", prompt_builder), - ("document_builder", document_builder), - ("document_joiner", document_joiner), - ] - pair = pipe._find_next_runnable_component(components_inputs, waiting_queue) - - assert pair == ("document_builder", document_builder) - - def test__is_stuck_in_a_loop(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - - pipe = Pipeline() - - waiting_queue = [("document_builder", document_builder)] - assert pipe._is_stuck_in_a_loop(waiting_queue) - - waiting_queue = [("document_joiner", document_joiner)] - assert pipe._is_stuck_in_a_loop(waiting_queue) - - waiting_queue = [("prompt_builder", prompt_builder)] - assert pipe._is_stuck_in_a_loop(waiting_queue) - - waiting_queue = [("document_joiner", document_joiner), ("prompt_builder", prompt_builder)] - assert not pipe._is_stuck_in_a_loop(waiting_queue) - - waiting_queue = [("document_builder", document_joiner), ("prompt_builder", prompt_builder)] - assert not pipe._is_stuck_in_a_loop(waiting_queue) - - waiting_queue = [("document_builder", document_joiner), ("document_joiner", document_joiner)] - assert not pipe._is_stuck_in_a_loop(waiting_queue) - - def test__enqueue_component(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - run_queue = [] - waiting_queue = [] - _enqueue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [("document_builder", document_builder)] - assert waiting_queue == [] - - run_queue = [("document_builder", document_builder)] - waiting_queue = [] - _enqueue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [("document_builder", document_builder)] - assert waiting_queue == [] - - run_queue = [] - waiting_queue = [("document_builder", document_builder)] - _enqueue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [("document_builder", document_builder)] - assert waiting_queue == [] - - run_queue = [] - waiting_queue = [("document_joiner", document_joiner)] - _enqueue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [("document_builder", document_builder)] - assert waiting_queue == [("document_joiner", document_joiner)] - - run_queue = [("document_joiner", document_joiner)] - waiting_queue = [] - _enqueue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [("document_joiner", document_joiner), ("document_builder", document_builder)] - assert waiting_queue == [] - - def test__dequeue_component(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - run_queue = [] - waiting_queue = [] - _dequeue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [] - assert waiting_queue == [] - - run_queue = [("document_builder", document_builder)] - waiting_queue = [] - _dequeue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [] - assert waiting_queue == [] - - run_queue = [] - waiting_queue = [("document_builder", document_builder)] - _dequeue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [] - assert waiting_queue == [] - - run_queue = [("document_builder", document_builder)] - waiting_queue = [("document_builder", document_builder)] - _dequeue_component(("document_builder", document_builder), run_queue, waiting_queue) - assert run_queue == [] - assert waiting_queue == [] - - run_queue = [("document_builder", document_builder)] - waiting_queue = [("document_builder", document_builder)] - _dequeue_component(("document_joiner", document_joiner), run_queue, waiting_queue) - assert run_queue == [("document_builder", document_builder)] - assert waiting_queue == [("document_builder", document_builder)] - - def test__add_missing_input_defaults(self): - name = "prompt_builder" - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - components_inputs = {} - _add_missing_input_defaults(name, prompt_builder, components_inputs) - assert components_inputs == {"prompt_builder": {"questions": "", "template": None, "template_variables": None}} - - name = "answer_builder" - answer_builder = AnswerBuilder() - components_inputs = {"answer_builder": {"query": "What is the answer?"}} - _add_missing_input_defaults(name, answer_builder, components_inputs) - assert components_inputs == { - "answer_builder": { - "query": "What is the answer?", - "meta": None, - "documents": None, - "pattern": None, - "reference_pattern": None, - } - } - - name = "branch_joiner" - branch_joiner = BranchJoiner(int) - components_inputs = {} - _add_missing_input_defaults(name, branch_joiner, components_inputs) - assert components_inputs == {"branch_joiner": {}} - - def test__find_next_runnable_lazy_variadic_or_default_component(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - prompt_builder = PromptBuilder(template="{{ questions | join('\n') }}") - pipe = Pipeline() - - waiting_queue = [("document_builder", document_builder)] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("document_builder", document_builder) - - waiting_queue = [("document_joiner", document_joiner)] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("document_joiner", document_joiner) - - waiting_queue = [("prompt_builder", prompt_builder)] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("prompt_builder", prompt_builder) - - waiting_queue = [ - ("document_builder", document_builder), - ("document_joiner", document_joiner), - ("prompt_builder", prompt_builder), - ] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("document_joiner", document_joiner) - - waiting_queue = [ - ("prompt_builder", prompt_builder), - ("document_builder", document_builder), - ("document_joiner", document_joiner), - ] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("prompt_builder", prompt_builder) - - waiting_queue = [ - ("document_builder", document_builder), - ("document_joiner", document_joiner), - ("prompt_builder", prompt_builder), - ] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("document_joiner", document_joiner) - - waiting_queue = [ - ("document_builder", document_builder), - ("prompt_builder", prompt_builder), - ("document_joiner", document_joiner), - ] - pair = pipe._find_next_runnable_lazy_variadic_or_default_component(waiting_queue) - assert pair == ("prompt_builder", prompt_builder) - - def test__enqueue_waiting_component(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - waiting_queue = [] - _enqueue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [("document_builder", document_builder)] - - waiting_queue = [("document_builder", document_builder)] - _enqueue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [("document_builder", document_builder)] - - waiting_queue = [("document_joiner", document_joiner)] - _enqueue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [("document_joiner", document_joiner), ("document_builder", document_builder)] - - waiting_queue = [("document_builder", document_builder), ("document_joiner", document_joiner)] - _enqueue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [("document_builder", document_builder), ("document_joiner", document_joiner)] - - def test__dequeue_waiting_component(self): - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - waiting_queue = [] - _dequeue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [] - - waiting_queue = [("document_builder", document_builder)] - _dequeue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [] - - waiting_queue = [("document_joiner", document_joiner)] - _dequeue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [("document_joiner", document_joiner)] - - waiting_queue = [("document_builder", document_builder), ("document_joiner", document_joiner)] - _dequeue_waiting_component(("document_builder", document_builder), waiting_queue) - assert waiting_queue == [("document_joiner", document_joiner)] - - def test__is_lazy_variadic(self): - VariadicAndGreedyVariadic = component_class( - "VariadicAndGreedyVariadic", input_types={"variadic": Variadic[int], "greedy_variadic": GreedyVariadic[int]} - ) - NonVariadic = component_class("NonVariadic", input_types={"value": int}) - VariadicNonGreedyVariadic = component_class( - "VariadicNonGreedyVariadic", input_types={"variadic": Variadic[int]} - ) - NonVariadicAndGreedyVariadic = component_class( - "NonVariadicAndGreedyVariadic", input_types={"greedy_variadic": GreedyVariadic[int]} - ) - assert not _is_lazy_variadic(VariadicAndGreedyVariadic()) - assert not _is_lazy_variadic(NonVariadic()) - assert _is_lazy_variadic(VariadicNonGreedyVariadic()) - assert not _is_lazy_variadic(NonVariadicAndGreedyVariadic()) - def test__find_receivers_from(self): sentence_builder = component_class( "SentenceBuilder", input_types={"words": List[str]}, output_types={"text": str} From 05ed852d10bfbe196f548228b90322d3f8b92ff9 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 14:53:20 +0100 Subject: [PATCH 13/88] add _calculate_priority tests --- test/core/pipeline/test_pipeline.py | 148 ++++++++++++++++++++++++++++ 1 file changed, 148 insertions(+) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index c967cf59e6..7e393cb198 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -14,6 +14,7 @@ from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic, _empty from haystack.core.errors import DeserializationError, PipelineConnectError, PipelineDrawingError, PipelineError from haystack.core.pipeline import Pipeline, PredefinedPipeline +from haystack.core.pipeline.pipeline import ComponentPriority, _NO_OUTPUT_PRODUCED from haystack.core.serialization import DeserializationCallbacks from haystack.testing.factory import component_class @@ -1107,3 +1108,150 @@ def test__find_receivers_from(self): ), ) ] + + @pytest.mark.parametrize( + "component, inputs, expected_priority, test_description", + [ + # Test case 1: BLOCKED - Missing mandatory input + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int), + "optional_input": InputSocket( + "optional_input", str, default_value="default", senders=["previous_component"] + ), + }, + }, + {"optional_input": [{"sender": "previous_component", "value": "test"}]}, + ComponentPriority.BLOCKED, + "Component should be BLOCKED when mandatory input is missing", + ), + # Test case 2: BLOCKED - No trigger after first visit + ( + { + "instance": "mock_instance", + "visits": 1, # Already visited + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int), + "optional_input": InputSocket("optional_input", str, default_value="default"), + }, + }, + {"mandatory_input": [{"sender": None, "value": 42}]}, + ComponentPriority.BLOCKED, + "Component should be BLOCKED when there's no new trigger after first visit", + ), + # Test case 3: HIGHEST - Greedy socket ready + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "greedy_input": InputSocket("greedy_input", GreedyVariadic[int], senders=["component1"]), + "normal_input": InputSocket("normal_input", str, senders=["component2"]), + }, + }, + { + "greedy_input": [{"sender": "component1", "value": 42}], + "normal_input": [{"sender": "component2", "value": "test"}], + }, + ComponentPriority.HIGHEST, + "Component should have HIGHEST priority when greedy socket has valid input", + ), + # Test case 4: READY - All predecessors executed + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int, senders=["previous_component"]), + "optional_input": InputSocket( + "optional_input", str, senders=["another_component"], default_value="default" + ), + }, + }, + { + "mandatory_input": [{"sender": "previous_component", "value": 42}], + "optional_input": [{"sender": "another_component", "value": "test"}], + }, + ComponentPriority.READY, + "Component should be READY when all predecessors have executed", + ), + # Test case 5: DEFER - Lazy variadic sockets resolved and optional missing. + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "variadic_input": InputSocket( + "variadic_input", Variadic[int], senders=["component1", "component2"] + ), + "normal_input": InputSocket("normal_input", str, senders=["component3"]), + "optional_input": InputSocket( + "optional_input", str, default_value="default", senders=["component4"] + ), + }, + }, + { + "variadic_input": [ + {"sender": "component1", "value": "test"}, + {"sender": "component2", "value": _NO_OUTPUT_PRODUCED}, + ], + "normal_input": [{"sender": "component3", "value": "test"}], + }, + ComponentPriority.DEFER, + "Component should DEFER when all lazy variadic sockets are resolved", + ), + # Test case 6: DEFER_LAST - Incomplete variadic inputs + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "variadic_input": InputSocket( + "variadic_input", Variadic[int], senders=["component1", "component2"] + ), + "normal_input": InputSocket("normal_input", str), + }, + }, + { + "variadic_input": [{"sender": "component1", "value": 42}], # Missing component2 + "normal_input": [{"sender": "component3", "value": "test"}], + }, + ComponentPriority.DEFER_LAST, + "Component should be DEFER_LAST when not all variadic senders have produced output", + ), + # Test case 7: READY - No input sockets, first visit + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": {"optional_input": InputSocket("optional_input", str, default_value="default")}, + }, + {}, # no inputs + ComponentPriority.READY, + "Component should be READY on first visit when it has no input sockets", + ), + # Test case 8: BLOCKED - No connected input sockets, subsequent visit + ( + { + "instance": "mock_instance", + "visits": 1, + "input_sockets": {"optional_input": InputSocket("optional_input", str, default_value="default")}, + }, + {}, # no inputs + ComponentPriority.BLOCKED, + "Component should be BLOCKED on subsequent visits when it has no input sockets", + ), + ], + ids=lambda p: p.name if isinstance(p, ComponentPriority) else str(p), + ) + def test__calculate_priority(self, component, inputs, expected_priority, test_description): + """Test priority calculation for various component and input combinations.""" + # For variadic inputs, set up senders if needed + for socket in component["input_sockets"].values(): + if socket.is_variadic and not hasattr(socket, "senders"): + socket.senders = ["component1", "component2"] + + assert Pipeline._calculate_priority(component, inputs) == expected_priority From 93f7a9def5c880c5c1e9d7fd0356c0f5b1ec73ca Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 14:54:15 +0100 Subject: [PATCH 14/88] revert accidental change in pyproject.toml --- pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyproject.toml b/pyproject.toml index 85750257c5..73031b8130 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -88,7 +88,7 @@ extra-dependencies = [ "huggingface_hub>=0.27.0", # Hugging Face API Generators and Embedders "sentence-transformers>=3.0.0", # SentenceTransformersTextEmbedder and SentenceTransformersDocumentEmbedder "langdetect", # TextLanguageRouter and DocumentLanguageClassifier - #"openai-whisper>=20231106", # LocalWhisperTranscriber + "openai-whisper>=20231106", # LocalWhisperTranscriber "arrow>=1.3.0", # Jinja2TimeExtension # NamedEntityExtractor From bbba1b2845a1d0430f0cc907ad7bb2f5144013a5 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 15:13:04 +0100 Subject: [PATCH 15/88] test format conversion --- test/core/pipeline/test_pipeline.py | 41 +++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 7e393cb198..505fd6d0fa 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1255,3 +1255,44 @@ def test__calculate_priority(self, component, inputs, expected_priority, test_de socket.senders = ["component1", "component2"] assert Pipeline._calculate_priority(component, inputs) == expected_priority + + @pytest.mark.parametrize( + "pipeline_inputs,expected_output", + [ + # Test case 1: Empty input + ({}, {}), + # Test case 2: Single component, multiple inputs + ( + {"component1": {"input1": 42, "input2": "test", "input3": True}}, + { + "component1": { + "input1": [{"sender": None, "value": 42}], + "input2": [{"sender": None, "value": "test"}], + "input3": [{"sender": None, "value": True}], + } + }, + ), + # Test case 3: Multiple components + ( + { + "component1": {"input1": 42, "input2": "test"}, + "component2": {"input3": [1, 2, 3], "input4": {"key": "value"}}, + }, + { + "component1": { + "input1": [{"sender": None, "value": 42}], + "input2": [{"sender": None, "value": "test"}], + }, + "component2": { + "input3": [{"sender": None, "value": [1, 2, 3]}], + "input4": [{"sender": None, "value": {"key": "value"}}], + }, + }, + ), + ], + ids=["empty_input", "single_component_multiple_inputs", "multiple_components"], + ) + def test_convert_from_legacy_format(self, pipeline_inputs, expected_output): + """Test conversion of legacy pipeline inputs to internal format.""" + result = Pipeline._convert_from_legacy_format(pipeline_inputs) + assert result == expected_output From c00f8c537221f5d82db7cc642138c345e531a9f0 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 15:13:53 +0100 Subject: [PATCH 16/88] adapt to naming convention --- test/core/pipeline/test_pipeline.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 505fd6d0fa..b0b6a152e3 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1292,7 +1292,7 @@ def test__calculate_priority(self, component, inputs, expected_priority, test_de ], ids=["empty_input", "single_component_multiple_inputs", "multiple_components"], ) - def test_convert_from_legacy_format(self, pipeline_inputs, expected_output): + def test__convert_from_legacy_format(self, pipeline_inputs, expected_output): """Test conversion of legacy pipeline inputs to internal format.""" result = Pipeline._convert_from_legacy_format(pipeline_inputs) assert result == expected_output From 235aa478cc942771f4acac930f0073609009f965 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 15:19:37 +0100 Subject: [PATCH 17/88] chore: proper docstrings and type hints for PQ --- haystack/core/pipeline/utils.py | 89 ++++++++++++++++++++++++++------- 1 file changed, 72 insertions(+), 17 deletions(-) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index ec337806d8..5f40168747 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -5,7 +5,7 @@ import heapq from itertools import count -from typing import Optional, Tuple +from typing import Optional, Tuple, Any def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: @@ -22,46 +22,101 @@ def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: return (split_str[0], split_str[1]) return connection, None - class FIFOPriorityQueue: """ A priority queue that maintains FIFO order for items of equal priority. - Items with the same priority are placed to the right of existing items. + Items with the same priority are processed in the order they were added. + + This queue ensures that when multiple items share the same priority level, + they are dequeued in the same order they were enqueued (First-In-First-Out). """ - def __init__(self): - self._queue = [] # list of tuples (priority, count, item) - self._counter = count() # unique sequence of numbers + def __init__(self) -> None: + """ + Initialize a new FIFO priority queue. + """ + # List of tuples (priority, count, item) where count ensures FIFO order + self._queue = [] + # Counter to maintain insertion order for equal priorities + self._counter = count() + + def push(self, item: Any, priority: int) -> None: + """ + Push an item into the queue with a given priority. - def push(self, item, priority): - """Push an item with given priority. Items with equal priority maintain FIFO order.""" + Items with equal priority maintain FIFO ordering based on insertion time. + Lower priority numbers are dequeued first. + + :param item: + The item to insert into the queue. + :param priority: + Priority level for the item. Lower numbers indicate higher priority. + """ count = next(self._counter) entry = (priority, count, item) heapq.heappush(self._queue, entry) - def pop(self): - """Remove and return tuple of (priority, item) with lowest priority.""" + def pop(self) -> Tuple[int, Any]: + """ + Remove and return the highest priority item from the queue. + + For items with equal priority, returns the one that was inserted first. + + :returns: + A tuple containing (priority, item) with the lowest priority number. + :raises IndexError: + If the queue is empty. + """ if not self._queue: raise IndexError("pop from empty queue") priority, count, item = heapq.heappop(self._queue) return priority, item - def peek(self): - """Return but don't remove tuple of (priority, item) with lowest priority.""" + def peek(self) -> Tuple[int, Any]: + """ + Return but don't remove the highest priority item from the queue. + + For items with equal priority, returns the one that was inserted first. + + :returns: + A tuple containing (priority, item) with the lowest priority number. + :raises IndexError: + If the queue is empty. + """ if not self._queue: raise IndexError("peek at empty queue") priority, count, item = self._queue[0] return priority, item - def get(self): - """Remove and return tuple of (priority, item), or None if queue is empty.""" + def get(self) -> Optional[Tuple[int, Any]]: + """ + Remove and return the highest priority item from the queue. + + For items with equal priority, returns the one that was inserted first. + Unlike pop(), returns None if the queue is empty instead of raising an exception. + + :returns: + A tuple containing (priority, item), or None if the queue is empty. + """ if not self._queue: return None priority, count, item = heapq.heappop(self._queue) return priority, item - def __len__(self): + def __len__(self) -> int: + """ + Return the number of items in the queue. + + :returns: + The number of items currently in the queue. + """ return len(self._queue) - def __bool__(self): - return bool(self._queue) + def __bool__(self) -> bool: + """ + Return True if the queue has items, False if empty. + + :returns: + True if the queue contains items, False otherwise. + """ + return bool(self._queue) \ No newline at end of file From 99ea5d5464d60fd8d17a32cda2fbdb6cb7d5ec46 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 15:19:51 +0100 Subject: [PATCH 18/88] format --- haystack/core/pipeline/utils.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 5f40168747..de18a7b508 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -22,6 +22,7 @@ def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: return (split_str[0], split_str[1]) return connection, None + class FIFOPriorityQueue: """ A priority queue that maintains FIFO order for items of equal priority. @@ -119,4 +120,4 @@ def __bool__(self) -> bool: :returns: True if the queue contains items, False otherwise. """ - return bool(self._queue) \ No newline at end of file + return bool(self._queue) From 7577d9b0cb6f670af4afcc4c1a9fc84b48b817ed Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 16:12:52 +0100 Subject: [PATCH 19/88] add more unit tests --- test/core/pipeline/test_pipeline.py | 201 +++++++++++++++++++++++++++- 1 file changed, 200 insertions(+), 1 deletion(-) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index b0b6a152e3..776783597c 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -12,9 +12,16 @@ from haystack.components.joiners import BranchJoiner from haystack.core.component import component from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic, _empty -from haystack.core.errors import DeserializationError, PipelineConnectError, PipelineDrawingError, PipelineError +from haystack.core.errors import ( + DeserializationError, + PipelineConnectError, + PipelineDrawingError, + PipelineError, + PipelineMaxComponentRuns, +) from haystack.core.pipeline import Pipeline, PredefinedPipeline from haystack.core.pipeline.pipeline import ComponentPriority, _NO_OUTPUT_PRODUCED +from haystack.core.pipeline.utils import FIFOPriorityQueue from haystack.core.serialization import DeserializationCallbacks from haystack.testing.factory import component_class @@ -44,6 +51,24 @@ def run(self, input_: str): return {"value": input_} +@pytest.fixture +def regular_output_socket(): + """Output socket for a regular (non-variadic) connection with receivers""" + return OutputSocket("output1", int, receivers=["receiver1", "receiver2"]) + + +@pytest.fixture +def regular_input_socket(): + """Regular (non-variadic) input socket with a single sender""" + return InputSocket("input1", int, senders=["sender1"]) + + +@pytest.fixture +def lazy_variadic_input_socket(): + """Lazy variadic input socket with multiple senders""" + return InputSocket("variadic_input", Variadic[int], senders=["sender1", "sender2"]) + + class TestPipeline: """ This class contains only unit tests for the Pipeline class. @@ -1296,3 +1321,177 @@ def test__convert_from_legacy_format(self, pipeline_inputs, expected_output): """Test conversion of legacy pipeline inputs to internal format.""" result = Pipeline._convert_from_legacy_format(pipeline_inputs) assert result == expected_output + + @pytest.mark.parametrize( + "socket_type,existing_inputs,expected_count", + [ + ("regular", None, 1), # Regular socket should overwrite + ("regular", [{"sender": "other", "value": 24}], 1), # Should still overwrite + ("lazy_variadic", None, 1), # First input to lazy variadic + ("lazy_variadic", [{"sender": "other", "value": 24}], 2), # Should append + ], + ids=["regular-new", "regular-existing", "variadic-new", "variadic-existing"], + ) + def test__write_component_outputs_different_sockets( + self, + socket_type, + existing_inputs, + expected_count, + regular_output_socket, + regular_input_socket, + lazy_variadic_input_socket, + ): + """Test writing to different socket types with various existing input states""" + receiver_socket = lazy_variadic_input_socket if socket_type == "lazy_variadic" else regular_input_socket + socket_name = receiver_socket.name + receivers = [("receiver1", regular_output_socket, receiver_socket)] + + inputs = {} + if existing_inputs: + inputs = {"receiver1": {socket_name: existing_inputs}} + + component_outputs = {"output1": 42} + + pruned_outputs, updated_inputs = Pipeline._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs=inputs, + receivers=receivers, + include_outputs_from=[], + ) + + assert len(updated_inputs["receiver1"][socket_name]) == expected_count + assert {"sender": "sender1", "value": 42} in updated_inputs["receiver1"][socket_name] + + @pytest.mark.parametrize( + "component_outputs,include_outputs,expected_pruned", + [ + ({"output1": 42, "output2": 24}, [], {"output2": 24}), # Prune consumed outputs only + ({"output1": 42, "output2": 24}, ["sender1"], {"output1": 42, "output2": 24}), # Keep all outputs + ({}, [], {}), # No outputs case + ], + ids=["prune-consumed", "keep-all", "no-outputs"], + ) + def test__write_component_outputs_output_pruning( + self, component_outputs, include_outputs, expected_pruned, regular_output_socket, regular_input_socket + ): + """Test output pruning behavior under different scenarios""" + receivers = [("receiver1", regular_output_socket, regular_input_socket)] + + pruned_outputs, _ = Pipeline._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs={}, + receivers=receivers, + include_outputs_from=include_outputs, + ) + + assert pruned_outputs == expected_pruned + + @pytest.mark.parametrize( + "output_value", + [42, None, _NO_OUTPUT_PRODUCED, "string_value", 3.14], + ids=["int", "none", "no-output", "string", "float"], + ) + def test__write_component_outputs_different_output_values( + self, output_value, regular_output_socket, regular_input_socket + ): + """Test handling of different output values""" + receivers = [("receiver1", regular_output_socket, regular_input_socket)] + component_outputs = {"output1": output_value} + + _, updated_inputs = Pipeline._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs={}, + receivers=receivers, + include_outputs_from=[], + ) + + assert updated_inputs["receiver1"]["input1"] == [{"sender": "sender1", "value": output_value}] + + @pytest.mark.parametrize("receivers_count", [1, 2, 3], ids=["single-receiver", "two-receivers", "three-receivers"]) + def test__write_component_outputs_multiple_receivers( + self, receivers_count, regular_output_socket, regular_input_socket + ): + """Test writing to multiple receivers""" + receivers = [(f"receiver{i}", regular_output_socket, regular_input_socket) for i in range(receivers_count)] + component_outputs = {"output1": 42} + + _, updated_inputs = Pipeline._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs={}, + receivers=receivers, + include_outputs_from=[], + ) + + for i in range(receivers_count): + receiver_name = f"receiver{i}" + assert receiver_name in updated_inputs + assert updated_inputs[receiver_name]["input1"] == [{"sender": "sender1", "value": 42}] + + def test__get_next_runnable_component_empty(self): + """Test with empty queue returns None""" + queue = FIFOPriorityQueue() + pipeline = Pipeline() + result = pipeline._get_next_runnable_component(queue) + assert result is None + + def test__get_next_runnable_component_blocked(self): + """Test component with BLOCKED priority returns None""" + pipeline = Pipeline() + queue = FIFOPriorityQueue() + queue.push("blocked_component", ComponentPriority.BLOCKED) + result = pipeline._get_next_runnable_component(queue) + assert result is None + + @patch("haystack.core.pipeline.Pipeline._get_component_with_graph_metadata") + def test__get_next_runnable_component_max_visits(self, mock_get_component_with_graph_metadata): + """Test component exceeding max visits raises exception""" + pipeline = Pipeline(max_runs_per_component=2) + queue = FIFOPriorityQueue() + queue.push("ready_component", ComponentPriority.READY) + mock_get_component_with_graph_metadata.return_value = {"instance": "test", "visits": 3} + + with pytest.raises(PipelineMaxComponentRuns) as exc_info: + pipeline._get_next_runnable_component(queue) + + assert "Maximum run count 2 reached for component 'ready_component'" in str(exc_info.value) + + @patch("haystack.core.pipeline.Pipeline._get_component_with_graph_metadata") + def test__get_next_runnable_component_ready(self, mock_get_component_with_graph_metadata): + """Test component that is READY""" + pipeline = Pipeline() + queue = FIFOPriorityQueue() + queue.push("ready_component", ComponentPriority.READY) + mock_get_component_with_graph_metadata.return_value = {"instance": "test", "visits": 1} + + priority, component_name, component = pipeline._get_next_runnable_component(queue) + + assert priority == ComponentPriority.READY + assert component_name == "ready_component" + assert component == {"instance": "test", "visits": 1} + + @pytest.mark.parametrize( + "queue_setup,expected_stale", + [ + # Empty queue case + (None, True), + # READY priority case + ((ComponentPriority.READY, "component1"), False), + # DEFER priority case + ((ComponentPriority.DEFER, "component1"), True), + ], + ids=["empty-queue", "ready-component", "deferred-component"], + ) + def test__is_queue_stale(self, queue_setup, expected_stale): + # Setup queue + queue = FIFOPriorityQueue() + if queue_setup: + priority, component_name = queue_setup + queue.push(component_name, priority) + + # Check if queue is stale + result = Pipeline._is_queue_stale(queue) + assert result == expected_stale From 25c64cddffe60069b19991a755a539720b640556 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 16:15:10 +0100 Subject: [PATCH 20/88] rm unneeded comments --- test/core/pipeline/test_pipeline.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 776783597c..6ece27b80b 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1486,12 +1486,10 @@ def test__get_next_runnable_component_ready(self, mock_get_component_with_graph_ ids=["empty-queue", "ready-component", "deferred-component"], ) def test__is_queue_stale(self, queue_setup, expected_stale): - # Setup queue queue = FIFOPriorityQueue() if queue_setup: priority, component_name = queue_setup queue.push(component_name, priority) - # Check if queue is stale result = Pipeline._is_queue_stale(queue) assert result == expected_stale From 5a1bbd87b3ab2aa3a49f507c970b5b8b3c32372c Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 16:28:26 +0100 Subject: [PATCH 21/88] test input consumption --- test/core/pipeline/test_pipeline.py | 124 ++++++++++++++++++++++++++++ 1 file changed, 124 insertions(+) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 6ece27b80b..9eaf51116d 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1493,3 +1493,127 @@ def test__is_queue_stale(self, queue_setup, expected_stale): result = Pipeline._is_queue_stale(queue) assert result == expected_stale + + @patch("haystack.core.pipeline.Pipeline._calculate_priority") + @patch("haystack.core.pipeline.Pipeline._get_component_with_graph_metadata") + def test_fill_queue(self, mock_get_metadata, mock_calc_priority): + pipeline = Pipeline() + component_names = ["comp1", "comp2"] + inputs = {"comp1": {"input1": "value1"}, "comp2": {"input2": "value2"}} + + mock_get_metadata.side_effect = lambda name: {"component": f"mock_{name}"} + mock_calc_priority.side_effect = [1, 2] # Different priorities for testing + + queue = pipeline._fill_queue(component_names, inputs) + + assert mock_get_metadata.call_count == 2 + assert mock_calc_priority.call_count == 2 + + # Verify correct calls for first component + mock_get_metadata.assert_any_call("comp1") + mock_calc_priority.assert_any_call({"component": "mock_comp1"}, {"input1": "value1"}) + + # Verify correct calls for second component + mock_get_metadata.assert_any_call("comp2") + mock_calc_priority.assert_any_call({"component": "mock_comp2"}, {"input2": "value2"}) + + assert queue.pop() == (1, "comp1") + assert queue.pop() == (2, "comp2") + + @pytest.mark.parametrize( + "input_sockets,component_inputs,expected_consumed,expected_remaining", + [ + # Regular socket test + ( + {"input1": InputSocket("input1", int)}, + {"input1": [{"sender": "comp1", "value": 42}, {"sender": "comp2", "value": 24}]}, + {"input1": 42}, # Should take first valid input + {}, # All pipeline inputs should be removed + ), + # Regular socket with user input + ( + {"input1": InputSocket("input1", int)}, + { + "input1": [ + {"sender": "comp1", "value": 42}, + {"sender": None, "value": 24}, # User input + ] + }, + {"input1": 42}, + {"input1": [{"sender": None, "value": 24}]}, # User input should remain + ), + # Greedy variadic socket + ( + {"greedy": InputSocket("greedy", GreedyVariadic[int])}, + { + "greedy": [ + {"sender": "comp1", "value": 42}, + {"sender": None, "value": 24}, # User input + {"sender": "comp2", "value": 33}, + ] + }, + {"greedy": [42]}, # Takes first valid input + {}, # All inputs removed for greedy sockets + ), + # Lazy variadic socket + ( + {"lazy": InputSocket("lazy", Variadic[int])}, + { + "lazy": [ + {"sender": "comp1", "value": 42}, + {"sender": "comp2", "value": 24}, + {"sender": None, "value": 33}, # User input + ] + }, + {"lazy": [42, 24, 33]}, # Takes all valid inputs + {"lazy": [{"sender": None, "value": 33}]}, # User input remains + ), + # Mixed socket types + ( + { + "regular": InputSocket("regular", int), + "greedy": InputSocket("greedy", GreedyVariadic[int]), + "lazy": InputSocket("lazy", Variadic[int]), + }, + { + "regular": [{"sender": "comp1", "value": 42}, {"sender": None, "value": 24}], + "greedy": [{"sender": "comp2", "value": 33}, {"sender": None, "value": 15}], + "lazy": [{"sender": "comp3", "value": 55}, {"sender": "comp4", "value": 66}], + }, + {"regular": 42, "greedy": [33], "lazy": [55, 66]}, + {"regular": [{"sender": None, "value": 24}]}, # Only non-greedy user input remains + ), + # Filtering _NO_OUTPUT_PRODUCED + ( + {"input1": InputSocket("input1", int)}, + { + "input1": [ + {"sender": "comp1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "comp2", "value": 42}, + {"sender": "comp2", "value": _NO_OUTPUT_PRODUCED}, + ] + }, + {"input1": 42}, # Should skip _NO_OUTPUT_PRODUCED values + {}, # All inputs consumed + ), + ], + ids=[ + "regular-socket", + "regular-with-user-input", + "greedy-variadic", + "lazy-variadic", + "mixed-sockets", + "no-output-filtering", + ], + ) + def test__consume_component_inputs(self, input_sockets, component_inputs, expected_consumed, expected_remaining): + # Setup + component = {"input_sockets": input_sockets} + inputs = {"test_component": component_inputs} + + # Run + consumed, updated_inputs = Pipeline._consume_component_inputs("test_component", component, inputs) + + # Verify + assert consumed == expected_consumed + assert updated_inputs["test_component"] == expected_remaining From 6820fc3307ab4e537e51ac98f66ebc0652344a07 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 17:21:57 +0100 Subject: [PATCH 22/88] lint --- haystack/core/pipeline/base.py | 3 +-- haystack/core/pipeline/component_checks.py | 4 ++-- haystack/core/pipeline/pipeline.py | 2 +- haystack/core/pipeline/utils.py | 6 +++--- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 730cd888dd..fe42c5e781 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -7,7 +7,7 @@ from copy import deepcopy from datetime import datetime from pathlib import Path -from typing import Any, Dict, Iterator, List, Optional, Set, TextIO, Tuple, Type, TypeVar, Union +from typing import Any, Dict, Iterator, List, Optional, TextIO, Tuple, Type, TypeVar, Union import networkx # type:ignore @@ -18,7 +18,6 @@ PipelineConnectError, PipelineDrawingError, PipelineError, - PipelineRuntimeError, PipelineUnmarshalError, PipelineValidationError, ) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index 92a99d89b2..13a91a106e 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -103,7 +103,7 @@ def can_not_receive_inputs_from_pipeline(component: Dict) -> bool: Checks if a component can not receive inputs from any other components in the pipeline. :param: Component metadata and the component instance. """ - return all([len(sock.senders) == 0 for sock in component["input_sockets"].values()]) + return all(len(sock.senders) == 0 for sock in component["input_sockets"].values()) def all_socket_predecessors_executed(socket: InputSocket, socket_inputs: List[Dict]) -> bool: @@ -163,7 +163,7 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict socket.is_variadic and socket.is_greedy and len(socket_inputs) > 0 - and any([sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs]) + and any(sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs) ): return True diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 7e378ba1a8..2330204413 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -431,7 +431,7 @@ def run( # noqa: PLR0915, PLR0912 if candidate is None: break - priority, component_name, component = candidate + _, component_name, component = candidate component_outputs, inputs = self._run_component(component, inputs, parent_span=span) component_pipeline_outputs, inputs = self._write_component_outputs( component_name=component_name, diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index de18a7b508..92178e0a20 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -70,7 +70,7 @@ def pop(self) -> Tuple[int, Any]: """ if not self._queue: raise IndexError("pop from empty queue") - priority, count, item = heapq.heappop(self._queue) + priority, _, item = heapq.heappop(self._queue) return priority, item def peek(self) -> Tuple[int, Any]: @@ -86,7 +86,7 @@ def peek(self) -> Tuple[int, Any]: """ if not self._queue: raise IndexError("peek at empty queue") - priority, count, item = self._queue[0] + priority, _, item = self._queue[0] return priority, item def get(self) -> Optional[Tuple[int, Any]]: @@ -101,7 +101,7 @@ def get(self) -> Optional[Tuple[int, Any]]: """ if not self._queue: return None - priority, count, item = heapq.heappop(self._queue) + priority, _, item = heapq.heappop(self._queue) return priority, item def __len__(self) -> int: From d624e5793297a180c306015a22e0d8b539128224 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 17:46:09 +0100 Subject: [PATCH 23/88] fix: docstrings --- haystack/core/pipeline/component_checks.py | 15 +++++++++++++-- haystack/core/pipeline/pipeline.py | 12 ++++++++---- haystack/core/pipeline/utils.py | 3 +-- 3 files changed, 22 insertions(+), 8 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index 13a91a106e..be371e12e5 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -1,7 +1,6 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 - from typing import Any, Dict, List from haystack.core.component.types import _empty, InputSocket @@ -12,6 +11,7 @@ def can_component_run(component: Dict, inputs: Dict) -> bool: """ Checks if the component can run, given the current state of its inputs. + A component needs to pass two gates so that it is ready to run: 1. It has received all mandatory inputs. 2. It has received a trigger. @@ -51,6 +51,7 @@ def has_any_trigger(component: Dict, inputs: Dict) -> bool: def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: """ Checks if all mandatory sockets of a component have enough inputs for the component to execute. + :param component: Component metadata and the component instance. :param inputs: Inputs for the component. """ @@ -84,6 +85,7 @@ def any_predecessors_provided_input(component: Dict, inputs: Dict) -> bool: def any_socket_value_from_predecessor_received(socket_inputs: List[Dict[str, Any]]) -> bool: """ Checks if a component socket received input from any predecessors. + :param socket_inputs: Inputs for the component's socket. """ # When sender is None, the input was provided from outside the pipeline. @@ -93,6 +95,7 @@ def any_socket_value_from_predecessor_received(socket_inputs: List[Dict[str, Any def has_user_input(inputs: Dict) -> bool: """ Checks if a component has received input from outside the pipeline (e.g. user input). + :param inputs: Inputs for the component. """ return any(inp for socket in inputs.values() for inp in socket if inp["sender"] is None) @@ -101,6 +104,7 @@ def has_user_input(inputs: Dict) -> bool: def can_not_receive_inputs_from_pipeline(component: Dict) -> bool: """ Checks if a component can not receive inputs from any other components in the pipeline. + :param: Component metadata and the component instance. """ return all(len(sock.senders) == 0 for sock in component["input_sockets"].values()) @@ -109,6 +113,7 @@ def can_not_receive_inputs_from_pipeline(component: Dict) -> bool: def all_socket_predecessors_executed(socket: InputSocket, socket_inputs: List[Dict]) -> bool: """ Checks if all components connecting to an InputSocket have executed. + :param: The InputSocket of a component. :param: socket_inputs: Inputs for the socket. """ @@ -121,6 +126,7 @@ def all_socket_predecessors_executed(socket: InputSocket, socket_inputs: List[Di def any_socket_input_received(socket_inputs: List[Dict]) -> bool: """ Checks if a socket has received any input from any other components in the pipeline or from outside the pipeline. + :param socket_inputs: Inputs for the socket. """ return any(inp["value"] != _NO_OUTPUT_PRODUCED for inp in socket_inputs) @@ -129,6 +135,7 @@ def any_socket_input_received(socket_inputs: List[Dict]) -> bool: def has_lazy_variadic_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict]) -> bool: """ Checks if a lazy variadic socket has received all expected inputs from other components in the pipeline. + :param socket: The InputSocket of a component. :param socket_inputs: Inputs for the socket. """ @@ -143,6 +150,7 @@ def has_lazy_variadic_socket_received_all_inputs(socket: InputSocket, socket_inp def is_socket_lazy_variadic(socket: InputSocket) -> bool: """ Checks if an InputSocket is a lazy variadic socket. + :param socket: The InputSocket of a component. """ return socket.is_variadic and not socket.is_greedy @@ -151,6 +159,7 @@ def is_socket_lazy_variadic(socket: InputSocket) -> bool: def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict]) -> bool: """ Checks if a socket has received all expected inputs. + :param socket: The InputSocket of a component. :param socket_inputs: Inputs for the socket. """ @@ -181,6 +190,7 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict def all_predecessors_executed(component: Dict, inputs: Dict) -> bool: """ Checks if all predecessors of a component have executed. + :param component: Component metadata and the component instance. :param inputs: Inputs for the component. """ @@ -193,7 +203,7 @@ def all_predecessors_executed(component: Dict, inputs: Dict) -> bool: def are_all_lazy_variadic_sockets_resolved(component: Dict, inputs: Dict) -> bool: """ Checks if the final state for all lazy variadic sockets of a component is resolved. - Either because all inputs were received, or because all predecessors executed. + :param component: Component metadata and the component instance. :param inputs: Inputs for the component. """ @@ -212,6 +222,7 @@ def are_all_lazy_variadic_sockets_resolved(component: Dict, inputs: Dict) -> boo def is_any_greedy_socket_ready(component: Dict, inputs: Dict) -> bool: """ Checks if the component has any greedy socket that is ready to run. + :param component: Component metadata and the component instance. :param inputs: Inputs for the component. """ diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 2330204413..87cced9898 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -105,6 +105,7 @@ def _run_component( def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict) -> Tuple[Dict, Dict]: """ Extracts the inputs needed to run for the component and removes them from the global inputs state. + :param component: Component with component metadata. :param inputs: Global inputs state. :returns: The inputs for the component and the new state of global inputs. @@ -146,6 +147,7 @@ def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict def _convert_from_legacy_format(pipeline_inputs: Dict[str, Any]) -> Dict[str, Dict[str, List]]: """ Converts the inputs to the pipeline to the format that is needed for the internal `Pipeline.run` logic. + :param pipeline_inputs: Inputs to the pipeline. :returns: Converted inputs that can be used by the internal `Pipeline.run` logic. """ @@ -177,6 +179,7 @@ def _fill_queue(self, component_names: List[str], inputs: Dict[str, Any]) -> FIF def _calculate_priority(component: Dict, inputs: Dict) -> ComponentPriority: """ Calculates the execution priority for a component depending on the component's inputs. + :param component: Component metadata and component instance. :param inputs: Inputs to the component. :returns: Priority value for the component. @@ -200,6 +203,7 @@ def _get_next_runnable_component( ) -> Union[Tuple[Component, str, Dict], None]: """ Returns the next runnable component alongside its metadata from the priority queue. + :param priority_queue: Priority queue of component names. :returns: The next runnable component, the component name, and its priority or None if no component in the queue can run. :raises: PipelineMaxComponentRuns if the next runnable component has exceeded the maximum number of runs. @@ -223,6 +227,7 @@ def _write_component_outputs( ) -> Tuple[Dict, Dict]: """ Distributes the outputs of a component to the input sockets that it is connected to. + :param component_name: The name of the component. :param component_outputs: The outputs of the component. :param inputs: The current global input state. @@ -268,6 +273,7 @@ def _merge_component_and_pipeline_outputs( ) -> Dict: """ Merges the outputs of a component with the current pipeline outputs. + :param component_name: The name of the component. :param component_outputs: The outputs of the component. :param pipeline_outputs: The pipeline outputs. @@ -288,13 +294,11 @@ def _merge_component_and_pipeline_outputs( def _is_queue_stale(priority_queue: FIFOPriorityQueue) -> bool: """ Checks if the priority queue needs to be recomputed because the priorities might have changed. + :param priority_queue: Priority queue of component names. """ - next_priority_and_component = priority_queue and priority_queue.peek() - if not next_priority_and_component or next_priority_and_component[0] > ComponentPriority.READY: - return True + return len(priority_queue) == 0 or priority_queue.peek()[0] > ComponentPriority.READY - return False def run( # noqa: PLR0915, PLR0912 self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 92178e0a20..7e87279145 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -1,7 +1,6 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 - import heapq from itertools import count @@ -26,8 +25,8 @@ def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: class FIFOPriorityQueue: """ A priority queue that maintains FIFO order for items of equal priority. - Items with the same priority are processed in the order they were added. + Items with the same priority are processed in the order they were added. This queue ensures that when multiple items share the same priority level, they are dequeued in the same order they were enqueued (First-In-First-Out). """ From e1912f8994108c9e35bf5194db13744cba604308 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 17:56:04 +0100 Subject: [PATCH 24/88] lint --- haystack/core/pipeline/component_checks.py | 15 +++++++-------- haystack/core/pipeline/pipeline.py | 15 ++++++--------- haystack/core/pipeline/utils.py | 3 +-- 3 files changed, 14 insertions(+), 19 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index be371e12e5..8c3f384e70 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -3,7 +3,7 @@ # SPDX-License-Identifier: Apache-2.0 from typing import Any, Dict, List -from haystack.core.component.types import _empty, InputSocket +from haystack.core.component.types import InputSocket, _empty _NO_OUTPUT_PRODUCED = _empty @@ -61,9 +61,11 @@ def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: if socket.is_mandatory: socket_inputs = inputs.get(socket_name, []) expected_mandatory_sockets.add(socket_name) - if is_socket_lazy_variadic(socket) and any_socket_input_received(socket_inputs): - filled_mandatory_sockets.add(socket_name) - elif has_socket_received_all_inputs(socket, socket_inputs): + if ( + is_socket_lazy_variadic(socket) and + any_socket_input_received(socket_inputs) or + has_socket_received_all_inputs(socket, socket_inputs) + ): filled_mandatory_sockets.add(socket_name) return filled_mandatory_sockets == expected_mandatory_sockets @@ -181,10 +183,7 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict return True # The socket is not variadic and the only expected input is complete. - if not socket.is_variadic and socket_inputs[0]["value"] != _NO_OUTPUT_PRODUCED: - return True - - return False + return not socket.is_variadic and socket_inputs[0]["value"] != _NO_OUTPUT_PRODUCED def all_predecessors_executed(component: Dict, inputs: Dict) -> bool: diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 87cced9898..15a5914cdd 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -3,25 +3,22 @@ # SPDX-License-Identifier: Apache-2.0 from copy import deepcopy -from typing import Any, Dict, List, Mapping, Optional, Set, Tuple, Union from enum import IntEnum +from typing import Any, Dict, List, Mapping, Optional, Set, Tuple, Union from haystack import logging, tracing from haystack.core.component import Component from haystack.core.errors import PipelineMaxComponentRuns, PipelineRuntimeError from haystack.core.pipeline.base import PipelineBase from haystack.core.pipeline.component_checks import ( + _NO_OUTPUT_PRODUCED, + all_predecessors_executed, + are_all_lazy_variadic_sockets_resolved, can_component_run, is_any_greedy_socket_ready, is_socket_lazy_variadic, ) -from haystack.core.pipeline.component_checks import ( - all_predecessors_executed, - are_all_lazy_variadic_sockets_resolved, - _NO_OUTPUT_PRODUCED, -) from haystack.core.pipeline.utils import FIFOPriorityQueue - from haystack.telemetry import pipeline_running logger = logging.getLogger(__name__) @@ -205,7 +202,8 @@ def _get_next_runnable_component( Returns the next runnable component alongside its metadata from the priority queue. :param priority_queue: Priority queue of component names. - :returns: The next runnable component, the component name, and its priority or None if no component in the queue can run. + :returns: The next runnable component, the component name, and its priority + or None if no component in the queue can run. :raises: PipelineMaxComponentRuns if the next runnable component has exceeded the maximum number of runs. """ priority_and_component_name = priority_queue.get() @@ -446,7 +444,6 @@ def run( # noqa: PLR0915, PLR0912 ) # TODO check original logic in pipeline, it looks like we don't want to override existing outputs # e.g. for cycles but the tests check if intermediate outputs from components in cycles are overwritten - # pipeline_outputs = self._merge_component_and_pipeline_outputs(component_name, component_pipeline_outputs, pipeline_outputs) if component_pipeline_outputs: pipeline_outputs = {**pipeline_outputs, component_name: component_pipeline_outputs} if self._is_queue_stale(priority_queue): diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 7e87279145..32f33d01ea 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -2,9 +2,8 @@ # # SPDX-License-Identifier: Apache-2.0 import heapq - from itertools import count -from typing import Optional, Tuple, Any +from typing import Any, Optional, Tuple def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: From acc17b1529dfcbe1170d93bac870a38cd804cc25 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 17:57:56 +0100 Subject: [PATCH 25/88] format --- haystack/core/pipeline/component_checks.py | 6 +++--- haystack/core/pipeline/pipeline.py | 1 - 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index 8c3f384e70..b8587ced9d 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -62,9 +62,9 @@ def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: socket_inputs = inputs.get(socket_name, []) expected_mandatory_sockets.add(socket_name) if ( - is_socket_lazy_variadic(socket) and - any_socket_input_received(socket_inputs) or - has_socket_received_all_inputs(socket, socket_inputs) + is_socket_lazy_variadic(socket) + and any_socket_input_received(socket_inputs) + or has_socket_received_all_inputs(socket, socket_inputs) ): filled_mandatory_sockets.add(socket_name) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 15a5914cdd..2cd7ae4ccf 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -297,7 +297,6 @@ def _is_queue_stale(priority_queue: FIFOPriorityQueue) -> bool: """ return len(priority_queue) == 0 or priority_queue.peek()[0] > ComponentPriority.READY - def run( # noqa: PLR0915, PLR0912 self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None ) -> Dict[str, Any]: From 207eaba9ee8eb409e462f1fe7359922d377ab8bf Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 18:04:57 +0100 Subject: [PATCH 26/88] format --- haystack/core/component/component.py | 6 +++--- test/components/audio/test_whisper_local.py | 12 ++++++------ .../converters/test_docx_file_to_document.py | 6 +++--- .../embedders/test_openai_document_embedder.py | 6 +++--- .../embedders/test_openai_text_embedder.py | 6 +++--- test/components/joiners/test_document_joiner.py | 6 +++--- test/components/routers/test_conditional_router.py | 6 +++--- 7 files changed, 24 insertions(+), 24 deletions(-) diff --git a/haystack/core/component/component.py b/haystack/core/component/component.py index fe6902b2f1..d77fd77593 100644 --- a/haystack/core/component/component.py +++ b/haystack/core/component/component.py @@ -268,9 +268,9 @@ def __call__(cls, *args, **kwargs): try: pre_init_hook.in_progress = True named_positional_args = ComponentMeta._positional_to_kwargs(cls, args) - assert ( - set(named_positional_args.keys()).intersection(kwargs.keys()) == set() - ), "positional and keyword arguments overlap" + assert set(named_positional_args.keys()).intersection(kwargs.keys()) == set(), ( + "positional and keyword arguments overlap" + ) kwargs.update(named_positional_args) pre_init_hook.callback(cls, kwargs) instance = super().__call__(**kwargs) diff --git a/test/components/audio/test_whisper_local.py b/test/components/audio/test_whisper_local.py index 28463c4ce6..394a9c4000 100644 --- a/test/components/audio/test_whisper_local.py +++ b/test/components/audio/test_whisper_local.py @@ -190,14 +190,14 @@ def test_whisper_local_transcriber(self, test_files_path): docs = output["documents"] assert len(docs) == 3 - assert all( - word in docs[0].content.strip().lower() for word in {"content", "the", "document"} - ), f"Expected words not found in: {docs[0].content.strip().lower()}" + assert all(word in docs[0].content.strip().lower() for word in {"content", "the", "document"}), ( + f"Expected words not found in: {docs[0].content.strip().lower()}" + ) assert test_files_path / "audio" / "this is the content of the document.wav" == docs[0].meta["audio_file"] - assert all( - word in docs[1].content.strip().lower() for word in {"context", "answer"} - ), f"Expected words not found in: {docs[1].content.strip().lower()}" + assert all(word in docs[1].content.strip().lower() for word in {"context", "answer"}), ( + f"Expected words not found in: {docs[1].content.strip().lower()}" + ) path = test_files_path / "audio" / "the context for this answer is here.wav" assert path.absolute() == docs[1].meta["audio_file"] diff --git a/test/components/converters/test_docx_file_to_document.py b/test/components/converters/test_docx_file_to_document.py index 9b4ee3fe60..c013759938 100644 --- a/test/components/converters/test_docx_file_to_document.py +++ b/test/components/converters/test_docx_file_to_document.py @@ -176,9 +176,9 @@ def test_run_with_table(self, test_files_path): table_index = next(i for i, part in enumerate(content_parts) if "| This | Is | Just a |" in part) # check that natural order of the document is preserved assert any("Donald Trump" in part for part in content_parts[:table_index]), "Text before table not found" - assert any( - "Now we are in Page 2" in part for part in content_parts[table_index + 1 :] - ), "Text after table not found" + assert any("Now we are in Page 2" in part for part in content_parts[table_index + 1 :]), ( + "Text after table not found" + ) def test_run_with_store_full_path_false(self, test_files_path): """ diff --git a/test/components/embedders/test_openai_document_embedder.py b/test/components/embedders/test_openai_document_embedder.py index 87ed6afbb6..7d43bcfa83 100644 --- a/test/components/embedders/test_openai_document_embedder.py +++ b/test/components/embedders/test_openai_document_embedder.py @@ -251,8 +251,8 @@ def test_run(self): assert len(doc.embedding) == 1536 assert all(isinstance(x, float) for x in doc.embedding) - assert ( - "text" in result["meta"]["model"] and "ada" in result["meta"]["model"] - ), "The model name does not contain 'text' and 'ada'" + assert "text" in result["meta"]["model"] and "ada" in result["meta"]["model"], ( + "The model name does not contain 'text' and 'ada'" + ) assert result["meta"]["usage"] == {"prompt_tokens": 15, "total_tokens": 15}, "Usage information does not match" diff --git a/test/components/embedders/test_openai_text_embedder.py b/test/components/embedders/test_openai_text_embedder.py index 31a0360555..695e6351f0 100644 --- a/test/components/embedders/test_openai_text_embedder.py +++ b/test/components/embedders/test_openai_text_embedder.py @@ -130,8 +130,8 @@ def test_run(self): assert len(result["embedding"]) == 1536 assert all(isinstance(x, float) for x in result["embedding"]) - assert ( - "text" in result["meta"]["model"] and "ada" in result["meta"]["model"] - ), "The model name does not contain 'text' and 'ada'" + assert "text" in result["meta"]["model"] and "ada" in result["meta"]["model"], ( + "The model name does not contain 'text' and 'ada'" + ) assert result["meta"]["usage"] == {"prompt_tokens": 6, "total_tokens": 6}, "Usage information does not match" diff --git a/test/components/joiners/test_document_joiner.py b/test/components/joiners/test_document_joiner.py index 6cc4f5f9e0..8160fdc48a 100644 --- a/test/components/joiners/test_document_joiner.py +++ b/test/components/joiners/test_document_joiner.py @@ -302,6 +302,6 @@ def test_test_score_norm_with_rrf(self): for i in range(len(join_results["documents"]) - 1) ) - assert ( - is_sorted - ), "Documents are not sorted in descending order by score, there is an issue with rff ranking" + assert is_sorted, ( + "Documents are not sorted in descending order by score, there is an issue with rff ranking" + ) diff --git a/test/components/routers/test_conditional_router.py b/test/components/routers/test_conditional_router.py index 66d941b645..478e62d5bf 100644 --- a/test/components/routers/test_conditional_router.py +++ b/test/components/routers/test_conditional_router.py @@ -436,9 +436,9 @@ def test_router_with_optional_parameters(self): # Test pipeline without path parameter result = pipe.run(data={"router": {"question": "What?"}}) - assert result["router"] == { - "fallback": "What?" - }, "Default route should work in pipeline when 'path' is not provided" + assert result["router"] == {"fallback": "What?"}, ( + "Default route should work in pipeline when 'path' is not provided" + ) # Test pipeline with path parameter result = pipe.run(data={"router": {"question": "What?", "path": "followup_short"}}) From 15611fc5ea7c97ce62fadb74136b3759899c61d3 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 18:06:27 +0100 Subject: [PATCH 27/88] fix license header --- haystack/core/pipeline/component_checks.py | 1 + 1 file changed, 1 insertion(+) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index b8587ced9d..30aaa153cb 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -1,6 +1,7 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 + from typing import Any, Dict, List from haystack.core.component.types import InputSocket, _empty From 87010dd6f4d6ec56484df4af761cb7557f858b35 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 18:07:28 +0100 Subject: [PATCH 28/88] fix license header --- haystack/core/pipeline/utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 32f33d01ea..2456ea886a 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -1,6 +1,7 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 + import heapq from itertools import count from typing import Any, Optional, Tuple From 7027572cabe0e157c4fdd4d76a5ad43ff3dcf7b7 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 13 Jan 2025 18:29:33 +0100 Subject: [PATCH 29/88] add component run tests --- test/core/pipeline/test_pipeline.py | 76 +++++++++++++---------------- 1 file changed, 34 insertions(+), 42 deletions(-) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 9eaf51116d..f14c5594d0 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -17,6 +17,7 @@ PipelineConnectError, PipelineDrawingError, PipelineError, + PipelineRuntimeError, PipelineMaxComponentRuns, ) from haystack.core.pipeline import Pipeline, PredefinedPipeline @@ -1025,52 +1026,43 @@ def test_connect_same_component_as_sender_and_receiver(self): with pytest.raises(PipelineConnectError): pipe.connect("single_component.out", "single_component.in") - def test__run_component(self, spying_tracer, caplog): - caplog.set_level(logging.INFO) - sentence_builder = component_class( - "SentenceBuilder", input_types={"words": List[str]}, output={"text": "some words"} - )() - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output={"doc": Document(content="some words")} - )() - document_cleaner = component_class( - "DocumentCleaner", - input_types={"doc": Document}, - output={"cleaned_doc": Document(content="some cleaner words")}, - )() + def test__run_component_success(self): + """Test successful component execution""" + joiner_1 = BranchJoiner(type_=str) + joiner_2 = BranchJoiner(type_=str) + pp = Pipeline() + pp.add_component("joiner_1", joiner_1) + pp.add_component("joiner_2", joiner_2) + pp.connect("joiner_1", "joiner_2") + inputs = {"joiner_1": {"value": [{"sender": None, "value": "test_value"}]}} + + outputs, updated_inputs = pp._run_component( + component=pp._get_component_with_graph_metadata("joiner_1"), inputs=inputs + ) - pipe = Pipeline() - pipe.add_component("sentence_builder", sentence_builder) - pipe.add_component("document_builder", document_builder) - pipe.add_component("document_cleaner", document_cleaner) - pipe.connect("sentence_builder.text", "document_builder.text") - pipe.connect("document_builder.doc", "document_cleaner.doc") - assert spying_tracer.spans == [] - res = pipe._run_component("document_builder", {"text": "whatever"}) - assert res == {"doc": Document(content="some words")} - - assert len(spying_tracer.spans) == 1 - span = spying_tracer.spans[0] - assert span.operation_name == "haystack.component.run" - assert span.tags == { - "haystack.component.name": "document_builder", - "haystack.component.type": "DocumentBuilder", - "haystack.component.input_types": {"text": "str"}, - "haystack.component.input_spec": {"text": {"type": "str", "senders": ["sentence_builder"]}}, - "haystack.component.output_spec": {"doc": {"type": "Document", "receivers": ["document_cleaner"]}}, - "haystack.component.visits": 1, - } + assert outputs == {"value": "test_value"} + # We remove input in greedy variadic sockets, even if they are from the user + assert "value" not in updated_inputs["joiner_1"] - assert caplog.messages == ["Running component document_builder"] + def test__run_component_fail(self): + """Test error when component doesn't return a dictionary""" - def test__run_component_with_variadic_input(self): - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() + @component + class WrongOutput: + @component.output_types(output=str) + def run(self, value: str): + return "not_a_dict" - pipe = Pipeline() - pipe.add_component("document_joiner", document_joiner) - inputs = {"docs": [Document(content="doc1"), Document(content="doc2")]} - pipe._run_component("document_joiner", inputs) - assert inputs == {"docs": []} + wrong = WrongOutput() + pp = Pipeline() + pp.add_component("wrong", wrong) + + inputs = {"wrong": {"value": [{"sender": None, "value": "test_value"}]}} + + with pytest.raises(PipelineRuntimeError) as exc_info: + pp._run_component(component=pp._get_component_with_graph_metadata("wrong"), inputs=inputs) + + assert "didn't return a dictionary" in str(exc_info.value) def test__find_receivers_from(self): sentence_builder = component_class( From 02c82b88eb0d74b5adffb4e17e62986ed085de78 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 14 Jan 2025 17:47:45 +0100 Subject: [PATCH 30/88] fix: pass correct input format to tracing --- haystack/core/pipeline/pipeline.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 2cd7ae4ccf..6e5dfa3048 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -54,12 +54,15 @@ def _run_component( """ instance: Component = component["instance"] component_name = self.get_component_name(instance) + component_inputs, inputs = self._consume_component_inputs( + component_name=component_name, component=component, inputs=inputs + ) with tracing.tracer.trace( "haystack.component.run", tags={ "haystack.component.name": component_name, "haystack.component.type": instance.__class__.__name__, - "haystack.component.input_types": {k: type(v).__name__ for k, v in inputs.items()}, + "haystack.component.input_types": {k: type(v).__name__ for k, v in component_inputs.items()}, "haystack.component.input_spec": { key: { "type": (value.type.__name__ if isinstance(value.type, type) else str(value.type)), @@ -77,9 +80,6 @@ def _run_component( }, parent_span=parent_span, ) as span: - component_inputs, inputs = self._consume_component_inputs( - component_name=component_name, component=component, inputs=inputs - ) # We deepcopy the inputs otherwise we might lose that information # when we delete them in case they're sent to other Components span.set_content_tag("haystack.component.input", deepcopy(component_inputs)) @@ -444,7 +444,7 @@ def run( # noqa: PLR0915, PLR0912 # TODO check original logic in pipeline, it looks like we don't want to override existing outputs # e.g. for cycles but the tests check if intermediate outputs from components in cycles are overwritten if component_pipeline_outputs: - pipeline_outputs = {**pipeline_outputs, component_name: component_pipeline_outputs} + pipeline_outputs[component_name] = component_pipeline_outputs if self._is_queue_stale(priority_queue): priority_queue = self._fill_queue(ordered_component_names, inputs) From 7e81ff97fc2c9a3787471013384e4c06ea09676a Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 14 Jan 2025 18:30:14 +0100 Subject: [PATCH 31/88] fix types --- haystack/core/pipeline/pipeline.py | 4 ++-- haystack/core/pipeline/utils.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 6e5dfa3048..a5ec5a9063 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -148,7 +148,7 @@ def _convert_from_legacy_format(pipeline_inputs: Dict[str, Any]) -> Dict[str, Di :param pipeline_inputs: Inputs to the pipeline. :returns: Converted inputs that can be used by the internal `Pipeline.run` logic. """ - inputs = {} + inputs: Dict[str, Dict[str, List[Dict[str, Any]]]] = {} for component_name, socket_dict in pipeline_inputs.items(): inputs[component_name] = {} for socket_name, value in socket_dict.items(): @@ -197,7 +197,7 @@ def _get_component_with_graph_metadata(self, component_name: str) -> Dict[str, A def _get_next_runnable_component( self, priority_queue: FIFOPriorityQueue - ) -> Union[Tuple[Component, str, Dict], None]: + ) -> Union[Tuple[ComponentPriority, str, Dict], None]: """ Returns the next runnable component alongside its metadata from the priority queue. diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 2456ea886a..7afdd261e3 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -4,7 +4,7 @@ import heapq from itertools import count -from typing import Any, Optional, Tuple +from typing import Any, Optional, Tuple, List def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: @@ -36,7 +36,7 @@ def __init__(self) -> None: Initialize a new FIFO priority queue. """ # List of tuples (priority, count, item) where count ensures FIFO order - self._queue = [] + self._queue: List[Tuple[int, int, Any]]= [] # Counter to maintain insertion order for equal priorities self._counter = count() From d2bee247d824859244a10682c463b3a7dd19a6db Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 14 Jan 2025 18:32:53 +0100 Subject: [PATCH 32/88] format --- haystack/core/pipeline/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 7afdd261e3..95d6886a02 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -36,7 +36,7 @@ def __init__(self) -> None: Initialize a new FIFO priority queue. """ # List of tuples (priority, count, item) where count ensures FIFO order - self._queue: List[Tuple[int, int, Any]]= [] + self._queue: List[Tuple[int, int, Any]] = [] # Counter to maintain insertion order for equal priorities self._counter = count() From ad16403c1487ed3d4d4bfd630d2cb2494a3e90f5 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 14 Jan 2025 18:35:11 +0100 Subject: [PATCH 33/88] format --- haystack/core/pipeline/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index 95d6886a02..a9006cf110 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -4,7 +4,7 @@ import heapq from itertools import count -from typing import Any, Optional, Tuple, List +from typing import Any, List, Optional, Tuple def parse_connect_string(connection: str) -> Tuple[str, Optional[str]]: From 2984fcb215a850ce647aa4fe38c4df74aaba986b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 14 Jan 2025 18:45:49 +0100 Subject: [PATCH 34/88] types --- haystack/core/pipeline/pipeline.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index a5ec5a9063..6bcbb2dd07 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -197,7 +197,7 @@ def _get_component_with_graph_metadata(self, component_name: str) -> Dict[str, A def _get_next_runnable_component( self, priority_queue: FIFOPriorityQueue - ) -> Union[Tuple[ComponentPriority, str, Dict], None]: + ) -> Union[Tuple[ComponentPriority, str, Dict[str, Any]], None]: """ Returns the next runnable component alongside its metadata from the priority queue. @@ -206,7 +206,7 @@ def _get_next_runnable_component( or None if no component in the queue can run. :raises: PipelineMaxComponentRuns if the next runnable component has exceeded the maximum number of runs. """ - priority_and_component_name = priority_queue.get() + priority_and_component_name: Union[Tuple[ComponentPriority, str], None] = priority_queue.get() if priority_and_component_name is not None and priority_and_component_name[0] != ComponentPriority.BLOCKED: priority, component_name = priority_and_component_name From 64a01257b980171812e6d27d47571ff684224ca3 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 15 Jan 2025 13:34:05 +0100 Subject: [PATCH 35/88] add defaults from Socket instead of signature - otherwise components with dynamic inputs would fail --- haystack/core/pipeline/pipeline.py | 24 ++++++- .../pipeline/features/pipeline_run.feature | 2 + test/core/pipeline/features/test_run.py | 65 +++++++++++++++++++ test/core/pipeline/test_pipeline.py | 34 ++++++++++ 4 files changed, 124 insertions(+), 1 deletion(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 6bcbb2dd07..2fdb0bc7e3 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -7,7 +7,7 @@ from typing import Any, Dict, List, Mapping, Optional, Set, Tuple, Union from haystack import logging, tracing -from haystack.core.component import Component +from haystack.core.component import Component, InputSocket from haystack.core.errors import PipelineMaxComponentRuns, PipelineRuntimeError from haystack.core.pipeline.base import PipelineBase from haystack.core.pipeline.component_checks import ( @@ -39,6 +39,23 @@ class Pipeline(PipelineBase): Orchestrates component execution according to the execution graph, one after the other. """ + @staticmethod + def _add_missing_input_defaults(component_inputs: Dict[str, Any], component_input_sockets: Dict[str, InputSocket]): + """ + Updates the inputs with the default values for the inputs that are missing + + :param component_inputs: Inputs for the component. + :param component_input_sockets: Input sockets of the component. + """ + for name, socket in component_input_sockets.items(): + if not socket.is_mandatory and name not in component_inputs: + if socket.is_variadic: + component_inputs[name] = [socket.default_value] + else: + component_inputs[name] = socket.default_value + + return component_inputs + def _run_component( self, component: Dict[str, Any], inputs: Dict[str, Any], parent_span: Optional[tracing.Span] = None ) -> Tuple[Dict, Dict]: @@ -57,6 +74,11 @@ def _run_component( component_inputs, inputs = self._consume_component_inputs( component_name=component_name, component=component, inputs=inputs ) + + # We need to add missing defaults using default values from input sockets because the run signature + # might not provide these defaults for components with inputs defined dynamically upon component initialization + component_inputs = self._add_missing_input_defaults(component_inputs, component["input_sockets"]) + with tracing.tracer.trace( "haystack.component.run", tags={ diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index 12ad1513cc..f68ce152c4 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -50,6 +50,8 @@ Feature: Pipeline running | created in a non-standard order that has a loop | | that has an agent with a feedback cycle | | that passes outputs that are consumed in cycle to outside the cycle | + | with a component that has dynamic default inputs | + | with a component that has variadic dynamic default inputs | Scenario Outline: Running a bad Pipeline Given a pipeline diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index fa46ffa2f0..054ff993a5 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -15,6 +15,7 @@ from haystack.components.retrievers.in_memory import InMemoryBM25Retriever from haystack.document_stores.in_memory import InMemoryDocumentStore from haystack.components.joiners import BranchJoiner, DocumentJoiner, AnswerJoiner, StringJoiner +from haystack.core.component.types import Variadic from haystack.testing.sample_components import ( Accumulate, AddFixedValue, @@ -2974,3 +2975,67 @@ def generate_santa_sleigh(): ) ], ) + + +@given("a pipeline with a component that has dynamic default inputs", target_fixture="pipeline_data") +def pipeline_that_is_linear(): + @component + class ParrotWithDynamicDefaultInputs: + def __init__(self, input_variable: str): + self.input_variable = input_variable + component.set_input_type(self, input_variable, str, default="Parrot doesn't only parrot!") + + @component.output_types(response=str) + def run(self, **kwargs): + return {"response": kwargs[self.input_variable]} + + parrot = ParrotWithDynamicDefaultInputs("parrot") + pipeline = Pipeline() + pipeline.add_component("parrot", parrot) + return ( + pipeline, + [ + PipelineRunData( + inputs={"parrot": {"parrot": "Are you a parrot?"}}, + expected_outputs={"parrot": {"response": "Are you a parrot?"}}, + expected_run_order=["parrot"], + ), + PipelineRunData( + inputs={}, + expected_outputs={"parrot": {"response": "Parrot doesn't only parrot!"}}, + expected_run_order=["parrot"], + ), + ], + ) + + +@given("a pipeline with a component that has variadic dynamic default inputs", target_fixture="pipeline_data") +def pipeline_that_is_linear(): + @component + class ParrotWithVariadicDynamicDefaultInputs: + def __init__(self, input_variable: str): + self.input_variable = input_variable + component.set_input_type(self, input_variable, Variadic[str], default="Parrot doesn't only parrot!") + + @component.output_types(response=List[str]) + def run(self, **kwargs): + return {"response": kwargs[self.input_variable]} + + parrot = ParrotWithVariadicDynamicDefaultInputs("parrot") + pipeline = Pipeline() + pipeline.add_component("parrot", parrot) + return ( + pipeline, + [ + PipelineRunData( + inputs={"parrot": {"parrot": "Are you a parrot?"}}, + expected_outputs={"parrot": {"response": ["Are you a parrot?"]}}, + expected_run_order=["parrot"], + ), + PipelineRunData( + inputs={}, + expected_outputs={"parrot": {"response": ["Parrot doesn't only parrot!"]}}, + expected_run_order=["parrot"], + ), + ], + ) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index f14c5594d0..d24b20425e 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1026,6 +1026,40 @@ def test_connect_same_component_as_sender_and_receiver(self): with pytest.raises(PipelineConnectError): pipe.connect("single_component.out", "single_component.in") + @pytest.mark.parametrize( + "component_inputs,sockets,expected_inputs", + [ + ({"mandatory": 1}, {"mandatory": InputSocket("mandatory", int)}, {"mandatory": 1}), + ({}, {"optional": InputSocket("optional", str, default_value="test")}, {"optional": "test"}), + ( + {"mandatory": 1}, + { + "mandatory": InputSocket("mandatory", int), + "optional": InputSocket("optional", str, default_value="test"), + }, + {"mandatory": 1, "optional": "test"}, + ), + ( + {}, + {"optional_variadic": InputSocket("optional_variadic", Variadic[str], default_value="test")}, + {"optional_variadic": ["test"]}, + ), + ( + {}, + { + "optional_1": InputSocket("optional_1", int, default_value=1), + "optional_2": InputSocket("optional_2", int, default_value=2), + }, + {"optional_1": 1, "optional_2": 2}, + ), + ], + ids=["no-defaults", "only-default", "mixed-default", "variadic-default", "multiple_defaults"], + ) + def test__add_missing_defaults(self, component_inputs, sockets, expected_inputs): + filled_inputs = Pipeline._add_missing_input_defaults(component_inputs, sockets) + + assert filled_inputs == expected_inputs + def test__run_component_success(self): """Test successful component execution""" joiner_1 = BranchJoiner(type_=str) From b2b8adc7b0a1b95b0e60ff94d749e5d70f340f94 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 15 Jan 2025 13:36:49 +0100 Subject: [PATCH 36/88] fix test names --- test/core/pipeline/features/test_run.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 054ff993a5..cdf1efb369 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -2978,7 +2978,7 @@ def generate_santa_sleigh(): @given("a pipeline with a component that has dynamic default inputs", target_fixture="pipeline_data") -def pipeline_that_is_linear(): +def pipeline_with_dynamic_defaults(): @component class ParrotWithDynamicDefaultInputs: def __init__(self, input_variable: str): @@ -3010,7 +3010,7 @@ def run(self, **kwargs): @given("a pipeline with a component that has variadic dynamic default inputs", target_fixture="pipeline_data") -def pipeline_that_is_linear(): +def pipeline_with_variadic_dynamic_defaults(): @component class ParrotWithVariadicDynamicDefaultInputs: def __init__(self, input_variable: str): From 6b2582533a86f8fd3f43813868f900151f81c753 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 15 Jan 2025 15:25:21 +0100 Subject: [PATCH 37/88] still wait for optional inputs on greedy variadic sockets - mirrors previous behavior --- haystack/core/pipeline/component_checks.py | 47 +++++++++++-------- haystack/core/pipeline/pipeline.py | 3 +- test/core/pipeline/test_component_checks.py | 50 ++++++++++++++------- test/core/pipeline/test_pipeline.py | 16 +++++++ 4 files changed, 81 insertions(+), 35 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index 30aaa153cb..ee6bf15deb 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -19,7 +19,7 @@ def can_component_run(component: Dict, inputs: Dict) -> bool: :param component: Component metadata and the component instance. :param inputs: Inputs for the component. """ - received_all_mandatory_inputs = are_all_mandatory_sockets_ready(component, inputs) + received_all_mandatory_inputs = are_all_sockets_ready(component, inputs, only_check_mandatory=True) received_trigger = has_any_trigger(component, inputs) return received_all_mandatory_inputs and received_trigger @@ -49,27 +49,38 @@ def has_any_trigger(component: Dict, inputs: Dict) -> bool: return trigger_from_predecessor or trigger_from_user or trigger_without_inputs -def are_all_mandatory_sockets_ready(component: Dict, inputs: Dict) -> bool: +def are_all_sockets_ready(component: Dict, inputs: Dict, only_check_mandatory: bool = False) -> bool: """ - Checks if all mandatory sockets of a component have enough inputs for the component to execute. + Checks if all sockets of a component have enough inputs for the component to execute. :param component: Component metadata and the component instance. :param inputs: Inputs for the component. - """ - filled_mandatory_sockets = set() - expected_mandatory_sockets = set() - for socket_name, socket in component["input_sockets"].items(): - if socket.is_mandatory: - socket_inputs = inputs.get(socket_name, []) - expected_mandatory_sockets.add(socket_name) - if ( - is_socket_lazy_variadic(socket) - and any_socket_input_received(socket_inputs) - or has_socket_received_all_inputs(socket, socket_inputs) - ): - filled_mandatory_sockets.add(socket_name) - - return filled_mandatory_sockets == expected_mandatory_sockets + :param only_check_mandatory: If only mandatory sockets should be checked. + """ + filled_sockets = set() + expected_sockets = set() + if only_check_mandatory: + sockets_to_check = { + socket_name: socket for socket_name, socket in component["input_sockets"].items() if socket.is_mandatory + } + else: + sockets_to_check = { + socket_name: socket + for socket_name, socket in component["input_sockets"].items() + if socket.is_mandatory or len(socket.senders) + } + + for socket_name, socket in sockets_to_check.items(): + socket_inputs = inputs.get(socket_name, []) + expected_sockets.add(socket_name) + if ( + is_socket_lazy_variadic(socket) + and any_socket_input_received(socket_inputs) + or has_socket_received_all_inputs(socket, socket_inputs) + ): + filled_sockets.add(socket_name) + + return filled_sockets == expected_sockets def any_predecessors_provided_input(component: Dict, inputs: Dict) -> bool: diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 2fdb0bc7e3..fc44f40451 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -13,6 +13,7 @@ from haystack.core.pipeline.component_checks import ( _NO_OUTPUT_PRODUCED, all_predecessors_executed, + are_all_sockets_ready, are_all_lazy_variadic_sockets_resolved, can_component_run, is_any_greedy_socket_ready, @@ -205,7 +206,7 @@ def _calculate_priority(component: Dict, inputs: Dict) -> ComponentPriority: """ if not can_component_run(component, inputs): return ComponentPriority.BLOCKED - elif is_any_greedy_socket_ready(component, inputs): + elif is_any_greedy_socket_ready(component, inputs) and are_all_sockets_ready(component, inputs): return ComponentPriority.HIGHEST elif all_predecessors_executed(component, inputs): return ComponentPriority.READY diff --git a/test/core/pipeline/test_component_checks.py b/test/core/pipeline/test_component_checks.py index c6ce9f2ba9..faa74b1eb4 100644 --- a/test/core/pipeline/test_component_checks.py +++ b/test/core/pipeline/test_component_checks.py @@ -13,7 +13,7 @@ def basic_component(): "instance": "mock_instance", "visits": 0, "input_sockets": { - "mandatory_input": InputSocket("mandatory_input", int), + "mandatory_input": InputSocket("mandatory_input", int, senders=["previous_component"]), "optional_input": InputSocket("optional_input", str, default_value="default"), }, "output_sockets": {"output": OutputSocket("output", int)}, @@ -27,8 +27,8 @@ def variadic_component(): "instance": "mock_instance", "visits": 0, "input_sockets": { - "variadic_input": InputSocket("variadic_input", Variadic[int]), - "normal_input": InputSocket("normal_input", str), + "variadic_input": InputSocket("variadic_input", Variadic[int], senders=["previous_component"]), + "normal_input": InputSocket("normal_input", str, senders=["another_component"]), }, "output_sockets": {"output": OutputSocket("output", int)}, } @@ -41,7 +41,9 @@ def greedy_variadic_component(): "instance": "mock_instance", "visits": 0, "input_sockets": { - "greedy_input": InputSocket("greedy_input", GreedyVariadic[int]), + "greedy_input": InputSocket( + "greedy_input", GreedyVariadic[int], senders=["previous_component", "other_component"] + ), "normal_input": InputSocket("normal_input", str), }, "output_sockets": {"output": OutputSocket("output", int)}, @@ -194,12 +196,12 @@ class TestAllMandatorySocketsReady: def test_all_mandatory_sockets_filled(self, basic_component): """Checks that all mandatory sockets are ready when they have valid input.""" inputs = {"mandatory_input": [{"sender": "previous_component", "value": 42}]} - assert are_all_mandatory_sockets_ready(basic_component, inputs) is True + assert are_all_sockets_ready(basic_component, inputs) is True def test_missing_mandatory_socket(self, basic_component): """Ensures that if a mandatory socket is missing, the component is not ready.""" inputs = {"optional_input": [{"sender": "previous_component", "value": "test"}]} - assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + assert are_all_sockets_ready(basic_component, inputs) is False def test_variadic_socket_with_input(self, variadic_component): """Verifies that a variadic socket is considered filled if it has at least one input.""" @@ -207,25 +209,41 @@ def test_variadic_socket_with_input(self, variadic_component): "variadic_input": [{"sender": "previous_component", "value": 42}], "normal_input": [{"sender": "previous_component", "value": "test"}], } - assert are_all_mandatory_sockets_ready(variadic_component, inputs) is True + assert are_all_sockets_ready(variadic_component, inputs) is True - def test_greedy_variadic_socket_with_partial_input(self, greedy_variadic_component): + def test_greedy_variadic_socket(self, greedy_variadic_component): """Greedy variadic sockets are ready with at least one valid input.""" inputs = { "greedy_input": [{"sender": "previous_component", "value": 42}], "normal_input": [{"sender": "previous_component", "value": "test"}], } - assert are_all_mandatory_sockets_ready(greedy_variadic_component, inputs) is True + assert are_all_sockets_ready(greedy_variadic_component, inputs) is True + + def test_greedy_variadic_socket_and_missing_mandatory(self, greedy_variadic_component): + """All mandatory sockets need to be filled even with GreedyVariadic sockets.""" + inputs = {"greedy_input": [{"sender": "previous_component", "value": 42}]} + assert are_all_sockets_ready(greedy_variadic_component, inputs, only_check_mandatory=True) is False def test_variadic_socket_no_input(self, variadic_component): """A variadic socket is not filled if it has zero valid inputs.""" inputs = {"normal_input": [{"sender": "previous_component", "value": "test"}]} - assert are_all_mandatory_sockets_ready(variadic_component, inputs) is False + assert are_all_sockets_ready(variadic_component, inputs) is False + + def test_mandatory_and_optional_sockets(self): + input_sockets = { + "mandatory": InputSocket("mandatory", str, senders=["previous_component"]), + "optional": InputSocket("optional", str, senders=["previous_component"], default_value="test"), + } + + component = {"input_sockets": input_sockets} + inputs = {"mandatory": [{"sender": "previous_component", "value": "hello"}]} + assert are_all_sockets_ready(component, inputs) is False + assert are_all_sockets_ready(component, inputs, only_check_mandatory=True) is True def test_empty_inputs(self, basic_component): """Checks that if there are no inputs at all, mandatory sockets are not ready.""" inputs = {} - assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + assert are_all_sockets_ready(basic_component, inputs) is False def test_no_mandatory_sockets(self, basic_component): """Ensures that if there are no mandatory sockets, the component is considered ready.""" @@ -234,24 +252,24 @@ def test_no_mandatory_sockets(self, basic_component): "optional_2": InputSocket("optional_2", str, default_value="default2"), } inputs = {} - assert are_all_mandatory_sockets_ready(basic_component, inputs) is True + assert are_all_sockets_ready(basic_component, inputs) is True def test_multiple_mandatory_sockets(self, basic_component): """Checks readiness when multiple mandatory sockets are defined.""" basic_component["input_sockets"] = { - "mandatory_1": InputSocket("mandatory_1", int), - "mandatory_2": InputSocket("mandatory_2", str), + "mandatory_1": InputSocket("mandatory_1", int, senders=["previous_component"]), + "mandatory_2": InputSocket("mandatory_2", str, senders=["some other component"]), "optional": InputSocket("optional", bool, default_value=False), } inputs = { "mandatory_1": [{"sender": "comp1", "value": 42}], "mandatory_2": [{"sender": "comp2", "value": "test"}], } - assert are_all_mandatory_sockets_ready(basic_component, inputs) is True + assert are_all_sockets_ready(basic_component, inputs) is True # Missing one mandatory input inputs = {"mandatory_1": [{"sender": "comp1", "value": 42}], "optional": [{"sender": "comp3", "value": True}]} - assert are_all_mandatory_sockets_ready(basic_component, inputs) is False + assert are_all_sockets_ready(basic_component, inputs) is False class TestPredecessorInputDetection: diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index d24b20425e..70e183b5b9 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1210,6 +1210,22 @@ def test__find_receivers_from(self): ComponentPriority.HIGHEST, "Component should have HIGHEST priority when greedy socket has valid input", ), + # Test case 4: DEFER - Greedy socket ready but optional missing + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "greedy_input": InputSocket("greedy_input", GreedyVariadic[int], senders=["component1"]), + "optional_input": InputSocket( + "optional_input", str, senders=["component2"], default_value="test" + ), + }, + }, + {"greedy_input": [{"sender": "component1", "value": 42}]}, + ComponentPriority.DEFER, + "Component should DEFER when greedy socket has valid input but expected optional input is missing", + ), # Test case 4: READY - All predecessors executed ( { From 2566cb83e1b2d8b5aa9ef1d6dd9fabbbe67f737c Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 15 Jan 2025 15:35:28 +0100 Subject: [PATCH 38/88] fix format --- haystack/core/pipeline/pipeline.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index fc44f40451..e86f3736ab 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -13,8 +13,8 @@ from haystack.core.pipeline.component_checks import ( _NO_OUTPUT_PRODUCED, all_predecessors_executed, - are_all_sockets_ready, are_all_lazy_variadic_sockets_resolved, + are_all_sockets_ready, can_component_run, is_any_greedy_socket_ready, is_socket_lazy_variadic, From ef8e754d909bc4b5bd332a155a7027130e40b3fc Mon Sep 17 00:00:00 2001 From: mathislucka Date: Thu, 16 Jan 2025 11:04:18 +0100 Subject: [PATCH 39/88] wip: warn for ambiguous running order --- haystack/core/pipeline/pipeline.py | 68 +++++++++++++++++++++++++++++- 1 file changed, 67 insertions(+), 1 deletion(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index e86f3736ab..5a2e959a5d 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -1,7 +1,7 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 - +import warnings from copy import deepcopy from enum import IntEnum from typing import Any, Dict, List, Mapping, Optional, Set, Tuple, Union @@ -40,6 +40,72 @@ class Pipeline(PipelineBase): Orchestrates component execution according to the execution graph, one after the other. """ + def _warn_if_ambiguous_intent( + self, inputs: Dict[str, Any], component_names: List[str], receivers: Dict[str, Any] + ) -> None: + """ + Issues warnings if the running order of the pipeline is potentially ambiguous. + + We simulate a full pass through the pipeline where all components produce outputs. + At every step, we check if more than one component is waiting for optional inputs. + If two components wait for optional input with the same priority, the user intention for the execution + order of these components is not clear. + A warning does not mean that the running order must be ambiguous when real data flows through the pipeline. + Depending on the users data and business logic, the running order might still be clear, but we can not check + for this before running the pipeline. + + :param inputs: The inputs to the pipeline. + :param component_names: Names of all components in the pipeline. + :param receivers: The receivers for each component in the pipeline. + """ + inp_cpy = deepcopy(inputs) + remaining_components = set(component_names) + pq = self._fill_queue(component_names, inp_cpy) + + # Pipeline has no components. + if len(pq) == 0: + return + + while True: + candidate = pq.pop() + + # We don't have any components left that could run. + if candidate is None or candidate[0] == ComponentPriority.BLOCKED: + return + + priority, component_name = candidate + + # The queue is empty so the next component can't have the same priority as the current component. + if len(pq) == 0: + return + + # We get the next component and its priority to check if the current component and the next component are + # both waiting for inputs with the same priority. + next_prio, next_name = pq.peek() + if priority in [ComponentPriority.DEFER, ComponentPriority.DEFER_LAST] and next_prio == priority: + msg = ( + f"Ambiguous running order: Components '{component_name}' and '{next_name}' are waiting for " + f"optional inputs at the same time. Component '{component_name}' executes first." + ) + warnings.warn(msg) + + # We simulate output distribution for the current component by filling all its output sockets. + comp_with_metadata = self._get_component_with_graph_metadata(component_name) + component_outputs = { + socket_name: "simulation" for socket_name in comp_with_metadata["output_sockets"].keys() + } + comp_receivers = receivers[component_name] + _, inp_cpy = self._write_component_outputs( + component_name, component_outputs, inp_cpy, comp_receivers, set() + ) + + # We need to remove the component that we just checked so that we don't get into an infinite loop. + remaining_components.remove(component_name) + + # We re-prioritize the queue to capture if any components changed priority after simulating a run for + # the current component. + pq = self._fill_queue(remaining_components, inp_cpy) + @staticmethod def _add_missing_input_defaults(component_inputs: Dict[str, Any], component_input_sockets: Dict[str, InputSocket]): """ From 025245038944156a7a8b17f518b13f3fabb8b28e Mon Sep 17 00:00:00 2001 From: mathislucka Date: Thu, 16 Jan 2025 11:08:36 +0100 Subject: [PATCH 40/88] wip: alternative warning --- haystack/core/pipeline/pipeline.py | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 5a2e959a5d..6eb4f11b10 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -513,7 +513,9 @@ def run( # noqa: PLR0915, PLR0912 }, ) as span: inputs = self._convert_from_legacy_format(pipeline_inputs=data) - + self._warn_if_ambiguous_intent( + inputs=inputs, component_names=ordered_component_names, receivers=cached_receivers + ) priority_queue = self._fill_queue(ordered_component_names, inputs) while True: @@ -521,7 +523,18 @@ def run( # noqa: PLR0915, PLR0912 if candidate is None: break - _, component_name, component = candidate + priority, component_name, component = candidate + if len(priority_queue) > 0: + next_priority, next_name = priority_queue.peek() + + # alternative to _warn_if_ambiguous_intent + if priority in [ComponentPriority.DEFER, ComponentPriority.DEFER_LAST] and next_priority == priority: + msg = ( + f"Ambiguous running order: Components '{component_name}' and '{next_name}' are waiting for " + f"optional inputs at the same time. Component '{component_name}' executes first." + ) + warnings.warn(msg) + component_outputs, inputs = self._run_component(component, inputs, parent_span=span) component_pipeline_outputs, inputs = self._write_component_outputs( component_name=component_name, From 580e4bd2a7a75c8da8293c12932d2390b2d379a4 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Thu, 16 Jan 2025 11:15:13 +0100 Subject: [PATCH 41/88] fix license header --- haystack/core/pipeline/pipeline.py | 1 + 1 file changed, 1 insertion(+) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 6eb4f11b10..d9445a904a 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -1,6 +1,7 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 + import warnings from copy import deepcopy from enum import IntEnum From 9426bbbf93d288f92691db4eb39257ae01a4979c Mon Sep 17 00:00:00 2001 From: mathislucka Date: Mon, 20 Jan 2025 16:25:52 +0100 Subject: [PATCH 42/88] make code more readable Co-authored-by: Amna Mubashar --- haystack/core/pipeline/component_checks.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index ee6bf15deb..b7db563292 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -73,11 +73,12 @@ def are_all_sockets_ready(component: Dict, inputs: Dict, only_check_mandatory: b for socket_name, socket in sockets_to_check.items(): socket_inputs = inputs.get(socket_name, []) expected_sockets.add(socket_name) - if ( - is_socket_lazy_variadic(socket) - and any_socket_input_received(socket_inputs) - or has_socket_received_all_inputs(socket, socket_inputs) - ): + # Check if socket has all required inputs or is a lazy variadic socket with any input + + if ( + has_socket_received_all_inputs(socket, socket_inputs) + or (is_socket_lazy_variadic(socket) and any_socket_input_received(socket_inputs)) +): filled_sockets.add(socket_name) return filled_sockets == expected_sockets From ad1b0aaaa6b9b46c1e8833de9a8289c257d765b5 Mon Sep 17 00:00:00 2001 From: Amna Mubashar Date: Mon, 20 Jan 2025 23:05:31 +0100 Subject: [PATCH 43/88] Introduce content tracing to a behavioral test --- pyproject.toml | 2 +- test/conftest.py | 1 + test/core/pipeline/features/conftest.py | 33 ++++++++++++++++++- .../pipeline/features/pipeline_run.feature | 1 + test/core/pipeline/features/test_run.py | 33 ++++++++++++++++++- test/tracing/utils.py | 8 ++++- 6 files changed, 74 insertions(+), 4 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 258e4e2710..84d2dee0e0 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -186,7 +186,7 @@ include = ["/haystack", "/VERSION.txt"] packages = ["haystack"] [tool.codespell] -ignore-words-list = "ans,astroid,nd,ned,nin,ue,rouge,ist" +ignore-words-list = "ans,astroid,nd,ned,nin,ue,rouge,ist, Claus" quiet-level = 3 skip = "./test,./e2e" diff --git a/test/conftest.py b/test/conftest.py index 513009d234..92d16eb310 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -57,6 +57,7 @@ def urlopen_mock(self, method, url, *args, **kwargs): def spying_tracer() -> Generator[SpyingTracer, None, None]: tracer = SpyingTracer() tracing.enable_tracing(tracer) + tracer.is_content_tracing_enabled = True yield tracer diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index 16c7be35ea..4ecea93f96 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -20,6 +20,7 @@ class PipelineRunData: include_outputs_from: Set[str] = field(default_factory=set) expected_outputs: Dict[str, Any] = field(default_factory=dict) expected_run_order: List[str] = field(default_factory=list) + expected_component_calls: Dict[Tuple[str, int], Dict[str, Any]] = field(default_factory=dict) @dataclass @@ -30,6 +31,7 @@ class _PipelineResult: outputs: Dict[str, Any] run_order: List[str] + component_calls: Dict[Tuple[str, int], Dict[str, Any]] = field(default_factory=dict) @when("I run the Pipeline", target_fixture="pipeline_result") @@ -57,7 +59,30 @@ def run_pipeline( for span in spying_tracer.spans if "haystack.component.name" in span.tags ] - results.append(_PipelineResult(outputs=outputs, run_order=run_order)) + + component_calls = { + (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ + "haystack.component.input" + ] + for span in spying_tracer.spans + if "haystack.component.name" in span.tags and "haystack.component.visits" in span.tags + } + results.append(_PipelineResult(outputs=outputs, run_order=run_order, component_calls=component_calls)) + + print("Debug - Raw span tags:") + for span in spying_tracer.spans: + if "haystack.component.input" in span.tags: + print( + f"Input for {span.tags.get('haystack.component.name')}: {span.tags['haystack.component.input']}" + ) + + component_calls = { + (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ + "haystack.component.input" + ] + for span in spying_tracer.spans + if "haystack.component.name" in span.tags and "haystack.component.visits" in span.tags + } spying_tracer.spans.clear() except Exception as e: return e @@ -83,6 +108,12 @@ def check_pipeline_result(pipeline_result: List[Tuple[_PipelineResult, PipelineR assert res.outputs == data.expected_outputs +@then("components are called with the expected inputs") +def check_component_calls(pipeline_result: List[Tuple[_PipelineResult, PipelineRunData]]): + for res, data in pipeline_result: + assert res.component_calls == data.expected_component_calls + + @then("components ran in the expected order") def check_pipeline_run_order(pipeline_result: List[Tuple[_PipelineResult, PipelineRunData]]): for res, data in pipeline_result: diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index f68ce152c4..cf073050f0 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -5,6 +5,7 @@ Feature: Pipeline running When I run the Pipeline Then it should return the expected result And components ran in the expected order + And components are called with the expected inputs Examples: | kind | diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index cdf1efb369..0a3199b08c 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -1525,10 +1525,36 @@ def run(self, query_embedding: List[float]): ] } }, + expected_component_calls={ + ("router", 1): {"query": "I'm a legit question"}, + ("text_embedder", 1): {"text": "I'm a legit question"}, + ("bm25retriever", 1): {"query": "I'm a legit question"}, + ("retriever", 1): {"query_embedding": [1.0, 2.0, 3.0]}, + ("joinerhybrid", 1): { + "documents": [ + [Document(content="This is a document")], + [Document(content="This is another document")], + ], + "top_k": None, + }, + ("ranker", 1): { + "query": "I'm a legit question", + "documents": [ + Document(content="This is a document"), + Document(content="This is another document"), + ], + }, + ("joinerfinal", 1): { + "documents": [ + [Document(content="This is a document"), Document(content="This is another document")] + ], + "top_k": None, + }, + }, expected_run_order=[ "router", - "text_embedder", "bm25retriever", + "text_embedder", "retriever", "joinerhybrid", "ranker", @@ -1539,6 +1565,11 @@ def run(self, query_embedding: List[float]): inputs={"router": {"query": "I'm a nasty prompt injection"}}, expected_outputs={"joinerfinal": {"documents": []}}, expected_run_order=["router", "emptyretriever", "joinerfinal"], + expected_component_calls={ + ("router", 1): {"query": "I'm a nasty prompt injection"}, + ("emptyretriever", 1): {"query": "I'm a nasty prompt injection"}, + ("joinerfinal", 1): {"documents": [[]], "top_k": None}, + }, ), ], ) diff --git a/test/tracing/utils.py b/test/tracing/utils.py index c2261baacf..8f28e4dae2 100644 --- a/test/tracing/utils.py +++ b/test/tracing/utils.py @@ -24,6 +24,12 @@ def set_tag(self, key: str, value: Any) -> None: def get_correlation_data_for_logs(self) -> Dict[str, Any]: return {"trace_id": self.trace_id, "span_id": self.span_id} + def set_content_tag(self, key: str, value: Any) -> None: + """ + Set a content tag, but only if content tracing is enabled in the tracer. + """ + self.set_tag(key, value) + class SpyingTracer(Tracer): def current_span(self) -> Optional[Span]: @@ -31,13 +37,13 @@ def current_span(self) -> Optional[Span]: def __init__(self) -> None: self.spans: List[SpyingSpan] = [] + self.is_content_tracing_enabled = True @contextlib.contextmanager def trace( self, operation_name: str, tags: Optional[Dict[str, Any]] = None, parent_span: Optional[Span] = None ) -> Iterator[Span]: new_span = SpyingSpan(operation_name, parent_span) - for key, value in (tags or {}).items(): new_span.set_tag(key, value) From a38bccbd121f2a79fa6c479ead40d033900bf9c7 Mon Sep 17 00:00:00 2001 From: Amna Mubashar Date: Mon, 20 Jan 2025 23:14:44 +0100 Subject: [PATCH 44/88] Fixing linting --- haystack/core/pipeline/component_checks.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index b7db563292..ee6bf15deb 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -73,12 +73,11 @@ def are_all_sockets_ready(component: Dict, inputs: Dict, only_check_mandatory: b for socket_name, socket in sockets_to_check.items(): socket_inputs = inputs.get(socket_name, []) expected_sockets.add(socket_name) - # Check if socket has all required inputs or is a lazy variadic socket with any input - - if ( - has_socket_received_all_inputs(socket, socket_inputs) - or (is_socket_lazy_variadic(socket) and any_socket_input_received(socket_inputs)) -): + if ( + is_socket_lazy_variadic(socket) + and any_socket_input_received(socket_inputs) + or has_socket_received_all_inputs(socket, socket_inputs) + ): filled_sockets.add(socket_name) return filled_sockets == expected_sockets From 0ac82b27396d08d5c491246b14410f30c5b6e9b6 Mon Sep 17 00:00:00 2001 From: Amna Mubashar Date: Mon, 20 Jan 2025 23:21:06 +0100 Subject: [PATCH 45/88] Remove debug print statements --- test/core/pipeline/features/conftest.py | 7 ------- 1 file changed, 7 deletions(-) diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index 4ecea93f96..4c1d7d5a1a 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -69,13 +69,6 @@ def run_pipeline( } results.append(_PipelineResult(outputs=outputs, run_order=run_order, component_calls=component_calls)) - print("Debug - Raw span tags:") - for span in spying_tracer.spans: - if "haystack.component.input" in span.tags: - print( - f"Input for {span.tags.get('haystack.component.name')}: {span.tags['haystack.component.input']}" - ) - component_calls = { (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ "haystack.component.input" From 339d340f5b21df5b884fa7c512ce1136152c1f87 Mon Sep 17 00:00:00 2001 From: Amna Mubashar Date: Wed, 22 Jan 2025 15:32:47 +0100 Subject: [PATCH 46/88] Fix tracer tests --- test/core/pipeline/features/test_run.py | 2 +- test/core/pipeline/test_tracing.py | 4 ++++ test/tracing/test_tracer.py | 22 ++++++---------------- test/tracing/utils.py | 3 ++- 4 files changed, 13 insertions(+), 18 deletions(-) diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 0a3199b08c..083ec93edc 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -1553,8 +1553,8 @@ def run(self, query_embedding: List[float]): }, expected_run_order=[ "router", - "bm25retriever", "text_embedder", + "bm25retriever", "retriever", "joinerhybrid", "ranker", diff --git a/test/core/pipeline/test_tracing.py b/test/core/pipeline/test_tracing.py index 6b5493383d..a643d11f41 100644 --- a/test/core/pipeline/test_tracing.py +++ b/test/core/pipeline/test_tracing.py @@ -58,7 +58,9 @@ def test_with_enabled_tracing(self, pipeline: Pipeline, spying_tracer: SpyingTra "haystack.component.type": "Hello", "haystack.component.input_types": {"word": "str"}, "haystack.component.input_spec": {"word": {"type": ANY, "senders": []}}, + "haystack.component.input": {"word": "world"}, "haystack.component.output_spec": {"output": {"type": "str", "receivers": ["hello2"]}}, + "haystack.component.output": {"output": "Hello, world!"}, "haystack.component.visits": 1, }, parent_span=pipeline_span, @@ -72,7 +74,9 @@ def test_with_enabled_tracing(self, pipeline: Pipeline, spying_tracer: SpyingTra "haystack.component.type": "Hello", "haystack.component.input_types": {"word": "str"}, "haystack.component.input_spec": {"word": {"type": ANY, "senders": ["hello"]}}, + "haystack.component.input": {"word": "Hello, world!"}, "haystack.component.output_spec": {"output": {"type": "str", "receivers": []}}, + "haystack.component.output": {"output": "Hello, Hello, world!!"}, "haystack.component.visits": 1, }, parent_span=pipeline_span, diff --git a/test/tracing/test_tracer.py b/test/tracing/test_tracer.py index 3ef63c7b26..2589bd65ef 100644 --- a/test/tracing/test_tracer.py +++ b/test/tracing/test_tracer.py @@ -153,21 +153,10 @@ def test__auto_configured_datadog_tracer_with_failing_import(self, monkeypatch): class TestTracingContent: - def test_set_content_tag_with_default_settings(self, spying_tracer: SpyingTracer) -> None: - with tracer.trace("test") as span: - span.set_content_tag("my_content", "my_content") + def test_set_content_tag_with_enabled_content_tracing(self, spying_tracer: SpyingTracer) -> None: + # SpyingTracer supports content tracing by default - assert len(spying_tracer.spans) == 1 - span = spying_tracer.spans[0] - assert span.tags == {} - - def test_set_content_tag_with_enabled_content_tracing( - self, monkeypatch: MonkeyPatch, spying_tracer: SpyingTracer - ) -> None: enable_tracing(spying_tracer) - # monkeypatch to avoid impact on other tests - monkeypatch.setattr(tracer, "is_content_tracing_enabled", True) - with tracer.trace("test") as span: span.set_content_tag("my_content", "my_content") @@ -175,9 +164,10 @@ def test_set_content_tag_with_enabled_content_tracing( span = spying_tracer.spans[0] assert span.tags == {"my_content": "my_content"} - def test_set_content_tag_when_enabled_via_env_variable(self, monkeypatch: MonkeyPatch) -> None: - monkeypatch.setenv(HAYSTACK_CONTENT_TRACING_ENABLED_ENV_VAR, "true") + def test_set_content_tag_when_disabled_via_env_variable(self, monkeypatch: MonkeyPatch) -> None: + # we test if content tracing is disabled when the env variable is set to false + monkeypatch.setenv(HAYSTACK_CONTENT_TRACING_ENABLED_ENV_VAR, "false") proxy_tracer = ProxyTracer(provided_tracer=SpyingTracer()) - assert proxy_tracer.is_content_tracing_enabled is True + assert proxy_tracer.is_content_tracing_enabled is False diff --git a/test/tracing/utils.py b/test/tracing/utils.py index 8f28e4dae2..f399fc99ff 100644 --- a/test/tracing/utils.py +++ b/test/tracing/utils.py @@ -13,6 +13,7 @@ class SpyingSpan(Span): operation_name: str parent_span: Optional[Span] = None + tags: Dict[str, Any] = dataclasses.field(default_factory=dict) trace_id: Optional[str] = dataclasses.field(default_factory=lambda: str(uuid.uuid4())) @@ -37,7 +38,6 @@ def current_span(self) -> Optional[Span]: def __init__(self) -> None: self.spans: List[SpyingSpan] = [] - self.is_content_tracing_enabled = True @contextlib.contextmanager def trace( @@ -48,5 +48,6 @@ def trace( new_span.set_tag(key, value) self.spans.append(new_span) + print(self.spans) yield new_span From ae295f74137de706a60f70490a34a4f36a867ce3 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 12:19:01 +0100 Subject: [PATCH 47/88] remove print --- test/tracing/utils.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/tracing/utils.py b/test/tracing/utils.py index f399fc99ff..198fd7089e 100644 --- a/test/tracing/utils.py +++ b/test/tracing/utils.py @@ -48,6 +48,5 @@ def trace( new_span.set_tag(key, value) self.spans.append(new_span) - print(self.spans) yield new_span From 592492e60d551ca920e521b9184bf2e0fe106ef3 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 12:19:33 +0100 Subject: [PATCH 48/88] test: test for component inputs --- .../pipeline/features/pipeline_run.feature | 1 - test/core/pipeline/features/test_run.py | 2331 ++++++++++++++++- 2 files changed, 2265 insertions(+), 67 deletions(-) diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index cf073050f0..a4b7fdce01 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -25,7 +25,6 @@ Feature: Pipeline running | that has a component with mutable input | | that has a component with mutable output sent to multiple inputs | | that has a greedy and variadic component after a component with default input | - | that has components added in a different order from the order of execution | | that has a component with only default inputs | | that has a component with only default inputs as first to run and receives inputs from a loop | | that has multiple branches that merge into a component with a single variadic input | diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 083ec93edc..e67f6c2050 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -7,7 +7,7 @@ from haystack import Pipeline, Document, component from haystack.document_stores.types import DuplicatePolicy -from haystack.dataclasses import ChatMessage, GeneratedAnswer +from haystack.dataclasses import ChatMessage, GeneratedAnswer, TextContent from haystack.components.routers import ConditionalRouter from haystack.components.builders import PromptBuilder, AnswerBuilder, ChatPromptBuilder from haystack.components.converters.output_adapter import OutputAdapter @@ -64,6 +64,11 @@ def pipeline_that_is_linear(): PipelineRunData( inputs={"first_addition": {"value": 1}}, expected_outputs={"second_addition": {"result": 7}}, + expected_component_calls={ + ("first_addition", 1): {"value": 1, "add": None}, + ("double", 1): {"value": 3}, + ("second_addition", 1): {"value": 6, "add": None}, + }, expected_run_order=["first_addition", "double", "second_addition"], ) ], @@ -156,6 +161,32 @@ def pipeline_complex(): PipelineRunData( inputs={"greet_first": {"value": 1}, "greet_enumerator": {"value": 1}}, expected_outputs={"accumulate_3": {"value": -7}, "add_five": {"result": -6}}, + expected_component_calls={ + ("greet_first", 1): {"value": 1, "log_level": None, "message": None}, + ("greet_enumerator", 1): {"value": 1, "log_level": None, "message": None}, + ("accumulate_1", 1): {"value": 1}, + ("add_two", 1): {"value": 1, "add": None}, + ("parity", 1): {"value": 3}, + ("add_one", 1): {"value": 3, "add": None}, + ("branch_joiner", 1): {"value": [4]}, + ("below_10", 1): {"value": 4, "threshold": None}, + ("double", 1): {"value": 4}, + ("branch_joiner", 2): {"value": [8]}, + ("below_10", 2): {"value": 8, "threshold": None}, + ("double", 2): {"value": 8}, + ("branch_joiner", 3): {"value": [16]}, + ("below_10", 3): {"value": 16, "threshold": None}, + ("accumulate_2", 1): {"value": 16}, + ("enumerate", 1): {"value": 1}, + ("add_three", 1): {"value": 1, "add": None}, + ("sum", 1): {"values": [1, 4]}, + ("diff", 1): {"first_value": 5, "second_value": 16}, + ("greet_one_last_time", 1): {"value": -11, "log_level": None, "message": None}, + ("replicate", 1): {"value": -11}, + ("add_five", 1): {"value": -11, "add": None}, + ("add_four", 1): {"value": -11, "add": None}, + ("accumulate_3", 1): {"value": -7}, + }, expected_run_order=[ "greet_first", "greet_enumerator", @@ -204,11 +235,13 @@ def run(self, a: int, b: int = 2): PipelineRunData( inputs={"with_defaults": {"a": 40, "b": 30}}, expected_outputs={"with_defaults": {"c": 70}}, + expected_component_calls={("with_defaults", 1): {"a": 40, "b": 30}}, expected_run_order=["with_defaults"], ), PipelineRunData( inputs={"with_defaults": {"a": 40}}, expected_outputs={"with_defaults": {"c": 42}}, + expected_component_calls={("with_defaults", 1): {"a": 40, "b": 2}}, expected_run_order=["with_defaults"], ), ], @@ -234,26 +267,43 @@ def pipeline_that_has_two_loops_of_identical_lengths(): PipelineRunData( inputs={"branch_joiner": {"value": 0}}, expected_outputs={"remainder": {"remainder_is_0": 0}}, + expected_component_calls={("branch_joiner", 1): {"value": [0]}, ("remainder", 1): {"value": 0}}, expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 3}}, expected_outputs={"remainder": {"remainder_is_0": 3}}, + expected_component_calls={("branch_joiner", 1): {"value": [3]}, ("remainder", 1): {"value": 3}}, expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 4}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, + expected_component_calls={ + ("branch_joiner", 1): {"value": [4]}, + ("remainder", 1): {"value": 4}, + ("add_two", 1): {"value": 4, "add": None}, + ("branch_joiner", 2): {"value": [6]}, + ("remainder", 2): {"value": 6}, + }, expected_run_order=["branch_joiner", "remainder", "add_two", "branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 5}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, + expected_component_calls={ + ("branch_joiner", 1): {"value": [5]}, + ("remainder", 1): {"value": 5}, + ("add_one", 1): {"value": 5, "add": None}, + ("branch_joiner", 2): {"value": [6]}, + ("remainder", 2): {"value": 6}, + }, expected_run_order=["branch_joiner", "remainder", "add_one", "branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 6}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, + expected_component_calls={("branch_joiner", 1): {"value": [6]}, ("remainder", 1): {"value": 6}}, expected_run_order=["branch_joiner", "remainder"], ), ], @@ -282,16 +332,26 @@ def pipeline_that_has_two_loops_of_different_lengths(): PipelineRunData( inputs={"branch_joiner": {"value": 0}}, expected_outputs={"remainder": {"remainder_is_0": 0}}, + expected_component_calls={("branch_joiner", 1): {"value": [0]}, ("remainder", 1): {"value": 0}}, expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 3}}, expected_outputs={"remainder": {"remainder_is_0": 3}}, + expected_component_calls={("branch_joiner", 1): {"value": [3]}, ("remainder", 1): {"value": 3}}, expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 4}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, + expected_component_calls={ + ("branch_joiner", 1): {"value": [4]}, + ("remainder", 1): {"value": 4}, + ("add_two_1", 1): {"value": 4, "add": None}, + ("add_two_2", 1): {"value": 5, "add": None}, + ("branch_joiner", 2): {"value": [6]}, + ("remainder", 2): {"value": 6}, + }, expected_run_order=[ "branch_joiner", "remainder", @@ -304,11 +364,19 @@ def pipeline_that_has_two_loops_of_different_lengths(): PipelineRunData( inputs={"branch_joiner": {"value": 5}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, + expected_component_calls={ + ("branch_joiner", 1): {"value": [5]}, + ("remainder", 1): {"value": 5}, + ("add_one", 1): {"value": 5, "add": None}, + ("branch_joiner", 2): {"value": [6]}, + ("remainder", 2): {"value": 6}, + }, expected_run_order=["branch_joiner", "remainder", "add_one", "branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 6}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, + expected_component_calls={("branch_joiner", 1): {"value": [6]}, ("remainder", 1): {"value": 6}}, expected_run_order=["branch_joiner", "remainder"], ), ], @@ -343,6 +411,21 @@ def pipeline_that_has_a_single_loop_with_two_conditional_branches(): PipelineRunData( inputs={"add_one": {"value": 3}}, expected_outputs={"add_two": {"result": 13}}, + expected_component_calls={ + ("accumulator", 1): {"value": 4}, + ("accumulator", 2): {"value": 4}, + ("add_one", 1): {"add": None, "value": 3}, + ("add_three", 1): {"add": None, "value": 8}, + ("add_two", 1): {"add": None, "value": 11}, + ("below_10", 1): {"threshold": None, "value": 4}, + ("below_10", 2): {"threshold": None, "value": 4}, + ("below_10", 3): {"threshold": None, "value": 11}, + ("below_5", 1): {"threshold": None, "value": 4}, + ("below_5", 2): {"threshold": None, "value": 8}, + ("branch_joiner", 1): {"value": [4]}, + ("branch_joiner", 2): {"value": [4]}, + ("branch_joiner", 3): {"value": [11]}, + }, expected_run_order=[ "add_one", "branch_joiner", @@ -378,11 +461,21 @@ def pipeline_that_has_a_component_with_dynamic_inputs_defined_in_init(): PipelineRunData( inputs={"hello": {"word": "Alice"}}, expected_outputs={"splitter": {"output": ["This", "is", "the", "greeting:", "Hello,", "Alice!!"]}}, + expected_component_calls={ + ("fstring", 1): {"greeting": "Hello, Alice!", "template": None}, + ("hello", 1): {"word": "Alice"}, + ("splitter", 1): {"sentence": "This is the greeting: Hello, Alice!!"}, + }, expected_run_order=["hello", "fstring", "splitter"], ), PipelineRunData( inputs={"hello": {"word": "Alice"}, "fstring": {"template": "Received: {greeting}"}}, expected_outputs={"splitter": {"output": ["Received:", "Hello,", "Alice!"]}}, + expected_component_calls={ + ("fstring", 1): {"greeting": "Hello, Alice!", "template": "Received: {greeting}"}, + ("hello", 1): {"word": "Alice"}, + ("splitter", 1): {"sentence": "Received: Hello, Alice!"}, + }, expected_run_order=["hello", "fstring", "splitter"], ), ], @@ -409,11 +502,22 @@ def pipeline_that_has_two_branches_that_dont_merge(): PipelineRunData( inputs={"add_one": {"value": 1}}, expected_outputs={"add_three": {"result": 15}}, + expected_component_calls={ + ("add_one", 1): {"add": None, "value": 1}, + ("add_ten", 1): {"add": None, "value": 2}, + ("add_three", 1): {"add": None, "value": 12}, + ("parity", 1): {"value": 2}, + }, expected_run_order=["add_one", "parity", "add_ten", "add_three"], ), PipelineRunData( inputs={"add_one": {"value": 2}}, expected_outputs={"double": {"value": 6}}, + expected_component_calls={ + ("add_one", 1): {"add": None, "value": 2}, + ("double", 1): {"value": 3}, + ("parity", 1): {"value": 3}, + }, expected_run_order=["add_one", "parity", "double"], ), ], @@ -442,6 +546,14 @@ def pipeline_that_has_three_branches_that_dont_merge(): PipelineRunData( inputs={"add_one": {"value": 1}}, expected_outputs={"add_one_again": {"result": 6}, "add_ten": {"result": 12}, "double": {"value": 4}}, + expected_component_calls={ + ("add_one", 1): {"add": None, "value": 1}, + ("add_one_again", 1): {"add": None, "value": 5}, + ("add_ten", 1): {"add": None, "value": 2}, + ("add_three", 1): {"add": None, "value": 2}, + ("double", 1): {"value": 2}, + ("repeat", 1): {"value": 2}, + }, expected_run_order=["add_one", "repeat", "add_ten", "double", "add_three", "add_one_again"], ) ], @@ -467,6 +579,13 @@ def pipeline_that_has_two_branches_that_merge(): PipelineRunData( inputs={"first_addition": {"value": 1}, "third_addition": {"value": 1}}, expected_outputs={"fourth_addition": {"result": 3}}, + expected_component_calls={ + ("diff", 1): {"first_value": 5, "second_value": 3}, + ("first_addition", 1): {"add": None, "value": 1}, + ("fourth_addition", 1): {"add": None, "value": 2}, + ("second_addition", 1): {"add": None, "value": 3}, + ("third_addition", 1): {"add": None, "value": 1}, + }, expected_run_order=["first_addition", "third_addition", "second_addition", "diff", "fourth_addition"], ) ], @@ -502,11 +621,25 @@ def pipeline_that_has_different_combinations_of_branches_that_merge_and_do_not_m PipelineRunData( inputs={"add_one": {"value": 1}, "add_two": {"add": 2}, "add_two_as_well": {"add": 2}}, expected_outputs={"add_two": {"result": 8}, "add_two_as_well": {"result": 8}}, + expected_component_calls={ + ("add_four", 1): {"add": None, "value": 2}, + ("add_one", 1): {"add": None, "value": 1}, + ("add_two", 1): {"add": 2, "value": 6}, + ("add_two_as_well", 1): {"add": 2, "value": 6}, + ("parity", 1): {"value": 2}, + }, expected_run_order=["add_one", "parity", "add_four", "add_two", "add_two_as_well"], ), PipelineRunData( inputs={"add_one": {"value": 2}, "add_two": {"add": 2}, "add_two_as_well": {"add": 2}}, expected_outputs={"diff": {"difference": 7}}, + expected_component_calls={ + ("add_one", 1): {"add": None, "value": 2}, + ("add_ten", 1): {"add": None, "value": 3}, + ("diff", 1): {"first_value": 13, "second_value": 6}, + ("double", 1): {"value": 3}, + ("parity", 1): {"value": 3}, + }, expected_run_order=["add_one", "parity", "double", "add_ten", "diff"], ), ], @@ -538,6 +671,21 @@ def pipeline_that_has_two_branches_one_of_which_loops_back(): PipelineRunData( inputs={"add_zero": {"value": 8}, "sum": {"values": 2}}, expected_outputs={"sum": {"total": 23}}, + expected_component_calls={ + ("add_one", 1): {"add": None, "value": 8}, + ("add_one", 2): {"add": None, "value": 9}, + ("add_two", 1): {"add": None, "value": 19}, + ("add_zero", 1): {"add": None, "value": 8}, + ("below_10", 1): {"threshold": None, "value": 8}, + ("below_10", 2): {"threshold": None, "value": 9}, + ("below_10", 3): {"threshold": None, "value": 19}, + ("branch_joiner", 1): {"value": [8]}, + ("branch_joiner", 2): {"value": [9]}, + ("branch_joiner", 3): {"value": [19]}, + ("counter", 1): {"value": 9}, + ("counter", 2): {"value": 10}, + ("sum", 1): {"values": [2, 21]}, + }, expected_run_order=[ "add_zero", "branch_joiner", @@ -586,6 +734,12 @@ def run(self, input_list: List[str]): "concat1": {"output": ["foo", "bar", "extra_item"]}, "concat2": {"output": ["foo", "bar", "extra_item"]}, }, + expected_component_calls={ + ("concat1", 1): {"inputs": [["foo", "bar", "extra_item"]]}, + ("concat2", 1): {"inputs": [["foo", "bar", "extra_item"]]}, + ("mangler1", 1): {"input_list": ["foo", "bar"]}, + ("mangler2", 1): {"input_list": ["foo", "bar"]}, + }, expected_run_order=["mangler1", "mangler2", "concat1", "concat2"], ) ], @@ -652,6 +806,68 @@ def run(self, messages: List[ChatMessage]): }, "mm2": {"merged_message": "Fake message"}, }, + expected_component_calls={ + ("llm", 1): { + "messages": [ + ChatMessage( + _role="system", + _content=[ + TextContent( + text="Always respond in English even if some input data is in other languages." + ) + ], + _name=None, + _meta={}, + ), + ChatMessage( + _role="user", _content=[TextContent(text="Tell me about Berlin")], _name=None, _meta={} + ), + ] + }, + ("mm1", 1): { + "messages": [ + ChatMessage( + _role="system", + _content=[ + TextContent( + text="Always respond in English even if some input data is in other languages." + ) + ], + _name=None, + _meta={}, + ), + ChatMessage( + _role="user", _content=[TextContent(text="Tell me about Berlin")], _name=None, _meta={} + ), + ], + "metadata": {"meta2": "value2", "metadata_key": "metadata_value"}, + }, + ("mm2", 1): { + "messages": [ + ChatMessage( + _role="assistant", _content=[TextContent(text="Fake message")], _name=None, _meta={} + ) + ], + "metadata": {"meta2": "value2", "metadata_key": "metadata_value"}, + }, + ("prompt_builder", 1): { + "prompt_source": [ + ChatMessage( + _role="system", + _content=[ + TextContent( + text="Always respond in English even if some input data is in other languages." + ) + ], + _name=None, + _meta={}, + ), + ChatMessage( + _role="user", _content=[TextContent(text="Tell me about Berlin")], _name=None, _meta={} + ), + ] + }, + }, expected_run_order=["prompt_builder", "llm", "mm1", "mm2"], ) ], @@ -701,6 +917,37 @@ def pipeline_that_has_a_greedy_and_variadic_component_after_a_component_with_def "question" } }, + expected_component_calls={ + ("branch_joiner", 1): { + "value": [ + [ + Document( + id="328f0cbb6722c5cfa290aa2b78bcda8dc5afa09f0e2c23092afc502ba89c85e7", + content="This is a simple document", + score=0.5993376509412102, + ) + ] + ] + }, + ("prompt_builder", 1): { + "documents": [ + Document( + id="328f0cbb6722c5cfa290aa2b78bcda8dc5afa09f0e2c23092afc502ba89c85e7", + content="This is a simple document", + score=0.5993376509412102, + ) + ], + "query": "This is my question", + "template": None, + "template_variables": None, + }, + ("retriever", 1): { + "filters": None, + "query": "This is my question", + "scale_score": None, + "top_k": None, + }, + }, expected_run_order=["retriever", "branch_joiner", "prompt_builder"], ) ], @@ -721,71 +968,6 @@ def pipeline_that_has_a_component_that_doesnt_return_a_dictionary(): return pipe, [PipelineRunData({"comp": {"a": 1}})] -@given( - "a pipeline that has components added in a different order from the order of execution", - target_fixture="pipeline_data", -) -def pipeline_that_has_components_added_in_a_different_order_from_the_order_of_execution(): - """ - We enqueue the Components in internal `to_run` data structure at the start of `Pipeline.run()` using the order - they are added in the Pipeline with `Pipeline.add_component()`. - If a Component A with defaults is added before a Component B that has no defaults, but in the Pipeline - logic A must be executed after B it could run instead before. - - This test verifies that the order of execution is correct. - """ - docs = [Document(content="Rome is the capital of Italy"), Document(content="Paris is the capital of France")] - doc_store = InMemoryDocumentStore() - doc_store.write_documents(docs) - template = ( - "Given the following information, answer the question.\n" - "Context:\n" - "{% for document in documents %}" - " {{ document.content }}\n" - "{% endfor %}" - "Question: {{ query }}" - ) - - pipe = Pipeline(max_runs_per_component=1) - - # The order of this addition is important for the test - # Do not edit them. - pipe.add_component("prompt_builder", PromptBuilder(template=template)) - pipe.add_component("retriever", InMemoryBM25Retriever(document_store=doc_store)) - pipe.connect("retriever", "prompt_builder.documents") - - query = "What is the capital of France?" - return ( - pipe, - [ - PipelineRunData( - inputs={"prompt_builder": {"query": query}, "retriever": {"query": query}}, - expected_outputs={ - "prompt_builder": { - "prompt": "Given the " - "following " - "information, " - "answer the " - "question.\n" - "Context:\n" - " Paris is " - "the capital " - "of France\n" - " Rome is " - "the capital " - "of Italy\n" - "Question: " - "What is the " - "capital of " - "France?" - } - }, - expected_run_order=["retriever", "prompt_builder"], - ) - ], - ) - - @given("a pipeline that has a component with only default inputs", target_fixture="pipeline_data") def pipeline_that_has_a_component_with_only_default_inputs(): FakeGenerator = component_class( @@ -843,6 +1025,58 @@ def pipeline_that_has_a_component_with_only_default_inputs(): ] } }, + expected_component_calls={ + ("answer_builder", 1): { + "documents": [ + Document( + id="413dccdf51a54cca75b7ed2eddac04e6e58560bd2f0caf4106a3efc023fe3651", + content="Paris is the capital of France", + score=1.600237583702734, + ), + Document( + id="a4a874fc2ef75015da7924d709fbdd2430e46a8e94add6e0f26cd32c1c03435d", + content="Rome is the capital of Italy", + score=1.2536639934227616, + ), + ], + "meta": None, + "pattern": None, + "query": "What is the capital of France?", + "reference_pattern": None, + "replies": ["Paris"], + }, + ("generator", 1): { + "prompt": "Given the following information, answer the " + "question.\n" + "Context:\n" + " Paris is the capital of France\n" + " Rome is the capital of Italy\n" + "Question: What is the capital of France?" + }, + ("prompt_builder", 1): { + "documents": [ + Document( + id="413dccdf51a54cca75b7ed2eddac04e6e58560bd2f0caf4106a3efc023fe3651", + content="Paris is the capital of France", + score=1.600237583702734, + ), + Document( + id="a4a874fc2ef75015da7924d709fbdd2430e46a8e94add6e0f26cd32c1c03435d", + content="Rome is the capital of Italy", + score=1.2536639934227616, + ), + ], + "query": "What is the capital of France?", + "template": None, + "template_variables": None, + }, + ("retriever", 1): { + "filters": None, + "query": "What is the capital of France?", + "scale_score": None, + "top_k": None, + }, + }, expected_run_order=["retriever", "prompt_builder", "generator", "answer_builder"], ) ], @@ -920,6 +1154,37 @@ def fake_generator_run(self, generation_kwargs: Optional[Dict[str, Any]] = None, PipelineRunData( inputs={"prompt_builder": {"query": "What is the capital of Italy?"}}, expected_outputs={"router": {"correct_replies": ["Rome"]}}, + expected_component_calls={ + ("generator", 1): { + "generation_kwargs": None, + "prompt": "Answer the following question.\n\nQuestion: What is the capital of Italy?", + }, + ("generator", 2): { + "generation_kwargs": None, + "prompt": "Answer the following question.\n" + "\n" + "Previously you replied incorrectly this:\n" + "\n" + " - Paris\n" + "\n" + "\n" + "Question: What is the capital of Italy?", + }, + ("prompt_builder", 1): { + "previous_replies": "", + "query": "What is the capital of Italy?", + "template": None, + "template_variables": None, + }, + ("prompt_builder", 2): { + "previous_replies": ["Paris"], + "query": "What is the capital of Italy?", + "template": None, + "template_variables": None, + }, + ("router", 1): {"replies": ["Paris"]}, + ("router", 2): {"replies": ["Rome"]}, + }, expected_run_order=["prompt_builder", "generator", "router", "prompt_builder", "generator", "router"], ) ], @@ -956,11 +1221,25 @@ def pipeline_that_has_multiple_branches_that_merge_into_a_component_with_a_singl PipelineRunData( inputs={"add_one": {"value": 1}}, expected_outputs={"sum": {"total": 14}}, + expected_component_calls={ + ("add_one", 1): {"add": None, "value": 1}, + ("add_ten", 1): {"add": None, "value": 2}, + ("parity", 1): {"value": 2}, + ("sum", 1): {"values": [2, 12]}, + }, expected_run_order=["add_one", "parity", "add_ten", "sum"], ), PipelineRunData( inputs={"add_one": {"value": 2}}, expected_outputs={"sum": {"total": 17}}, + expected_component_calls={ + ("add_four", 1): {"add": None, "value": 3}, + ("add_one", 1): {"add": None, "value": 2}, + ("add_one_again", 1): {"add": None, "value": 7}, + ("double", 1): {"value": 3}, + ("parity", 1): {"value": 3}, + ("sum", 1): {"values": [3, 6, 8]}, + }, expected_run_order=["add_one", "parity", "double", "add_four", "add_one_again", "sum"], ), ], @@ -991,6 +1270,13 @@ def pipeline_that_has_multiple_branches_of_different_lengths_that_merge_into_a_c PipelineRunData( inputs={"first_addition": {"value": 1}, "third_addition": {"value": 1}}, expected_outputs={"fourth_addition": {"result": 12}}, + expected_component_calls={ + ("first_addition", 1): {"add": None, "value": 1}, + ("fourth_addition", 1): {"add": None, "value": 11}, + ("second_addition", 1): {"add": None, "value": 3}, + ("sum", 1): {"values": [3, 3, 5]}, + ("third_addition", 1): {"add": None, "value": 1}, + }, expected_run_order=["first_addition", "third_addition", "second_addition", "sum", "fourth_addition"], ) ], @@ -1017,12 +1303,22 @@ def pipeline_that_is_linear_and_returns_intermediate_outputs(): "first_addition": {"result": 3}, "second_addition": {"result": 7}, }, + expected_component_calls={ + ("double", 1): {"value": 3}, + ("first_addition", 1): {"add": None, "value": 1}, + ("second_addition", 1): {"add": None, "value": 6}, + }, expected_run_order=["first_addition", "double", "second_addition"], ), PipelineRunData( inputs={"first_addition": {"value": 1}}, include_outputs_from={"double"}, expected_outputs={"double": {"value": 6}, "second_addition": {"result": 7}}, + expected_component_calls={ + ("double", 1): {"value": 3}, + ("first_addition", 1): {"add": None, "value": 1}, + ("second_addition", 1): {"add": None, "value": 6}, + }, expected_run_order=["first_addition", "double", "second_addition"], ), ], @@ -1072,6 +1368,21 @@ def pipeline_that_has_a_loop_and_returns_intermediate_outputs_from_it(): "below_5": {"above": 8}, "add_three": {"result": 11}, }, + expected_component_calls={ + ("accumulator", 1): {"value": 4}, + ("accumulator", 2): {"value": 4}, + ("add_one", 1): {"add": None, "value": 3}, + ("add_three", 1): {"add": None, "value": 8}, + ("add_two", 1): {"add": None, "value": 11}, + ("below_10", 1): {"threshold": None, "value": 4}, + ("below_10", 2): {"threshold": None, "value": 4}, + ("below_10", 3): {"threshold": None, "value": 11}, + ("below_5", 1): {"threshold": None, "value": 4}, + ("below_5", 2): {"threshold": None, "value": 8}, + ("branch_joiner", 1): {"value": [4]}, + ("branch_joiner", 2): {"value": [4]}, + ("branch_joiner", 3): {"value": [11]}, + }, expected_run_order=[ "add_one", "branch_joiner", @@ -1124,12 +1435,22 @@ def run(self, value: int): "first_addition": {"result": 3}, "second_addition": {"result": 7}, }, + expected_component_calls={ + ("double", 1): {"value": 3}, + ("first_addition", 1): {"add": None, "value": 1}, + ("second_addition", 1): {"add": None, "value": 6}, + }, expected_run_order=["first_addition", "double", "second_addition"], ), PipelineRunData( inputs={"first_addition": {"value": 1}}, include_outputs_from={"double"}, expected_outputs={"double": {"original": 3, "value": 6}, "second_addition": {"result": 7}}, + expected_component_calls={ + ("double", 1): {"value": 3}, + ("first_addition", 1): {"add": None, "value": 1}, + ("second_addition", 1): {"add": None, "value": 6}, + }, expected_run_order=["first_addition", "double", "second_addition"], ), ], @@ -1158,11 +1479,16 @@ def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anythi PipelineRunData( inputs={"router": {"sentence": "Wir mussen reisen"}}, expected_outputs={"router": {"language_1": "German"}}, + expected_component_calls={("router", 1): {"sentence": "Wir mussen reisen"}}, expected_run_order=["router"], ), PipelineRunData( inputs={"router": {"sentence": "Yo tengo que viajar"}}, expected_outputs={"pb": {"prompt": "Ok, I know, that's Spanish"}}, + expected_component_calls={ + ("pb", 1): {"language": "Spanish", "template": None, "template_variables": None}, + ("router", 1): {"sentence": "Yo tengo que viajar"}, + }, expected_run_order=["router", "pb"], ), ], @@ -1247,6 +1573,45 @@ def run(self, query: str): "router": {"question": "This is a question with no_answer"}, }, expected_outputs={"fallback_llm": {"replies": ["There's simply no_answer to this question"]}}, + expected_component_calls={ + ("fallback_llm", 1): { + "prompt": "User entered a query that cannot be answered " + "with the given table.\n" + " The query was: This is a " + "question with no_answer and the table had " + "columns: .\n" + " Let the user know why " + "the question cannot be answered" + }, + ("fallback_prompt", 1): { + "columns": "", + "question": "This is a question with no_answer", + "template": None, + "template_variables": None, + }, + ("llm", 1): { + "prompt": "Please generate an SQL query. The query should answer " + "the following Question: This is a question with " + "no_answer;\n" + " If the question cannot be answered given " + "the provided table and columns, return 'no_answer'\n" + " The query is to be answered for the table " + "is called 'absenteeism' with the following\n" + " Columns: Age, Absenteeism_time_in_hours, " + "Days, Disciplinary_failure;\n" + " Answer:" + }, + ("prompt", 1): { + "columns": "Age, Absenteeism_time_in_hours, Days, Disciplinary_failure", + "question": "This is a question with no_answer", + "template": None, + "template_variables": None, + }, + ("router", 1): { + "question": "This is a question with no_answer", + "replies": ["There's simply no_answer to this question"], + }, + }, expected_run_order=["prompt", "llm", "router", "fallback_prompt", "fallback_llm"], ) ], @@ -1257,6 +1622,104 @@ def run(self, query: str): "router": {"question": "This is a question that has an answer"}, }, expected_outputs={"sql_querier": {"results": "This is the query result", "query": "Some SQL query"}}, + expected_component_calls={ + ("llm", 1): { + "prompt": "\n" + " You are an experienced and accurate Turkish CX " + "speacialist that classifies customer comments into " + "pre-defined categories below:\n" + "\n" + " Negative experience labels:\n" + " - Late delivery\n" + " - Rotten/spoilt item\n" + " - Bad Courier behavior\n" + "\n" + " Positive experience labels:\n" + " - Good courier behavior\n" + " - Thanks & appreciation\n" + " - Love message to courier\n" + " - Fast delivery\n" + " - Quality of products\n" + "\n" + " Create a JSON object as a response. The fields " + "are: 'positive_experience', 'negative_experience'.\n" + " Assign at least one of the pre-defined labels to " + "the given customer comment under positive and " + "negative experience fields.\n" + " If the comment has a positive experience, list " + "the label under 'positive_experience' field.\n" + " If the comments has a negative_experience, list " + "it under the 'negative_experience' field.\n" + " Here is the comment:\n" + "I loved the quality of the meal but the courier was " + "rude\n" + ". Just return the category names in the list. If " + "there aren't any, return an empty list.\n" + "\n" + " \n" + " " + }, + ("llm", 2): { + "prompt": "\n" + " You are an experienced and accurate Turkish CX " + "speacialist that classifies customer comments into " + "pre-defined categories below:\n" + "\n" + " Negative experience labels:\n" + " - Late delivery\n" + " - Rotten/spoilt item\n" + " - Bad Courier behavior\n" + "\n" + " Positive experience labels:\n" + " - Good courier behavior\n" + " - Thanks & appreciation\n" + " - Love message to courier\n" + " - Fast delivery\n" + " - Quality of products\n" + "\n" + " Create a JSON object as a response. The fields " + "are: 'positive_experience', 'negative_experience'.\n" + " Assign at least one of the pre-defined labels to " + "the given customer comment under positive and " + "negative experience fields.\n" + " If the comment has a positive experience, list " + "the label under 'positive_experience' field.\n" + " If the comments has a negative_experience, list " + "it under the 'negative_experience' field.\n" + " Here is the comment:\n" + "I loved the quality of the meal but the courier was " + "rude\n" + ". Just return the category names in the list. If " + "there aren't any, return an empty list.\n" + "\n" + " \n" + " You already created the following output in a " + "previous attempt: ['This is an invalid reply']\n" + " However, this doesn't comply with the format " + "requirements from above and triggered this Python " + "exception: this is an error message\n" + " Correct the output and try again. Just return the " + "corrected output without any extra explanations.\n" + " \n" + " " + }, + ("output_validator", 1): {"replies": ["This is a valid reply"]}, + ("output_validator", 2): {"replies": ["This is a valid reply"]}, + ("prompt_builder", 1): { + "comment": "", + "error_message": "", + "invalid_replies": "", + "template": None, + "template_variables": {"comment": "I loved the quality of the meal but the courier was rude"}, + }, + ("prompt_builder", 2): { + "comment": "", + "error_message": "this is an error message", + "invalid_replies": ["This is an invalid reply"], + "template": None, + "template_variables": {"comment": "I loved the quality of the meal but the courier was rude"}, + }, + }, expected_run_order=["prompt", "llm", "router", "sql_querier"], ) ], @@ -1335,6 +1798,104 @@ def run(self, prompt: str): PipelineRunData( inputs={"prompt_builder": {"template_variables": {"comment": comment}}}, expected_outputs={"output_validator": {"valid_replies": ["This is a valid reply"]}}, + expected_component_calls={ + ("llm", 1): { + "prompt": "\n" + " You are an experienced and accurate Turkish CX " + "speacialist that classifies customer comments into " + "pre-defined categories below:\n" + "\n" + " Negative experience labels:\n" + " - Late delivery\n" + " - Rotten/spoilt item\n" + " - Bad Courier behavior\n" + "\n" + " Positive experience labels:\n" + " - Good courier behavior\n" + " - Thanks & appreciation\n" + " - Love message to courier\n" + " - Fast delivery\n" + " - Quality of products\n" + "\n" + " Create a JSON object as a response. The fields " + "are: 'positive_experience', 'negative_experience'.\n" + " Assign at least one of the pre-defined labels to " + "the given customer comment under positive and " + "negative experience fields.\n" + " If the comment has a positive experience, list " + "the label under 'positive_experience' field.\n" + " If the comments has a negative_experience, list " + "it under the 'negative_experience' field.\n" + " Here is the comment:\n" + "I loved the quality of the meal but the courier was " + "rude\n" + ". Just return the category names in the list. If " + "there aren't any, return an empty list.\n" + "\n" + " \n" + " " + }, + ("llm", 2): { + "prompt": "\n" + " You are an experienced and accurate Turkish CX " + "speacialist that classifies customer comments into " + "pre-defined categories below:\n" + "\n" + " Negative experience labels:\n" + " - Late delivery\n" + " - Rotten/spoilt item\n" + " - Bad Courier behavior\n" + "\n" + " Positive experience labels:\n" + " - Good courier behavior\n" + " - Thanks & appreciation\n" + " - Love message to courier\n" + " - Fast delivery\n" + " - Quality of products\n" + "\n" + " Create a JSON object as a response. The fields " + "are: 'positive_experience', 'negative_experience'.\n" + " Assign at least one of the pre-defined labels to " + "the given customer comment under positive and " + "negative experience fields.\n" + " If the comment has a positive experience, list " + "the label under 'positive_experience' field.\n" + " If the comments has a negative_experience, list " + "it under the 'negative_experience' field.\n" + " Here is the comment:\n" + "I loved the quality of the meal but the courier was " + "rude\n" + ". Just return the category names in the list. If " + "there aren't any, return an empty list.\n" + "\n" + " \n" + " You already created the following output in a " + "previous attempt: ['This is an invalid reply']\n" + " However, this doesn't comply with the format " + "requirements from above and triggered this Python " + "exception: this is an error message\n" + " Correct the output and try again. Just return the " + "corrected output without any extra explanations.\n" + " \n" + " " + }, + ("output_validator", 1): {"replies": ["This is a valid reply"]}, + ("output_validator", 2): {"replies": ["This is a valid reply"]}, + ("prompt_builder", 1): { + "comment": "", + "error_message": "", + "invalid_replies": "", + "template": None, + "template_variables": {"comment": "I loved the quality of the meal but the courier was rude"}, + }, + ("prompt_builder", 2): { + "comment": "", + "error_message": "this is an error message", + "invalid_replies": ["This is an invalid reply"], + "template": None, + "template_variables": {"comment": "I loved the quality of the meal but the courier was rude"}, + }, + }, expected_run_order=[ "prompt_builder", "llm", @@ -1440,6 +2001,76 @@ def run(self, prompt: str, generation_kwargs: Optional[Dict[str, Any]] = None): "llm": {"replies": ["This is a reply"], "meta": {"meta_key": "meta_value"}}, "spellchecker": {"meta": {"meta_key": "meta_value"}}, }, + expected_component_calls={ + ("llm", 1): { + "generation_kwargs": None, + "prompt": "\n" + " According to these documents:\n" + "\n" + " \n" + " This is a document\n" + " \n" + "\n" + " Answer the given question: \n" + " \n" + " This is a reply\n" + " \n" + " \n" + " Answer:\n" + " ", + }, + ("prompt_builder1", 1): { + "question": "Wha i Acromegaly?", + "template": None, + "template_variables": None, + }, + ("prompt_builder2", 1): { + "documents": [ + Document( + id="9d51914541072d3d822910785727db8a3838dba5ca6ebb0a543969260ecdeda6", + content="This is a document", + ) + ], + "question": "\n \n This is a reply\n \n ", + "template": None, + "template_variables": None, + }, + ("prompt_builder3", 1): { + "replies": ["This is a reply"], + "template": None, + "template_variables": None, + }, + ("ranker", 1): { + "calibration_factor": None, + "documents": [ + Document( + id="9d51914541072d3d822910785727db8a3838dba5ca6ebb0a543969260ecdeda6", + content="This is a document", + ) + ], + "query": "\n \n This is a reply\n \n ", + "scale_score": None, + "score_threshold": None, + "top_k": None, + }, + ("retriever", 1): { + "filters": None, + "query": "\n \n This is a reply\n \n ", + "scale_score": None, + "top_k": None, + }, + ("spellchecker", 1): { + "generation_kwargs": None, + "prompt": "\n" + " You are a spellchecking system. Check " + "the given query and fill in the corrected " + "query.\n" + "\n" + " Question: Wha i Acromegaly?\n" + " Corrected question:\n" + " ", + }, + }, expected_run_order=[ "prompt_builder1", "spellchecker", @@ -1763,6 +2394,256 @@ def run(self, replies: List[ChatMessage]): "search_prompt_builder": {"template": search_message}, }, expected_outputs={"router": {"finish": "Eiffel Tower"}}, + expected_component_calls={ + ("llm", 1): { + "generation_kwargs": None, + "messages": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n " + ) + ], + _name=None, + _meta={}, + ) + ], + }, + ("llm", 2): { + "generation_kwargs": None, + "messages": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\nObservation: Tower of Pisa is 55 meters tall\n\n\nThought: " + ) + ], + _name=None, + _meta={}, + ) + ], + }, + ("main_input", 1): { + "value": [ + [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: {{query}}\n\n Thought:\n " + ) + ], + _name=None, + _meta={}, + ) + ] + ] + }, + ("main_input", 2): { + "value": [ + [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\nObservation: Tower of Pisa is 55 meters tall\n\n\nThought: " + ) + ], + _name=None, + _meta={}, + ) + ] + ] + }, + ("prompt_builder", 1): { + "query": "which tower is taller: eiffel tower or tower of pisa?", + "template": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: {{query}}\n\n Thought:\n " + ) + ], + _name=None, + _meta={}, + ) + ], + "template_variables": None, + }, + ("prompt_builder", 2): { + "query": "which tower is taller: eiffel tower or tower of pisa?", + "template": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\nObservation: Tower of Pisa is 55 meters tall\n\n\nThought: " + ) + ], + _name=None, + _meta={}, + ) + ], + "template_variables": None, + }, + ("prompt_concatenator_after_action", 1): { + "current_prompt": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n " + ) + ], + _name=None, + _meta={}, + ) + ], + "replies": [ + ChatMessage( + _role="assistant", + _content=[ + TextContent( + text="thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\n" + ) + ], + _name=None, + _meta={}, + ) + ], + }, + ("prompt_concatenator_after_action", 2): { + "current_prompt": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\nObservation: Tower of Pisa is 55 meters tall\n\n\nThought: " + ) + ], + _name=None, + _meta={}, + ) + ], + "replies": [ + ChatMessage( + _role="assistant", + _content=[TextContent(text="thinking\n Action: finish[Eiffel Tower]\n")], + _name=None, + _meta={}, + ) + ], + }, + ("prompt_concatenator_after_observation", 1): { + "current_prompt": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\n" + ) + ], + _name=None, + _meta={}, + ) + ], + "replies": [ + ChatMessage( + _role="assistant", + _content=[TextContent(text="Observation: Tower of Pisa is 55 meters tall\n\n")], + _name=None, + _meta={}, + ) + ], + }, + ("router", 1): { + "tool_id_and_param": ["google_search", "What is taller, Eiffel Tower or Leaning Tower of Pisa"] + }, + ("router", 2): {"tool_id_and_param": ["finish", "Eiffel Tower"]}, + ("router_search", 1): {"query": "What is taller, Eiffel Tower or Leaning Tower of Pisa"}, + ("search_llm", 1): { + "generation_kwargs": None, + "messages": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Given these web search results:\n\n \n Eiffel Tower is 300 meters tall\n \n Tower of Pisa is 55 meters tall\n \n\n Be as brief as possible, max one sentence.\n Answer the question: What is taller, Eiffel Tower or Leaning Tower of Pisa\n " + ) + ], + _name=None, + _meta={}, + ) + ], + }, + ("search_output_adapter", 1): { + "replies": [ + ChatMessage( + _role="assistant", + _content=[TextContent(text="Tower of Pisa is 55 meters tall\n")], + _name=None, + _meta={}, + ) + ] + }, + ("search_prompt_builder", 1): { + "documents": [ + Document( + id="c37eb19352b261b17314cac9e1539921b5996f88c99ad0b134f12effb38ed467", + content="Eiffel Tower is 300 meters tall", + ), + Document( + id="c5281056a220c32e6fa1c4ae7d3f263c0f25fd620592c5e45049a9dcb778f129", + content="Tower of Pisa is 55 meters tall", + ), + ], + "search_query": "What is taller, Eiffel Tower or Leaning Tower of Pisa", + "template": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Given these web search results:\n\n {% for doc in documents %}\n {{ doc.content }}\n {% endfor %}\n\n Be as brief as possible, max one sentence.\n Answer the question: {{search_query}}\n " + ) + ], + _name=None, + _meta={}, + ) + ], + "template_variables": None, + }, + ("tool_extractor", 1): { + "messages": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\n" + ) + ], + _name=None, + _meta={}, + ) + ] + }, + ("tool_extractor", 2): { + "messages": [ + ChatMessage( + _role="user", + _content=[ + TextContent( + text="\n Solve a question answering task with interleaving Thought, Action, Observation steps.\n\n Thought reasons about the current situation\n\n Action can be:\n google_search - Searches Google for the exact concept/entity (given in square brackets) and returns the results for you to use\n finish - Returns the final answer (given in square brackets) and finishes the task\n\n Observation summarizes the Action outcome and helps in formulating the next\n Thought in Thought, Action, Observation interleaving triplet of steps.\n\n After each Observation, provide the next Thought and next Action.\n Don't execute multiple steps even though you know the answer.\n Only generate Thought and Action, never Observation, you'll get Observation from Action.\n Follow the pattern in the example below.\n\n Example:\n ###########################\n Question: Which magazine was started first Arthur’s Magazine or First for Women?\n Thought: I need to search Arthur’s Magazine and First for Women, and find which was started\n first.\n Action: google_search[When was 'Arthur’s Magazine' started?]\n Observation: Arthur’s Magazine was an American literary periodical ˘\n published in Philadelphia and founded in 1844. Edited by Timothy Shay Arthur, it featured work by\n Edgar A. Poe, J.H. Ingraham, Sarah Josepha Hale, Thomas G. Spear, and others. In May 1846\n it was merged into Godey’s Lady’s Book.\n Thought: Arthur’s Magazine was started in 1844. I need to search First for Women founding date next\n Action: google_search[When was 'First for Women' magazine started?]\n Observation: First for Women is a woman’s magazine published by Bauer Media Group in the\n USA. The magazine was started in 1989. It is based in Englewood Cliffs, New Jersey. In 2011\n the circulation of the magazine was 1,310,696 copies.\n Thought: First for Women was started in 1989. 1844 (Arthur’s Magazine) ¡ 1989 (First for\n Women), so Arthur’s Magazine was started first.\n Action: finish[Arthur’s Magazine]\n ############################\n\n Let's start, the question is: which tower is taller: eiffel tower or tower of pisa?\n\n Thought:\n thinking\n Action: google_search[What is taller, Eiffel Tower or Leaning Tower of Pisa]\nObservation: Tower of Pisa is 55 meters tall\n\n\nThought: thinking\n Action: finish[Eiffel Tower]\n" + ) + ], + _name=None, + _meta={}, + ) + ] + }, + }, expected_run_order=[ "main_input", "prompt_builder", @@ -1823,6 +2704,18 @@ def run(self, create_document: bool = False): ] }, }, + expected_component_calls={ + ("documents_joiner", 1): { + "documents": [ + [Document(id="First document", content="First document")], + [Document(id="Third document", content="Third document")], + ], + "top_k": None, + }, + ("first_creator", 1): {"create_document": True}, + ("second_creator", 1): {"create_document": False}, + ("third_creator", 1): {"create_document": True}, + }, expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), PipelineRunData( @@ -1836,6 +2729,18 @@ def run(self, create_document: bool = False): ] }, }, + expected_component_calls={ + ("documents_joiner", 1): { + "documents": [ + [Document(id="First document", content="First document")], + [Document(id="Second document", content="Second document")], + ], + "top_k": None, + }, + ("first_creator", 1): {"create_document": True}, + ("second_creator", 1): {"create_document": True}, + ("third_creator", 1): {"create_document": False}, + }, expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), ], @@ -1882,6 +2787,18 @@ def run(self, create_document: bool = False): ] }, }, + expected_component_calls={ + ("documents_joiner", 1): { + "documents": [ + [Document(id="First document", content="First document")], + [Document(id="Third document", content="Third document")], + ], + "top_k": None, + }, + ("first_creator", 1): {"create_document": True}, + ("second_creator", 1): {"create_document": False}, + ("third_creator", 1): {"create_document": True}, + }, expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), PipelineRunData( @@ -1895,6 +2812,18 @@ def run(self, create_document: bool = False): ] }, }, + expected_component_calls={ + ("documents_joiner", 1): { + "documents": [ + [Document(id="First document", content="First document")], + [Document(id="Second document", content="Second document")], + ], + "top_k": None, + }, + ("first_creator", 1): {"create_document": True}, + ("second_creator", 1): {"create_document": True}, + ("third_creator", 1): {"create_document": False}, + }, expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), ], @@ -1939,6 +2868,45 @@ def that_has_an_answer_joiner_variadic_component(): ] } }, + expected_component_calls={ + ("answer_builder_1", 1): { + "documents": None, + "meta": None, + "pattern": None, + "query": "What's Natural Language Processing?", + "reference_pattern": None, + "replies": ["This is a test answer"], + }, + ("answer_builder_2", 1): { + "documents": None, + "meta": None, + "pattern": None, + "query": "What's Natural Language Processing?", + "reference_pattern": None, + "replies": ["This is a second test answer"], + }, + ("answer_joiner", 1): { + "answers": [ + [ + GeneratedAnswer( + data="This is a test answer", + query="What's Natural Language Processing?", + documents=[], + meta={}, + ) + ], + [ + GeneratedAnswer( + data="This is a second test answer", + query="What's Natural Language Processing?", + documents=[], + meta={}, + ) + ], + ], + "top_k": None, + }, + }, expected_run_order=["answer_builder_1", "answer_builder_2", "answer_joiner"], ) ], @@ -2015,6 +2983,35 @@ def run(self, prompt: str): ] } }, + expected_component_calls={ + ("builder", 1): {"template": None, "template_variables": None}, + ("document_joiner", 1): { + "documents": [ + [ + Document( + id="doc2", + content="some text about investigation and treatment of Alzheimer disease", + meta={"year": 2023, "disease": "Alzheimer", "author": "John Bread"}, + score=3.324112496100923, + ) + ] + ], + "top_k": None, + }, + ("metadata_extractor", 1): {"prompt": '{"disease": "Alzheimer", "year": 2023}'}, + ("retriever", 1): { + "filters": { + "conditions": [ + {"field": "meta.disease", "operator": "==", "value": "Alzheimer"}, + {"field": "meta.year", "operator": "==", "value": 2023}, + ], + "operator": "AND", + }, + "query": "publications 2023 Alzheimer's disease", + "scale_score": None, + "top_k": None, + }, + }, expected_run_order=["builder", "metadata_extractor", "retriever", "document_joiner"], ) ], @@ -2150,6 +3147,33 @@ def run(self, prompt: str): expected_outputs={ "answer_builder": {"answers": [GeneratedAnswer(data="42", query=question, documents=[])]} }, + expected_component_calls={ + ("answer_builder", 1): { + "documents": None, + "meta": None, + "pattern": None, + "query": "What is the answer?", + "reference_pattern": None, + "replies": ["42"], + }, + ("answer_validator", 1): {"replies": ["No answer"]}, + ("answer_validator", 2): {"replies": ["42"]}, + ("llm", 1): {"prompt": "Random template"}, + ("llm", 2): {"prompt": ""}, + ("prompt_builder", 1): { + "invalid_replies": "", + "question": "What is the answer?", + "template": "Random template", + "template_variables": None, + }, + ("prompt_builder", 2): { + "invalid_replies": ["No answer"], + "question": "What is the answer?", + "template": None, + "template_variables": None, + }, + ("prompt_cleaner", 1): {"prompt": "Random template"}, + }, expected_run_order=[ "prompt_cleaner", "prompt_builder", @@ -2246,6 +3270,62 @@ def run(self, documents: List[Document]): ] }, }, + expected_component_calls={ + ("comma_splitter", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ) + ] + }, + ("conditional_router", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ) + ] + }, + ("document_cleaner", 1): { + "documents": [ + Document(id="0", content="This document has so many"), + Document(id="1", content=" sentences. Like this one"), + Document(id="2", content=" or this one. Or even this other one."), + ] + }, + ("document_joiner", 1): { + "documents": [ + [ + Document(id="0", content="This document has so many"), + Document(id="1", content=" sentences. Like this one"), + Document(id="2", content=" or this one. Or even this other one."), + ], + [ + Document(id="0", content="This document has so many"), + Document(id="1", content="sentences. Like this one"), + Document(id="2", content="or this one. Or even this other one."), + ], + ], + "top_k": None, + }, + ("noop2", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ) + ] + }, + ("noop3", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ) + ] + }, + }, expected_run_order=[ "comma_splitter", "noop2", @@ -2276,6 +3356,109 @@ def run(self, documents: List[Document]): ] } }, + expected_component_calls={ + ("comma_splitter", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + ] + }, + ("conditional_router", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + ] + }, + ("document_cleaner", 1): { + "documents": [ + Document(id="0", content="This document has so many"), + Document(id="1", content=" sentences. Like this one"), + Document(id="2", content=" or this one. Or even this other one."), + Document(id="3", content="This document has so many"), + Document(id="4", content=" sentences. Like this one"), + Document(id="5", content=" or this one. Or even this other one."), + ] + }, + ("document_joiner", 1): { + "documents": [ + [ + Document(id="0", content="This document has so many"), + Document(id="1", content=" sentences. Like this one"), + Document(id="2", content=" or this one. Or even this other one."), + Document(id="3", content="This document has so many"), + Document(id="4", content=" sentences. Like this one"), + Document(id="5", content=" or this one. Or even this other one."), + ], + [ + Document(id="0", content="This document has so many"), + Document(id="1", content="sentences. Like this one"), + Document(id="2", content="or this one. Or even this other one."), + Document(id="3", content="This document has so many"), + Document(id="4", content="sentences. Like this one"), + Document(id="5", content="or this one. Or even this other one."), + ], + [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + ], + ], + "top_k": None, + }, + ("empty_lines_cleaner", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + ] + }, + ("noop2", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + ] + }, + ("noop3", 1): { + "documents": [ + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + Document( + id="1000", + content="This document has so many, sentences. Like this one, or this one. Or even this other one.", + ), + ] + }, + }, expected_run_order=[ "comma_splitter", "noop2", @@ -2312,6 +3495,17 @@ def that_has_a_string_variadic_component(): "strings": ["Builder 1: What's Natural Language Processing?", "Builder 2: What's is life?"] } }, + expected_component_calls={ + ("prompt_builder_1", 1): { + "query": "What's Natural Language Processing?", + "template": None, + "template_variables": None, + }, + ("prompt_builder_2", 1): {"query": "What's is life?", "template": None, "template_variables": None}, + ("string_joiner", 1): { + "strings": ["Builder 1: What's Natural Language Processing?", "Builder 2: What's is life?"] + }, + }, expected_run_order=["prompt_builder_1", "prompt_builder_2", "string_joiner"], ) ], @@ -2437,6 +3631,130 @@ def run(self, query: str): "answers": [GeneratedAnswer(data="answer: here is my answer", query=query, documents=[])] } }, + expected_component_calls={ + ("agent_llm", 1): { + "prompt": "\n" + "Your task is to answer the user's question.\n" + "You can use a RAG system to find information.\n" + "Use the RAG system until you have sufficient " + "information to answer the question.\n" + 'To use the RAG system, output "search:" ' + "followed by your question.\n" + 'Once you have an answer, output "answer:" ' + "followed by your answer.\n" + "\n" + "Here is the question: Does this run reliably?\n" + " " + }, + ("agent_llm", 2): { + "prompt": "\n" + "Your task is to answer the user's question.\n" + "You can use a RAG system to find information.\n" + "Use the RAG system until you have sufficient " + "information to answer the question.\n" + 'To use the RAG system, output "search:" ' + "followed by your question.\n" + 'Once you have an answer, output "answer:" ' + "followed by your answer.\n" + "\n" + "Here is the question: Does this run reliably?\n" + " \n" + "This is all the information I found!" + }, + ("agent_prompt", 1): { + "query": "Does this run reliably?", + "template": None, + "template_variables": None, + }, + ("answer_builder", 1): { + "documents": None, + "meta": None, + "pattern": None, + "query": "Does this run reliably?", + "reference_pattern": None, + "replies": ["answer: here is my answer"], + }, + ("concatenator", 1): { + "current_prompt": "\n" + "Your task is to answer the user's " + "question.\n" + "You can use a RAG system to find " + "information.\n" + "Use the RAG system until you have " + "sufficient information to answer the " + "question.\n" + "To use the RAG system, output " + '"search:" followed by your ' + "question.\n" + "Once you have an answer, output " + '"answer:" followed by your answer.\n' + "\n" + "Here is the question: Does this run " + "reliably?\n" + " ", + "rag_answer": ["This is all the information I found!"], + }, + ("joiner", 1): { + "value": [ + "\n" + "Your task is to answer the user's question.\n" + "You can use a RAG system to find information.\n" + "Use the RAG system until you have sufficient " + "information to answer the question.\n" + 'To use the RAG system, output "search:" followed ' + "by your question.\n" + 'Once you have an answer, output "answer:" followed ' + "by your answer.\n" + "\n" + "Here is the question: Does this run reliably?\n" + " " + ] + }, + ("joiner", 2): { + "value": [ + "\n" + "Your task is to answer the user's question.\n" + "You can use a RAG system to find information.\n" + "Use the RAG system until you have sufficient " + "information to answer the question.\n" + 'To use the RAG system, output "search:" followed ' + "by your question.\n" + 'Once you have an answer, output "answer:" followed ' + "by your answer.\n" + "\n" + "Here is the question: Does this run reliably?\n" + " \n" + "This is all the information I found!" + ] + }, + ("rag_llm", 1): { + "prompt": "\n" + "Answer the question based on the provided " + "documents.\n" + "Question: Does this run reliably?\n" + "Documents:\n" + "\n" + "This is a document potentially answering the " + "question.\n" + "\n" + " " + }, + ("rag_prompt", 1): { + "documents": [ + Document( + id="969664d0cf76e52b0ffb719d00d3e5a6b1c90bb29e56f6107dfd87bf2f5388ed", + content="This is a document potentially answering the question.", + meta={"access_group": 1}, + ) + ], + "query": "Does this run reliably?", + "template": None, + "template_variables": None, + }, + ("retriever", 1): {"query": "search: Can you help me?"}, + ("router", 1): {"replies": ["search: Can you help me?"]}, + ("router", 2): {"replies": ["answer: here is my answer"]}, + }, expected_run_order=[ "agent_prompt", "joiner", @@ -2548,6 +3866,73 @@ def run(self, prompt: str): expected_outputs={ "answer_builder": {"answers": [GeneratedAnswer(data="valid code", query=task, documents=[])]} }, + expected_component_calls={ + ("answer_builder", 1): { + "documents": None, + "meta": None, + "pattern": None, + "query": "Generate code to generate christmas ascii-art", + "reference_pattern": None, + "replies": ["valid code"], + }, + ("code_llm", 1): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "\n" + " " + }, + ("code_llm", 2): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "\n" + "Here is your initial attempt and some feedback:\n" + "invalid code\n" + "F\n" + "\n" + " " + }, + ("code_prompt", 1): { + "feedback": "", + "task": "Generate code to generate christmas ascii-art", + "template": None, + "template_variables": None, + }, + ("code_prompt", 2): { + "feedback": "invalid code\nF", + "task": "Generate code to generate christmas ascii-art", + "template": None, + "template_variables": None, + }, + ("concatenator", 1): {"current_prompt": ["invalid code"], "feedback": "FAIL"}, + ("feedback_llm", 1): { + "prompt": "\n" + "Check if this code is valid and can run: " + "invalid code\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_llm", 2): { + "prompt": "\n" + "Check if this code is valid and can run: " + "valid code\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_prompt", 1): {"code": ["invalid code"], "template": None, "template_variables": None}, + ("feedback_prompt", 2): {"code": ["valid code"], "template": None, "template_variables": None}, + ("router", 1): {"code": ["invalid code"], "replies": ["FAIL"]}, + ("router", 2): {"code": ["valid code"], "replies": ["PASS"]}, + }, expected_run_order=[ "code_prompt", "code_llm", @@ -2660,6 +4045,73 @@ def run(self, prompt: str): expected_outputs={ "answer_builder": {"answers": [GeneratedAnswer(data="valid code", query=task, documents=[])]} }, + expected_component_calls={ + ("answer_builder", 1): { + "documents": None, + "meta": None, + "pattern": None, + "query": "Generate code to generate christmas ascii-art", + "reference_pattern": None, + "replies": ["valid code"], + }, + ("code_llm", 1): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "\n" + " " + }, + ("code_llm", 2): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "\n" + "Here is your initial attempt and some feedback:\n" + "invalid code\n" + "F\n" + "\n" + " " + }, + ("code_prompt", 1): { + "feedback": "", + "task": "Generate code to generate christmas ascii-art", + "template": None, + "template_variables": None, + }, + ("code_prompt", 2): { + "feedback": "invalid code\nF", + "task": "Generate code to generate christmas ascii-art", + "template": None, + "template_variables": None, + }, + ("concatenator", 1): {"current_prompt": ["invalid code"], "feedback": "FAIL"}, + ("feedback_llm", 1): { + "prompt": "\n" + "Check if this code is valid and can run: " + "invalid code\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_llm", 2): { + "prompt": "\n" + "Check if this code is valid and can run: " + "valid code\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_prompt", 1): {"code": ["invalid code"], "template": None, "template_variables": None}, + ("feedback_prompt", 2): {"code": ["valid code"], "template": None, "template_variables": None}, + ("router", 1): {"code": ["invalid code"], "replies": ["FAIL"]}, + ("router", 2): {"code": ["valid code"], "replies": ["PASS"]}, + }, expected_run_order=[ "code_prompt", "code_llm", @@ -2807,6 +4259,607 @@ def run(self, replies: List[str]): PipelineRunData( inputs={"code_prompt": {"task": task}}, expected_outputs={"feedback_router": {"pass": ["PASS"]}}, + expected_component_calls={ + ("agent_concatenator", 1): { + "current_prompt": "\n" + "Generate code to solve the " + "task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by " + "returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, " + "respond with:\n" + "Task finished!\n" + "\n" + "\n" + " ", + "files": "This is the edited file content.", + }, + ("agent_concatenator", 2): { + "current_prompt": "\n" + "Generate code to solve the " + "task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by " + "returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, " + "respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file " + "content.", + "files": "This is the edited file content.", + }, + ("agent_concatenator", 3): { + "current_prompt": "\n" + "Generate code to solve the " + "task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by " + "returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, " + "respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.", + "files": "This is the edited file content.", + }, + ("agent_concatenator", 4): { + "current_prompt": "\n" + "Generate code to solve the " + "task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by " + "returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, " + "respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.\n" + "FAIL", + "files": "This is the edited file content.", + }, + ("agent_concatenator", 5): { + "current_prompt": "\n" + "Generate code to solve the " + "task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by " + "returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, " + "respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.\n" + "FAIL\n" + "This is the edited file " + "content.", + "files": "This is the edited file content.", + }, + ("agent_concatenator", 6): { + "current_prompt": "\n" + "Generate code to solve the " + "task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by " + "returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, " + "respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.\n" + "FAIL\n" + "This is the edited file " + "content.\n" + "This is the edited file " + "content.", + "files": "This is the edited file content.", + }, + ("code_llm", 1): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " " + }, + ("code_llm", 2): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content." + }, + ("code_llm", 3): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content." + }, + ("code_llm", 4): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content." + }, + ("code_llm", 5): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL" + }, + ("code_llm", 6): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content." + }, + ("code_llm", 7): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content." + }, + ("code_llm", 8): { + "prompt": "\n" + "Generate code to solve the task: Generate code " + "to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content." + }, + ("code_prompt", 1): { + "feedback": "", + "task": "Generate code to generate christmas ascii-art", + "template": None, + "template_variables": None, + }, + ("feedback_llm", 1): { + "prompt": "\n" + "\n" + "Check if this code is valid and can run: \n" + "Generate code to solve the task: Generate " + "code to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + "\n" + " " + }, + ("feedback_llm", 2): { + "prompt": "\n" + "\n" + "Check if this code is valid and can run: \n" + "Generate code to solve the task: Generate " + "code to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + "\n" + " " + }, + ("feedback_prompt", 1): { + "code": "\n" + "Generate code to solve the task: Generate " + "code to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.", + "task_finished": ["Task finished!"], + "template": None, + "template_variables": None, + }, + ("feedback_prompt", 2): { + "code": "\n" + "Generate code to solve the task: Generate " + "code to generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.", + "task_finished": ["Task finished!"], + "template": None, + "template_variables": None, + }, + ("feedback_router", 1): { + "current_prompt": "\n" + "Generate code to solve the task: " + "Generate code to generate " + "christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond " + "with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.", + "replies": ["FAIL"], + }, + ("feedback_router", 2): { + "current_prompt": "\n" + "Generate code to solve the task: " + "Generate code to generate " + "christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond " + "with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.", + "replies": ["PASS"], + }, + ("file_editor", 1): {"replies": ["Edit: file_1.py"]}, + ("file_editor", 2): {"replies": ["Edit: file_2.py"]}, + ("file_editor", 3): {"replies": ["Edit: file_3.py"]}, + ("file_editor", 4): {"replies": ["Edit: file_1.py"]}, + ("file_editor", 5): {"replies": ["Edit: file_2.py"]}, + ("file_editor", 6): {"replies": ["Edit: file_3.py"]}, + ("joiner", 1): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " " + ] + }, + ("joiner", 2): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content." + ] + }, + ("joiner", 3): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content." + ] + }, + ("joiner", 4): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content." + ] + }, + ("joiner", 5): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL" + ] + }, + ("joiner", 6): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content." + ] + }, + ("joiner", 7): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content." + ] + }, + ("joiner", 8): { + "value": [ + "\n" + "Generate code to solve the task: Generate code to " + "generate christmas ascii-art\n" + "\n" + "You can edit files by returning:\n" + "Edit: file_name\n" + "\n" + "Once you solved the task, respond with:\n" + "Task finished!\n" + "\n" + "\n" + " \n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "FAIL\n" + "This is the edited file content.\n" + "This is the edited file content.\n" + "This is the edited file content." + ] + }, + ("tool_use_router", 1): {"replies": ["Edit: file_1.py"]}, + ("tool_use_router", 2): {"replies": ["Edit: file_2.py"]}, + ("tool_use_router", 3): {"replies": ["Edit: file_3.py"]}, + ("tool_use_router", 4): {"replies": ["Task finished!"]}, + ("tool_use_router", 5): {"replies": ["Edit: file_1.py"]}, + ("tool_use_router", 6): {"replies": ["Edit: file_2.py"]}, + ("tool_use_router", 7): {"replies": ["Edit: file_3.py"]}, + ("tool_use_router", 8): {"replies": ["Task finished!"]}, + }, expected_run_order=[ "code_prompt", "joiner", @@ -2980,6 +5033,148 @@ def generate_santa_sleigh(): ] } }, + expected_component_calls={ + ("answer_builder", 1): { + "prompt": "Generate code to generate christmas " + "ascii-art\n" + "invalid code\n" + "FAIL\n" + "invalid code\n" + "FAIL, come on, try again.", + "query": "Generate code to generate christmas ascii-art", + "replies": [ + "\n" + "def generate_santa_sleigh():\n" + " '''\n" + " Returns ASCII art of Santa Claus on " + "his sleigh with Rudolph leading the " + "way.\n" + " '''\n" + " # implementation goes here.\n" + " return art\n" + " " + ], + }, + ("code_llm", 1): {"prompt": "Generate code to generate christmas ascii-art"}, + ("code_llm", 2): {"prompt": "Generate code to generate christmas ascii-art\ninvalid code\nFAIL"}, + ("code_llm", 3): { + "prompt": "Generate code to generate christmas ascii-art\n" + "invalid code\n" + "FAIL\n" + "invalid code\n" + "FAIL, come on, try again." + }, + ("code_prompt", 1): { + "task": "Generate code to generate christmas ascii-art", + "template": None, + "template_variables": None, + }, + ("code_prompt", 2): { + "task": "Generate code to generate christmas ascii-art\ninvalid code\nFAIL", + "template": None, + "template_variables": None, + }, + ("code_prompt", 3): { + "task": "Generate code to generate christmas ascii-art\n" + "invalid code\n" + "FAIL\n" + "invalid code\n" + "FAIL, come on, try again.", + "template": None, + "template_variables": None, + }, + ("concatenator", 1): { + "code_prompt": "Generate code to generate christmas ascii-art", + "feedback": "FAIL", + "generated_code": ["invalid code"], + }, + ("concatenator", 2): { + "code_prompt": "Generate code to generate christmas ascii-art\ninvalid code\nFAIL", + "feedback": "FAIL, come on, try again.", + "generated_code": ["invalid code"], + }, + ("feedback_llm", 1): { + "prompt": "\n" + "Check if this code is valid and can run: " + "invalid code\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_llm", 2): { + "prompt": "\n" + "Check if this code is valid and can run: " + "invalid code\n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_llm", 3): { + "prompt": "\n" + "Check if this code is valid and can run: \n" + "def generate_santa_sleigh():\n" + " '''\n" + " Returns ASCII art of Santa Claus on his " + "sleigh with Rudolph leading the way.\n" + " '''\n" + " # implementation goes here.\n" + " return art\n" + " \n" + 'Return "PASS" if it passes and "FAIL" if it ' + "fails.\n" + "Provide additional feedback on why it " + "fails.\n" + " " + }, + ("feedback_prompt", 1): {"code": ["invalid code"], "template": None, "template_variables": None}, + ("feedback_prompt", 2): {"code": ["invalid code"], "template": None, "template_variables": None}, + ("feedback_prompt", 3): { + "code": [ + "\n" + "def generate_santa_sleigh():\n" + " '''\n" + " Returns ASCII art of Santa Claus on " + "his sleigh with Rudolph leading the way.\n" + " '''\n" + " # implementation goes here.\n" + " return art\n" + " " + ], + "template": None, + "template_variables": None, + }, + ("joiner", 1): {"value": ["Generate code to generate christmas ascii-art"]}, + ("joiner", 2): {"value": ["Generate code to generate christmas ascii-art\ninvalid code\nFAIL"]}, + ("joiner", 3): { + "value": [ + "Generate code to generate christmas ascii-art\n" + "invalid code\n" + "FAIL\n" + "invalid code\n" + "FAIL, come on, try again." + ] + }, + ("router", 1): {"code": ["invalid code"], "replies": ["FAIL"]}, + ("router", 2): {"code": ["invalid code"], "replies": ["FAIL, come on, try again."]}, + ("router", 3): { + "code": [ + "\n" + "def generate_santa_sleigh():\n" + " '''\n" + " Returns ASCII art of Santa Claus on his sleigh " + "with Rudolph leading the way.\n" + " '''\n" + " # implementation goes here.\n" + " return art\n" + " " + ], + "replies": ["PASS"], + }, + }, expected_run_order=[ "joiner", "code_prompt", @@ -3029,11 +5224,13 @@ def run(self, **kwargs): PipelineRunData( inputs={"parrot": {"parrot": "Are you a parrot?"}}, expected_outputs={"parrot": {"response": "Are you a parrot?"}}, + expected_component_calls={("parrot", 1): {"parrot": "Are you a parrot?"}}, expected_run_order=["parrot"], ), PipelineRunData( inputs={}, expected_outputs={"parrot": {"response": "Parrot doesn't only parrot!"}}, + expected_component_calls={("parrot", 1): {"parrot": "Parrot doesn't only parrot!"}}, expected_run_order=["parrot"], ), ], @@ -3061,11 +5258,13 @@ def run(self, **kwargs): PipelineRunData( inputs={"parrot": {"parrot": "Are you a parrot?"}}, expected_outputs={"parrot": {"response": ["Are you a parrot?"]}}, + expected_component_calls={("parrot", 1): {"parrot": ["Are you a parrot?"]}}, expected_run_order=["parrot"], ), PipelineRunData( inputs={}, expected_outputs={"parrot": {"response": ["Parrot doesn't only parrot!"]}}, + expected_component_calls={("parrot", 1): {"parrot": ["Parrot doesn't only parrot!"]}}, expected_run_order=["parrot"], ), ], From cfdc3dc8fc9f4711d029b97b5b0811b50e0f082b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 12:30:02 +0100 Subject: [PATCH 49/88] test: remove testing for run order --- test/core/pipeline/features/conftest.py | 22 +- .../pipeline/features/pipeline_run.feature | 1 - test/core/pipeline/features/test_run.py | 309 ------------------ 3 files changed, 1 insertion(+), 331 deletions(-) diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index 4c1d7d5a1a..e1e411df1e 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -19,7 +19,6 @@ class PipelineRunData: inputs: Dict[str, Any] include_outputs_from: Set[str] = field(default_factory=set) expected_outputs: Dict[str, Any] = field(default_factory=dict) - expected_run_order: List[str] = field(default_factory=list) expected_component_calls: Dict[Tuple[str, int], Dict[str, Any]] = field(default_factory=dict) @@ -30,7 +29,6 @@ class _PipelineResult: """ outputs: Dict[str, Any] - run_order: List[str] component_calls: Dict[Tuple[str, int], Dict[str, Any]] = field(default_factory=dict) @@ -54,11 +52,6 @@ def run_pipeline( for data in pipeline_run_data: try: outputs = pipeline.run(data=data.inputs, include_outputs_from=data.include_outputs_from) - run_order = [ - span.tags["haystack.component.name"] - for span in spying_tracer.spans - if "haystack.component.name" in span.tags - ] component_calls = { (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ @@ -67,15 +60,8 @@ def run_pipeline( for span in spying_tracer.spans if "haystack.component.name" in span.tags and "haystack.component.visits" in span.tags } - results.append(_PipelineResult(outputs=outputs, run_order=run_order, component_calls=component_calls)) + results.append(_PipelineResult(outputs=outputs, component_calls=component_calls)) - component_calls = { - (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ - "haystack.component.input" - ] - for span in spying_tracer.spans - if "haystack.component.name" in span.tags and "haystack.component.visits" in span.tags - } spying_tracer.spans.clear() except Exception as e: return e @@ -107,12 +93,6 @@ def check_component_calls(pipeline_result: List[Tuple[_PipelineResult, PipelineR assert res.component_calls == data.expected_component_calls -@then("components ran in the expected order") -def check_pipeline_run_order(pipeline_result: List[Tuple[_PipelineResult, PipelineRunData]]): - for res, data in pipeline_result: - assert res.run_order == data.expected_run_order - - @then(parsers.parse("it must have raised {exception_class_name}")) def check_pipeline_raised(pipeline_result: Exception, exception_class_name: str): assert pipeline_result.__class__.__name__ == exception_class_name diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index a4b7fdce01..22375c7691 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -4,7 +4,6 @@ Feature: Pipeline running Given a pipeline When I run the Pipeline Then it should return the expected result - And components ran in the expected order And components are called with the expected inputs Examples: diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index e67f6c2050..c307fe0105 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -69,7 +69,6 @@ def pipeline_that_is_linear(): ("double", 1): {"value": 3}, ("second_addition", 1): {"value": 6, "add": None}, }, - expected_run_order=["first_addition", "double", "second_addition"], ) ], ) @@ -187,32 +186,6 @@ def pipeline_complex(): ("add_four", 1): {"value": -11, "add": None}, ("accumulate_3", 1): {"value": -7}, }, - expected_run_order=[ - "greet_first", - "greet_enumerator", - "accumulate_1", - "enumerate", - "add_two", - "add_three", - "parity", - "add_one", - "branch_joiner", - "below_10", - "double", - "branch_joiner", - "below_10", - "double", - "branch_joiner", - "below_10", - "accumulate_2", - "sum", - "diff", - "greet_one_last_time", - "replicate", - "add_five", - "add_four", - "accumulate_3", - ], ) ], ) @@ -236,13 +209,11 @@ def run(self, a: int, b: int = 2): inputs={"with_defaults": {"a": 40, "b": 30}}, expected_outputs={"with_defaults": {"c": 70}}, expected_component_calls={("with_defaults", 1): {"a": 40, "b": 30}}, - expected_run_order=["with_defaults"], ), PipelineRunData( inputs={"with_defaults": {"a": 40}}, expected_outputs={"with_defaults": {"c": 42}}, expected_component_calls={("with_defaults", 1): {"a": 40, "b": 2}}, - expected_run_order=["with_defaults"], ), ], ) @@ -268,13 +239,11 @@ def pipeline_that_has_two_loops_of_identical_lengths(): inputs={"branch_joiner": {"value": 0}}, expected_outputs={"remainder": {"remainder_is_0": 0}}, expected_component_calls={("branch_joiner", 1): {"value": [0]}, ("remainder", 1): {"value": 0}}, - expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 3}}, expected_outputs={"remainder": {"remainder_is_0": 3}}, expected_component_calls={("branch_joiner", 1): {"value": [3]}, ("remainder", 1): {"value": 3}}, - expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 4}}, @@ -286,7 +255,6 @@ def pipeline_that_has_two_loops_of_identical_lengths(): ("branch_joiner", 2): {"value": [6]}, ("remainder", 2): {"value": 6}, }, - expected_run_order=["branch_joiner", "remainder", "add_two", "branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 5}}, @@ -298,13 +266,11 @@ def pipeline_that_has_two_loops_of_identical_lengths(): ("branch_joiner", 2): {"value": [6]}, ("remainder", 2): {"value": 6}, }, - expected_run_order=["branch_joiner", "remainder", "add_one", "branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 6}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, expected_component_calls={("branch_joiner", 1): {"value": [6]}, ("remainder", 1): {"value": 6}}, - expected_run_order=["branch_joiner", "remainder"], ), ], ) @@ -333,13 +299,11 @@ def pipeline_that_has_two_loops_of_different_lengths(): inputs={"branch_joiner": {"value": 0}}, expected_outputs={"remainder": {"remainder_is_0": 0}}, expected_component_calls={("branch_joiner", 1): {"value": [0]}, ("remainder", 1): {"value": 0}}, - expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 3}}, expected_outputs={"remainder": {"remainder_is_0": 3}}, expected_component_calls={("branch_joiner", 1): {"value": [3]}, ("remainder", 1): {"value": 3}}, - expected_run_order=["branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 4}}, @@ -352,14 +316,6 @@ def pipeline_that_has_two_loops_of_different_lengths(): ("branch_joiner", 2): {"value": [6]}, ("remainder", 2): {"value": 6}, }, - expected_run_order=[ - "branch_joiner", - "remainder", - "add_two_1", - "add_two_2", - "branch_joiner", - "remainder", - ], ), PipelineRunData( inputs={"branch_joiner": {"value": 5}}, @@ -371,13 +327,11 @@ def pipeline_that_has_two_loops_of_different_lengths(): ("branch_joiner", 2): {"value": [6]}, ("remainder", 2): {"value": 6}, }, - expected_run_order=["branch_joiner", "remainder", "add_one", "branch_joiner", "remainder"], ), PipelineRunData( inputs={"branch_joiner": {"value": 6}}, expected_outputs={"remainder": {"remainder_is_0": 6}}, expected_component_calls={("branch_joiner", 1): {"value": [6]}, ("remainder", 1): {"value": 6}}, - expected_run_order=["branch_joiner", "remainder"], ), ], ) @@ -426,21 +380,6 @@ def pipeline_that_has_a_single_loop_with_two_conditional_branches(): ("branch_joiner", 2): {"value": [4]}, ("branch_joiner", 3): {"value": [11]}, }, - expected_run_order=[ - "add_one", - "branch_joiner", - "below_10", - "accumulator", - "below_5", - "branch_joiner", - "below_10", - "accumulator", - "below_5", - "add_three", - "branch_joiner", - "below_10", - "add_two", - ], ) ], ) @@ -466,7 +405,6 @@ def pipeline_that_has_a_component_with_dynamic_inputs_defined_in_init(): ("hello", 1): {"word": "Alice"}, ("splitter", 1): {"sentence": "This is the greeting: Hello, Alice!!"}, }, - expected_run_order=["hello", "fstring", "splitter"], ), PipelineRunData( inputs={"hello": {"word": "Alice"}, "fstring": {"template": "Received: {greeting}"}}, @@ -476,7 +414,6 @@ def pipeline_that_has_a_component_with_dynamic_inputs_defined_in_init(): ("hello", 1): {"word": "Alice"}, ("splitter", 1): {"sentence": "Received: Hello, Alice!"}, }, - expected_run_order=["hello", "fstring", "splitter"], ), ], ) @@ -508,7 +445,6 @@ def pipeline_that_has_two_branches_that_dont_merge(): ("add_three", 1): {"add": None, "value": 12}, ("parity", 1): {"value": 2}, }, - expected_run_order=["add_one", "parity", "add_ten", "add_three"], ), PipelineRunData( inputs={"add_one": {"value": 2}}, @@ -518,7 +454,6 @@ def pipeline_that_has_two_branches_that_dont_merge(): ("double", 1): {"value": 3}, ("parity", 1): {"value": 3}, }, - expected_run_order=["add_one", "parity", "double"], ), ], ) @@ -554,7 +489,6 @@ def pipeline_that_has_three_branches_that_dont_merge(): ("double", 1): {"value": 2}, ("repeat", 1): {"value": 2}, }, - expected_run_order=["add_one", "repeat", "add_ten", "double", "add_three", "add_one_again"], ) ], ) @@ -586,7 +520,6 @@ def pipeline_that_has_two_branches_that_merge(): ("second_addition", 1): {"add": None, "value": 3}, ("third_addition", 1): {"add": None, "value": 1}, }, - expected_run_order=["first_addition", "third_addition", "second_addition", "diff", "fourth_addition"], ) ], ) @@ -628,7 +561,6 @@ def pipeline_that_has_different_combinations_of_branches_that_merge_and_do_not_m ("add_two_as_well", 1): {"add": 2, "value": 6}, ("parity", 1): {"value": 2}, }, - expected_run_order=["add_one", "parity", "add_four", "add_two", "add_two_as_well"], ), PipelineRunData( inputs={"add_one": {"value": 2}, "add_two": {"add": 2}, "add_two_as_well": {"add": 2}}, @@ -640,7 +572,6 @@ def pipeline_that_has_different_combinations_of_branches_that_merge_and_do_not_m ("double", 1): {"value": 3}, ("parity", 1): {"value": 3}, }, - expected_run_order=["add_one", "parity", "double", "add_ten", "diff"], ), ], ) @@ -686,21 +617,6 @@ def pipeline_that_has_two_branches_one_of_which_loops_back(): ("counter", 2): {"value": 10}, ("sum", 1): {"values": [2, 21]}, }, - expected_run_order=[ - "add_zero", - "branch_joiner", - "below_10", - "add_one", - "counter", - "branch_joiner", - "below_10", - "add_one", - "counter", - "branch_joiner", - "below_10", - "add_two", - "sum", - ], ) ], ) @@ -740,7 +656,6 @@ def run(self, input_list: List[str]): ("mangler1", 1): {"input_list": ["foo", "bar"]}, ("mangler2", 1): {"input_list": ["foo", "bar"]}, }, - expected_run_order=["mangler1", "mangler2", "concat1", "concat2"], ) ], ) @@ -868,7 +783,6 @@ def run(self, messages: List[ChatMessage]): ] }, }, - expected_run_order=["prompt_builder", "llm", "mm1", "mm2"], ) ], ) @@ -948,7 +862,6 @@ def pipeline_that_has_a_greedy_and_variadic_component_after_a_component_with_def "top_k": None, }, }, - expected_run_order=["retriever", "branch_joiner", "prompt_builder"], ) ], ) @@ -1077,7 +990,6 @@ def pipeline_that_has_a_component_with_only_default_inputs(): "top_k": None, }, }, - expected_run_order=["retriever", "prompt_builder", "generator", "answer_builder"], ) ], ) @@ -1185,7 +1097,6 @@ def fake_generator_run(self, generation_kwargs: Optional[Dict[str, Any]] = None, ("router", 1): {"replies": ["Paris"]}, ("router", 2): {"replies": ["Rome"]}, }, - expected_run_order=["prompt_builder", "generator", "router", "prompt_builder", "generator", "router"], ) ], ) @@ -1227,7 +1138,6 @@ def pipeline_that_has_multiple_branches_that_merge_into_a_component_with_a_singl ("parity", 1): {"value": 2}, ("sum", 1): {"values": [2, 12]}, }, - expected_run_order=["add_one", "parity", "add_ten", "sum"], ), PipelineRunData( inputs={"add_one": {"value": 2}}, @@ -1240,7 +1150,6 @@ def pipeline_that_has_multiple_branches_that_merge_into_a_component_with_a_singl ("parity", 1): {"value": 3}, ("sum", 1): {"values": [3, 6, 8]}, }, - expected_run_order=["add_one", "parity", "double", "add_four", "add_one_again", "sum"], ), ], ) @@ -1277,7 +1186,6 @@ def pipeline_that_has_multiple_branches_of_different_lengths_that_merge_into_a_c ("sum", 1): {"values": [3, 3, 5]}, ("third_addition", 1): {"add": None, "value": 1}, }, - expected_run_order=["first_addition", "third_addition", "second_addition", "sum", "fourth_addition"], ) ], ) @@ -1308,7 +1216,6 @@ def pipeline_that_is_linear_and_returns_intermediate_outputs(): ("first_addition", 1): {"add": None, "value": 1}, ("second_addition", 1): {"add": None, "value": 6}, }, - expected_run_order=["first_addition", "double", "second_addition"], ), PipelineRunData( inputs={"first_addition": {"value": 1}}, @@ -1319,7 +1226,6 @@ def pipeline_that_is_linear_and_returns_intermediate_outputs(): ("first_addition", 1): {"add": None, "value": 1}, ("second_addition", 1): {"add": None, "value": 6}, }, - expected_run_order=["first_addition", "double", "second_addition"], ), ], ) @@ -1383,21 +1289,6 @@ def pipeline_that_has_a_loop_and_returns_intermediate_outputs_from_it(): ("branch_joiner", 2): {"value": [4]}, ("branch_joiner", 3): {"value": [11]}, }, - expected_run_order=[ - "add_one", - "branch_joiner", - "below_10", - "accumulator", - "below_5", - "branch_joiner", - "below_10", - "accumulator", - "below_5", - "add_three", - "branch_joiner", - "below_10", - "add_two", - ], ) ], ) @@ -1440,7 +1331,6 @@ def run(self, value: int): ("first_addition", 1): {"add": None, "value": 1}, ("second_addition", 1): {"add": None, "value": 6}, }, - expected_run_order=["first_addition", "double", "second_addition"], ), PipelineRunData( inputs={"first_addition": {"value": 1}}, @@ -1451,7 +1341,6 @@ def run(self, value: int): ("first_addition", 1): {"add": None, "value": 1}, ("second_addition", 1): {"add": None, "value": 6}, }, - expected_run_order=["first_addition", "double", "second_addition"], ), ], ) @@ -1480,7 +1369,6 @@ def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anythi inputs={"router": {"sentence": "Wir mussen reisen"}}, expected_outputs={"router": {"language_1": "German"}}, expected_component_calls={("router", 1): {"sentence": "Wir mussen reisen"}}, - expected_run_order=["router"], ), PipelineRunData( inputs={"router": {"sentence": "Yo tengo que viajar"}}, @@ -1489,7 +1377,6 @@ def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anythi ("pb", 1): {"language": "Spanish", "template": None, "template_variables": None}, ("router", 1): {"sentence": "Yo tengo que viajar"}, }, - expected_run_order=["router", "pb"], ), ], ) @@ -1612,7 +1499,6 @@ def run(self, query: str): "replies": ["There's simply no_answer to this question"], }, }, - expected_run_order=["prompt", "llm", "router", "fallback_prompt", "fallback_llm"], ) ], [ @@ -1720,7 +1606,6 @@ def run(self, query: str): "template_variables": {"comment": "I loved the quality of the meal but the courier was rude"}, }, }, - expected_run_order=["prompt", "llm", "router", "sql_querier"], ) ], ) @@ -1896,14 +1781,6 @@ def run(self, prompt: str): "template_variables": {"comment": "I loved the quality of the meal but the courier was rude"}, }, }, - expected_run_order=[ - "prompt_builder", - "llm", - "output_validator", - "prompt_builder", - "llm", - "output_validator", - ], ) ], ) @@ -2071,15 +1948,6 @@ def run(self, prompt: str, generation_kwargs: Optional[Dict[str, Any]] = None): " ", }, }, - expected_run_order=[ - "prompt_builder1", - "spellchecker", - "prompt_builder3", - "retriever", - "ranker", - "prompt_builder2", - "llm", - ], ) ], ) @@ -2182,20 +2050,10 @@ def run(self, query_embedding: List[float]): "top_k": None, }, }, - expected_run_order=[ - "router", - "text_embedder", - "bm25retriever", - "retriever", - "joinerhybrid", - "ranker", - "joinerfinal", - ], ), PipelineRunData( inputs={"router": {"query": "I'm a nasty prompt injection"}}, expected_outputs={"joinerfinal": {"documents": []}}, - expected_run_order=["router", "emptyretriever", "joinerfinal"], expected_component_calls={ ("router", 1): {"query": "I'm a nasty prompt injection"}, ("emptyretriever", 1): {"query": "I'm a nasty prompt injection"}, @@ -2644,25 +2502,6 @@ def run(self, replies: List[ChatMessage]): ] }, }, - expected_run_order=[ - "main_input", - "prompt_builder", - "llm", - "prompt_concatenator_after_action", - "tool_extractor", - "router", - "router_search", - "search_prompt_builder", - "search_llm", - "search_output_adapter", - "prompt_concatenator_after_observation", - "main_input", - "prompt_builder", - "llm", - "prompt_concatenator_after_action", - "tool_extractor", - "router", - ], ) ] @@ -2716,7 +2555,6 @@ def run(self, create_document: bool = False): ("second_creator", 1): {"create_document": False}, ("third_creator", 1): {"create_document": True}, }, - expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), PipelineRunData( inputs={"first_creator": {"create_document": True}, "second_creator": {"create_document": True}}, @@ -2741,7 +2579,6 @@ def run(self, create_document: bool = False): ("second_creator", 1): {"create_document": True}, ("third_creator", 1): {"create_document": False}, }, - expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), ], ) @@ -2799,7 +2636,6 @@ def run(self, create_document: bool = False): ("second_creator", 1): {"create_document": False}, ("third_creator", 1): {"create_document": True}, }, - expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), PipelineRunData( inputs={"first_creator": {"create_document": True}, "second_creator": {"create_document": True}}, @@ -2824,7 +2660,6 @@ def run(self, create_document: bool = False): ("second_creator", 1): {"create_document": True}, ("third_creator", 1): {"create_document": False}, }, - expected_run_order=["first_creator", "second_creator", "third_creator", "documents_joiner"], ), ], ) @@ -2907,7 +2742,6 @@ def that_has_an_answer_joiner_variadic_component(): "top_k": None, }, }, - expected_run_order=["answer_builder_1", "answer_builder_2", "answer_joiner"], ) ], ) @@ -3012,7 +2846,6 @@ def run(self, prompt: str): "top_k": None, }, }, - expected_run_order=["builder", "metadata_extractor", "retriever", "document_joiner"], ) ], ) @@ -3174,16 +3007,6 @@ def run(self, prompt: str): }, ("prompt_cleaner", 1): {"prompt": "Random template"}, }, - expected_run_order=[ - "prompt_cleaner", - "prompt_builder", - "llm", - "answer_validator", - "prompt_builder", - "llm", - "answer_validator", - "answer_builder", - ], ) ], ) @@ -3326,14 +3149,6 @@ def run(self, documents: List[Document]): ] }, }, - expected_run_order=[ - "comma_splitter", - "noop2", - "document_cleaner", - "noop3", - "conditional_router", - "document_joiner", - ], ), PipelineRunData( inputs={ @@ -3459,15 +3274,6 @@ def run(self, documents: List[Document]): ] }, }, - expected_run_order=[ - "comma_splitter", - "noop2", - "document_cleaner", - "noop3", - "conditional_router", - "empty_lines_cleaner", - "document_joiner", - ], ), ] @@ -3506,7 +3312,6 @@ def that_has_a_string_variadic_component(): "strings": ["Builder 1: What's Natural Language Processing?", "Builder 2: What's is life?"] }, }, - expected_run_order=["prompt_builder_1", "prompt_builder_2", "string_joiner"], ) ], ) @@ -3755,20 +3560,6 @@ def run(self, query: str): ("router", 1): {"replies": ["search: Can you help me?"]}, ("router", 2): {"replies": ["answer: here is my answer"]}, }, - expected_run_order=[ - "agent_prompt", - "joiner", - "agent_llm", - "router", - "retriever", - "rag_prompt", - "rag_llm", - "concatenator", - "joiner", - "agent_llm", - "router", - "answer_builder", - ], ) ], ) @@ -3933,20 +3724,6 @@ def run(self, prompt: str): ("router", 1): {"code": ["invalid code"], "replies": ["FAIL"]}, ("router", 2): {"code": ["valid code"], "replies": ["PASS"]}, }, - expected_run_order=[ - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "concatenator", - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "answer_builder", - ], ) ], ) @@ -4112,20 +3889,6 @@ def run(self, prompt: str): ("router", 1): {"code": ["invalid code"], "replies": ["FAIL"]}, ("router", 2): {"code": ["valid code"], "replies": ["PASS"]}, }, - expected_run_order=[ - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "concatenator", - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "answer_builder", - ], ) ], ) @@ -4860,51 +4623,6 @@ def run(self, replies: List[str]): ("tool_use_router", 7): {"replies": ["Edit: file_3.py"]}, ("tool_use_router", 8): {"replies": ["Task finished!"]}, }, - expected_run_order=[ - "code_prompt", - "joiner", - "code_llm", - "tool_use_router", - "file_editor", - "agent_concatenator", - "joiner", - "code_llm", - "tool_use_router", - "file_editor", - "agent_concatenator", - "joiner", - "code_llm", - "tool_use_router", - "file_editor", - "agent_concatenator", - "joiner", - "code_llm", - "tool_use_router", - "feedback_prompt", - "feedback_llm", - "feedback_router", - "joiner", - "code_llm", - "tool_use_router", - "file_editor", - "agent_concatenator", - "joiner", - "code_llm", - "tool_use_router", - "file_editor", - "agent_concatenator", - "joiner", - "code_llm", - "tool_use_router", - "file_editor", - "agent_concatenator", - "joiner", - "code_llm", - "tool_use_router", - "feedback_prompt", - "feedback_llm", - "feedback_router", - ], ) ], ) @@ -5175,29 +4893,6 @@ def generate_santa_sleigh(): "replies": ["PASS"], }, }, - expected_run_order=[ - "joiner", - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "concatenator", - "joiner", - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "concatenator", - "joiner", - "code_prompt", - "code_llm", - "feedback_prompt", - "feedback_llm", - "router", - "answer_builder", - ], ) ], ) @@ -5225,13 +4920,11 @@ def run(self, **kwargs): inputs={"parrot": {"parrot": "Are you a parrot?"}}, expected_outputs={"parrot": {"response": "Are you a parrot?"}}, expected_component_calls={("parrot", 1): {"parrot": "Are you a parrot?"}}, - expected_run_order=["parrot"], ), PipelineRunData( inputs={}, expected_outputs={"parrot": {"response": "Parrot doesn't only parrot!"}}, expected_component_calls={("parrot", 1): {"parrot": "Parrot doesn't only parrot!"}}, - expected_run_order=["parrot"], ), ], ) @@ -5259,13 +4952,11 @@ def run(self, **kwargs): inputs={"parrot": {"parrot": "Are you a parrot?"}}, expected_outputs={"parrot": {"response": ["Are you a parrot?"]}}, expected_component_calls={("parrot", 1): {"parrot": ["Are you a parrot?"]}}, - expected_run_order=["parrot"], ), PipelineRunData( inputs={}, expected_outputs={"parrot": {"response": ["Parrot doesn't only parrot!"]}}, expected_component_calls={("parrot", 1): {"parrot": ["Parrot doesn't only parrot!"]}}, - expected_run_order=["parrot"], ), ], ) From 25a4b40e412cd65a163208249327c039426dd7ff Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 12:32:58 +0100 Subject: [PATCH 50/88] chore: update component checks from experimental --- haystack/core/pipeline/component_checks.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index ee6bf15deb..7986b08a64 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -73,10 +73,10 @@ def are_all_sockets_ready(component: Dict, inputs: Dict, only_check_mandatory: b for socket_name, socket in sockets_to_check.items(): socket_inputs = inputs.get(socket_name, []) expected_sockets.add(socket_name) - if ( - is_socket_lazy_variadic(socket) - and any_socket_input_received(socket_inputs) - or has_socket_received_all_inputs(socket, socket_inputs) + + # Check if socket has all required inputs or is a lazy variadic socket with any input + if has_socket_received_all_inputs(socket, socket_inputs) or ( + is_socket_lazy_variadic(socket) and any_socket_input_received(socket_inputs) ): filled_sockets.add(socket_name) @@ -185,7 +185,6 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict if ( socket.is_variadic and socket.is_greedy - and len(socket_inputs) > 0 and any(sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs) ): return True @@ -221,6 +220,12 @@ def are_all_lazy_variadic_sockets_resolved(component: Dict, inputs: Dict) -> boo for socket_name, socket in component["input_sockets"].items(): if is_socket_lazy_variadic(socket): socket_inputs = inputs.get(socket_name, []) + + # Checks if a lazy variadic socket is ready to run. + # A socket is ready if either: + # - it has received all expected inputs, or + # - all its predecessors have executed + # If none of the conditions are met, the socket is not ready to run and we defer the component. if not ( has_lazy_variadic_socket_received_all_inputs(socket, socket_inputs) or all_socket_predecessors_executed(socket, socket_inputs) From 74657cdecfa5d8e4c69df20a85d206f5aa911d85 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 14:14:09 +0100 Subject: [PATCH 51/88] chore: update pipeline and base from experimental --- haystack/core/pipeline/base.py | 350 ++++- haystack/core/pipeline/component_checks.py | 6 +- haystack/core/pipeline/pipeline.py | 345 +---- test/core/pipeline/test_pipeline.py | 1634 +------------------ test/core/pipeline/test_pipeline_base.py | 1637 ++++++++++++++++++++ test/test_files/yaml/test_pipeline.yaml | 4 +- 6 files changed, 1974 insertions(+), 2002 deletions(-) create mode 100644 test/core/pipeline/test_pipeline_base.py diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 5c509392bc..9794433711 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -6,11 +6,10 @@ from collections import defaultdict from copy import deepcopy from datetime import datetime +from enum import IntEnum from pathlib import Path from typing import Any, Dict, Iterator, List, Optional, TextIO, Tuple, Type, TypeVar, Union -import networkx # type:ignore - from haystack import logging from haystack.core.component import Component, InputSocket, OutputSocket, component from haystack.core.errors import ( @@ -18,22 +17,35 @@ PipelineConnectError, PipelineDrawingError, PipelineError, + PipelineMaxComponentRuns, + PipelineRuntimeError, PipelineUnmarshalError, PipelineValidationError, ) +from haystack.core.pipeline.descriptions import find_pipeline_inputs, find_pipeline_outputs +from haystack.core.pipeline.draw import _to_mermaid_image +from haystack.core.pipeline.template import PipelineTemplate, PredefinedPipeline +from haystack.core.pipeline.utils import parse_connect_string from haystack.core.serialization import DeserializationCallbacks, component_from_dict, component_to_dict from haystack.core.type_utils import _type_name, _types_are_compatible from haystack.marshal import Marshaller, YamlMarshaller from haystack.utils import is_in_jupyter, type_serialization - -from .descriptions import find_pipeline_inputs, find_pipeline_outputs -from .draw import _to_mermaid_image -from .template import PipelineTemplate, PredefinedPipeline -from .utils import parse_connect_string +from networkx import MultiDiGraph # type:ignore + +from haystack.core.pipeline.component_checks import ( + _NO_OUTPUT_PRODUCED, + all_predecessors_executed, + are_all_lazy_variadic_sockets_resolved, + are_all_sockets_ready, + can_component_run, + is_any_greedy_socket_ready, + is_socket_lazy_variadic, +) +from haystack.core.pipeline.utils import FIFOPriorityQueue DEFAULT_MARSHALLER = YamlMarshaller() -# We use a generic type to annotate the return value of class methods, +# We use a generic type to annotate the return value of classmethods, # so that static analyzers won't be confused when derived classes # use those methods. T = TypeVar("T", bound="PipelineBase") @@ -41,6 +53,14 @@ logger = logging.getLogger(__name__) +class ComponentPriority(IntEnum): + HIGHEST = 1 + READY = 2 + DEFER = 3 + DEFER_LAST = 4 + BLOCKED = 5 + + class PipelineBase: """ Components orchestration engine. @@ -63,7 +83,7 @@ def __init__(self, metadata: Optional[Dict[str, Any]] = None, max_runs_per_compo self._telemetry_runs = 0 self._last_telemetry_sent: Optional[datetime] = None self.metadata = metadata or {} - self.graph = networkx.MultiDiGraph() + self.graph = MultiDiGraph() self._max_runs_per_component = max_runs_per_component def __eq__(self, other) -> bool: @@ -125,7 +145,7 @@ def to_dict(self) -> Dict[str, Any]: } @classmethod - def from_dict( + def from_dict( # noqa: PLR0912 cls: Type[T], data: Dict[str, Any], callbacks: Optional[DeserializationCallbacks] = None, **kwargs ) -> T: """ @@ -166,10 +186,8 @@ def from_dict( f"Successfully imported module {module} but can't find it in the component registry." "This is unexpected and most likely a bug." ) - except (ImportError, PipelineError, ValueError) as e: - raise PipelineError( - f"Component '{component_data['type']}' (name: '{name}') not imported." - ) from e + except (ImportError, PipelineError) as e: + raise PipelineError(f"Component '{component_data['type']}' not imported.") from e # Create a new one component_class = component.registry[component_data["type"]] @@ -368,7 +386,7 @@ def remove_component(self, name: str) -> Component: return instance - def connect(self, sender: str, receiver: str) -> "PipelineBase": # noqa: PLR0915 + def connect(self, sender: str, receiver: str) -> "PipelineBase": # noqa: PLR0915 PLR0912 """ Connects two components together. @@ -618,76 +636,31 @@ def outputs(self, include_components_with_connected_outputs: bool = False) -> Di } return outputs - def show(self, server_url: str = "https://mermaid.ink", params: Optional[dict] = None) -> None: + def show(self) -> None: """ - Display an image representing this `Pipeline` in a Jupyter notebook. + If running in a Jupyter notebook, display an image representing this `Pipeline`. - This function generates a diagram of the `Pipeline` using a Mermaid server and displays it directly in - the notebook. - - :param server_url: - The base URL of the Mermaid server used for rendering (default: 'https://mermaid.ink'). - See https://github.com/jihchi/mermaid.ink and https://github.com/mermaid-js/mermaid-live-editor for more - info on how to set up your own Mermaid server. - - :param params: - Dictionary of customization parameters to modify the output. Refer to Mermaid documentation for more details - Supported keys: - - format: Output format ('img', 'svg', or 'pdf'). Default: 'img'. - - type: Image type for /img endpoint ('jpeg', 'png', 'webp'). Default: 'png'. - - theme: Mermaid theme ('default', 'neutral', 'dark', 'forest'). Default: 'neutral'. - - bgColor: Background color in hexadecimal (e.g., 'FFFFFF') or named format (e.g., '!white'). - - width: Width of the output image (integer). - - height: Height of the output image (integer). - - scale: Scaling factor (1–3). Only applicable if 'width' or 'height' is specified. - - fit: Whether to fit the diagram size to the page (PDF only, boolean). - - paper: Paper size for PDFs (e.g., 'a4', 'a3'). Ignored if 'fit' is true. - - landscape: Landscape orientation for PDFs (boolean). Ignored if 'fit' is true. - - :raises PipelineDrawingError: - If the function is called outside of a Jupyter notebook or if there is an issue with rendering. """ if is_in_jupyter(): from IPython.display import Image, display # type: ignore - image_data = _to_mermaid_image(self.graph, server_url=server_url, params=params) + image_data = _to_mermaid_image(self.graph) + display(Image(image_data)) else: msg = "This method is only supported in Jupyter notebooks. Use Pipeline.draw() to save an image locally." raise PipelineDrawingError(msg) - def draw(self, path: Path, server_url: str = "https://mermaid.ink", params: Optional[dict] = None) -> None: + def draw(self, path: Path) -> None: """ - Save an image representing this `Pipeline` to the specified file path. - - This function generates a diagram of the `Pipeline` using the Mermaid server and saves it to the provided path. + Save an image representing this `Pipeline` to `path`. :param path: - The file path where the generated image will be saved. - :param server_url: - The base URL of the Mermaid server used for rendering (default: 'https://mermaid.ink'). - See https://github.com/jihchi/mermaid.ink and https://github.com/mermaid-js/mermaid-live-editor for more - info on how to set up your own Mermaid server. - :param params: - Dictionary of customization parameters to modify the output. Refer to Mermaid documentation for more details - Supported keys: - - format: Output format ('img', 'svg', or 'pdf'). Default: 'img'. - - type: Image type for /img endpoint ('jpeg', 'png', 'webp'). Default: 'png'. - - theme: Mermaid theme ('default', 'neutral', 'dark', 'forest'). Default: 'neutral'. - - bgColor: Background color in hexadecimal (e.g., 'FFFFFF') or named format (e.g., '!white'). - - width: Width of the output image (integer). - - height: Height of the output image (integer). - - scale: Scaling factor (1–3). Only applicable if 'width' or 'height' is specified. - - fit: Whether to fit the diagram size to the page (PDF only, boolean). - - paper: Paper size for PDFs (e.g., 'a4', 'a3'). Ignored if 'fit' is true. - - landscape: Landscape orientation for PDFs (boolean). Ignored if 'fit' is true. - - :raises PipelineDrawingError: - If there is an issue with rendering or saving the image. + The path to save the image to. """ # Before drawing we edit a bit the graph, to avoid modifying the original that is # used for running the pipeline we copy it. - image_data = _to_mermaid_image(self.graph, server_url=server_url, params=params) + image_data = _to_mermaid_image(self.graph) Path(path).write_bytes(image_data) def walk(self) -> Iterator[Tuple[str, Component]]: @@ -861,6 +834,247 @@ def _find_receivers_from(self, component_name: str) -> List[Tuple[str, OutputSoc res.append((receiver_name, sender_socket, receiver_socket)) return res + @staticmethod + def _convert_to_internal_format(pipeline_inputs: Dict[str, Any]) -> Dict[str, Dict[str, List]]: + """ + Converts the inputs to the pipeline to the format that is needed for the internal `Pipeline.run` logic. + + Example Input: + {'prompt_builder': {'question': 'Who lives in Paris?'}, 'retriever': {'query': 'Who lives in Paris?'}} + Example Output: + {'prompt_builder': {'question': [{'sender': None, 'value': 'Who lives in Paris?'}]}, + 'retriever': {'query': [{'sender': None, 'value': 'Who lives in Paris?'}]}} + + :param pipeline_inputs: Inputs to the pipeline. + :returns: Converted inputs that can be used by the internal `Pipeline.run` logic. + """ + inputs: Dict[str, Dict[str, List[Dict[str, Any]]]] = {} + for component_name, socket_dict in pipeline_inputs.items(): + inputs[component_name] = {} + for socket_name, value in socket_dict.items(): + inputs[component_name][socket_name] = [{"sender": None, "value": value}] + + return inputs + + @staticmethod + def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict) -> Tuple[Dict, Dict]: + """ + Extracts the inputs needed to run for the component and removes them from the global inputs state. + + :param component_name: The name of a component. + :param component: Component with component metadata. + :param inputs: Global inputs state. + :returns: The inputs for the component and the new state of global inputs. + """ + component_inputs = inputs.get(component_name, {}) + consumed_inputs = {} + greedy_inputs_to_remove = set() + for socket_name, socket in component["input_sockets"].items(): + socket_inputs = component_inputs.get(socket_name, []) + socket_inputs = [sock["value"] for sock in socket_inputs if sock["value"] != _NO_OUTPUT_PRODUCED] + if socket_inputs: + if not socket.is_variadic: + # We only care about the first input provided to the socket. + consumed_inputs[socket_name] = socket_inputs[0] + elif socket.is_greedy: + # We need to keep track of greedy inputs because we always remove them, even if they come from + # outside the pipeline. Otherwise, a greedy input from the user would trigger a pipeline to run + # indefinitely. + greedy_inputs_to_remove.add(socket_name) + consumed_inputs[socket_name] = [socket_inputs[0]] + elif is_socket_lazy_variadic(socket): + # We use all inputs provided to the socket on a lazy variadic socket. + consumed_inputs[socket_name] = socket_inputs + + # We prune all inputs except for those that were provided from outside the pipeline (e.g. user inputs). + pruned_inputs = { + socket_name: [ + sock for sock in socket if sock["sender"] is None and not socket_name in greedy_inputs_to_remove + ] + for socket_name, socket in component_inputs.items() + } + pruned_inputs = {socket_name: socket for socket_name, socket in pruned_inputs.items() if len(socket) > 0} + + inputs[component_name] = pruned_inputs + + return consumed_inputs, inputs + + def _fill_queue( + self, component_names: List[str], inputs: Dict[str, Any], component_visits: Dict[str, int] + ) -> FIFOPriorityQueue: + """ + Calculates the execution priority for each component and inserts it into the priority queue. + + :param component_names: Names of the components to put into the queue. + :param inputs: Inputs to the components. + :param component_visits: Current state of component visits. + :returns: A prioritized queue of component names. + """ + priority_queue = FIFOPriorityQueue() + for component_name in component_names: + component = self._get_component_with_graph_metadata_and_visits( + component_name, component_visits[component_name] + ) + priority = self._calculate_priority(component, inputs.get(component_name, {})) + priority_queue.push(component_name, priority) + + return priority_queue + + @staticmethod + def _calculate_priority(component: Dict, inputs: Dict) -> ComponentPriority: + """ + Calculates the execution priority for a component depending on the component's inputs. + + :param component: Component metadata and component instance. + :param inputs: Inputs to the component. + :returns: Priority value for the component. + """ + if not can_component_run(component, inputs): + return ComponentPriority.BLOCKED + elif is_any_greedy_socket_ready(component, inputs) and are_all_sockets_ready(component, inputs): + return ComponentPriority.HIGHEST + elif all_predecessors_executed(component, inputs): + return ComponentPriority.READY + elif are_all_lazy_variadic_sockets_resolved(component, inputs): + return ComponentPriority.DEFER + else: + return ComponentPriority.DEFER_LAST + + def _get_component_with_graph_metadata_and_visits(self, component_name: str, visits: int) -> Dict[str, Any]: + """ + Returns the component instance alongside input/output-socket metadata from the graph and adds current visits. + + We can't store visits in the pipeline graph because this would prevent reentrance / thread-safe execution. + + :param component_name: The name of the component. + :param visits: Number of visits for the component. + :returns: Dict including component instance, input/output-sockets and visits. + """ + comp_dict = self.graph.nodes[component_name] + comp_dict = {**comp_dict, "visits": visits} + return comp_dict + + def _get_next_runnable_component( + self, priority_queue: FIFOPriorityQueue, component_visits: Dict[str, int] + ) -> Union[Tuple[ComponentPriority, str, Dict[str, Any]], None]: + """ + Returns the next runnable component alongside its metadata from the priority queue. + + :param priority_queue: Priority queue of component names. + :param component_visits: Current state of component visits. + :returns: The next runnable component, the component name, and its priority + or None if no component in the queue can run. + :raises: PipelineMaxComponentRuns if the next runnable component has exceeded the maximum number of runs. + """ + priority_and_component_name: Union[Tuple[ComponentPriority, str], None] = ( + None if (item := priority_queue.get()) is None else (ComponentPriority(item[0]), str(item[1])) + ) + + if priority_and_component_name is not None and priority_and_component_name[0] != ComponentPriority.BLOCKED: + priority, component_name = priority_and_component_name + component = self._get_component_with_graph_metadata_and_visits( + component_name, component_visits[component_name] + ) + if component["visits"] > self._max_runs_per_component: + msg = f"Maximum run count {self._max_runs_per_component} reached for component '{component_name}'" + raise PipelineMaxComponentRuns(msg) + + return priority, component_name, component + + return None + + @staticmethod + def _add_missing_input_defaults(component_inputs: Dict[str, Any], component_input_sockets: Dict[str, InputSocket]): + """ + Updates the inputs with the default values for the inputs that are missing + + :param component_inputs: Inputs for the component. + :param component_input_sockets: Input sockets of the component. + """ + for name, socket in component_input_sockets.items(): + if not socket.is_mandatory and name not in component_inputs: + if socket.is_variadic: + component_inputs[name] = [socket.default_value] + else: + component_inputs[name] = socket.default_value + + return component_inputs + + @staticmethod + def _write_component_outputs( + component_name, component_outputs, inputs, receivers, include_outputs_from + ) -> Tuple[Dict, Dict]: + """ + Distributes the outputs of a component to the input sockets that it is connected to. + + :param component_name: The name of the component. + :param component_outputs: The outputs of the component. + :param inputs: The current global input state. + :param receivers: List of receiver_name, sender_socket, receiver_socket for connected components. + :param include_outputs_from: List of component names that should always return an output from the pipeline. + """ + for receiver_name, sender_socket, receiver_socket in receivers: + # We either get the value that was produced by the actor or we use the _NO_OUTPUT_PRODUCED class to indicate + # that the sender did not produce an output for this socket. + # This allows us to track if a pre-decessor already ran but did not produce an output. + value = component_outputs.get(sender_socket.name, _NO_OUTPUT_PRODUCED) + if receiver_name not in inputs: + inputs[receiver_name] = {} + + # If we have a non-variadic or a greedy variadic receiver socket, we can just overwrite any inputs + # that might already exist (to be reconsidered but mirrors current behavior). + if not is_socket_lazy_variadic(receiver_socket): + inputs[receiver_name][receiver_socket.name] = [{"sender": component_name, "value": value}] + + # If the receiver socket is lazy variadic, and it already has an input, we need to append the new input. + # Lazy variadic sockets can collect multiple inputs. + else: + if not inputs[receiver_name].get(receiver_socket.name): + inputs[receiver_name][receiver_socket.name] = [] + + inputs[receiver_name][receiver_socket.name].append({"sender": component_name, "value": value}) + + # If we want to include all outputs from this actor in the final outputs, we don't need to prune any consumed + # outputs + if component_name in include_outputs_from: + return component_outputs, inputs + + # We prune outputs that were consumed by any receiving sockets. + # All remaining outputs will be added to the final outputs of the pipeline. + consumed_outputs = {sender_socket.name for _, sender_socket, __ in receivers} + pruned_outputs = {key: value for key, value in component_outputs.items() if key not in consumed_outputs} + + return pruned_outputs, inputs + + @staticmethod + def _is_queue_stale(priority_queue: FIFOPriorityQueue) -> bool: + """ + Checks if the priority queue needs to be recomputed because the priorities might have changed. + + :param priority_queue: Priority queue of component names. + """ + return len(priority_queue) == 0 or priority_queue.peek()[0] > ComponentPriority.READY + + @staticmethod + def validate_pipeline(priority_queue: FIFOPriorityQueue) -> None: + """ + Validate the pipeline to check if it is blocked or has no valid entry point. + + :param priority_queue: Priority queue of component names. + """ + if len(priority_queue) == 0: + return + + candidate = priority_queue.peek() + if candidate is not None and candidate[0] == ComponentPriority.BLOCKED: + raise PipelineRuntimeError( + "Cannot run pipeline - all components are blocked. " + "This typically happens when:\n" + "1. There is no valid entry point for the pipeline\n" + "2. There is a circular dependency preventing the pipeline from running\n" + "Check the connections between these components and ensure all required inputs are provided." + ) + def _connections_status( sender_node: str, receiver_node: str, sender_sockets: List[OutputSocket], receiver_sockets: List[InputSocket] diff --git a/haystack/core/pipeline/component_checks.py b/haystack/core/pipeline/component_checks.py index 7986b08a64..b4d42a8ac0 100644 --- a/haystack/core/pipeline/component_checks.py +++ b/haystack/core/pipeline/component_checks.py @@ -182,11 +182,7 @@ def has_socket_received_all_inputs(socket: InputSocket, socket_inputs: List[Dict return False # The socket is greedy variadic and at least one input was produced, it is complete. - if ( - socket.is_variadic - and socket.is_greedy - and any(sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs) - ): + if socket.is_variadic and socket.is_greedy and any(sock["value"] != _NO_OUTPUT_PRODUCED for sock in socket_inputs): return True # The socket is lazy variadic and all expected inputs were produced. diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index d9445a904a..8bf59771ec 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -4,34 +4,16 @@ import warnings from copy import deepcopy -from enum import IntEnum -from typing import Any, Dict, List, Mapping, Optional, Set, Tuple, Union +from typing import Any, Dict, Mapping, Optional, Set, Tuple, cast from haystack import logging, tracing -from haystack.core.component import Component, InputSocket -from haystack.core.errors import PipelineMaxComponentRuns, PipelineRuntimeError -from haystack.core.pipeline.base import PipelineBase -from haystack.core.pipeline.component_checks import ( - _NO_OUTPUT_PRODUCED, - all_predecessors_executed, - are_all_lazy_variadic_sockets_resolved, - are_all_sockets_ready, - can_component_run, - is_any_greedy_socket_ready, - is_socket_lazy_variadic, -) -from haystack.core.pipeline.utils import FIFOPriorityQueue +from haystack.core.component import Component +from haystack.core.errors import PipelineRuntimeError from haystack.telemetry import pipeline_running -logger = logging.getLogger(__name__) - +from haystack.core.pipeline.base import ComponentPriority, PipelineBase -class ComponentPriority(IntEnum): - HIGHEST = 1 - READY = 2 - DEFER = 3 - DEFER_LAST = 4 - BLOCKED = 5 +logger = logging.getLogger(__name__) class Pipeline(PipelineBase): @@ -41,97 +23,19 @@ class Pipeline(PipelineBase): Orchestrates component execution according to the execution graph, one after the other. """ - def _warn_if_ambiguous_intent( - self, inputs: Dict[str, Any], component_names: List[str], receivers: Dict[str, Any] - ) -> None: - """ - Issues warnings if the running order of the pipeline is potentially ambiguous. - - We simulate a full pass through the pipeline where all components produce outputs. - At every step, we check if more than one component is waiting for optional inputs. - If two components wait for optional input with the same priority, the user intention for the execution - order of these components is not clear. - A warning does not mean that the running order must be ambiguous when real data flows through the pipeline. - Depending on the users data and business logic, the running order might still be clear, but we can not check - for this before running the pipeline. - - :param inputs: The inputs to the pipeline. - :param component_names: Names of all components in the pipeline. - :param receivers: The receivers for each component in the pipeline. - """ - inp_cpy = deepcopy(inputs) - remaining_components = set(component_names) - pq = self._fill_queue(component_names, inp_cpy) - - # Pipeline has no components. - if len(pq) == 0: - return - - while True: - candidate = pq.pop() - - # We don't have any components left that could run. - if candidate is None or candidate[0] == ComponentPriority.BLOCKED: - return - - priority, component_name = candidate - - # The queue is empty so the next component can't have the same priority as the current component. - if len(pq) == 0: - return - - # We get the next component and its priority to check if the current component and the next component are - # both waiting for inputs with the same priority. - next_prio, next_name = pq.peek() - if priority in [ComponentPriority.DEFER, ComponentPriority.DEFER_LAST] and next_prio == priority: - msg = ( - f"Ambiguous running order: Components '{component_name}' and '{next_name}' are waiting for " - f"optional inputs at the same time. Component '{component_name}' executes first." - ) - warnings.warn(msg) - - # We simulate output distribution for the current component by filling all its output sockets. - comp_with_metadata = self._get_component_with_graph_metadata(component_name) - component_outputs = { - socket_name: "simulation" for socket_name in comp_with_metadata["output_sockets"].keys() - } - comp_receivers = receivers[component_name] - _, inp_cpy = self._write_component_outputs( - component_name, component_outputs, inp_cpy, comp_receivers, set() - ) - - # We need to remove the component that we just checked so that we don't get into an infinite loop. - remaining_components.remove(component_name) - - # We re-prioritize the queue to capture if any components changed priority after simulating a run for - # the current component. - pq = self._fill_queue(remaining_components, inp_cpy) - - @staticmethod - def _add_missing_input_defaults(component_inputs: Dict[str, Any], component_input_sockets: Dict[str, InputSocket]): - """ - Updates the inputs with the default values for the inputs that are missing - - :param component_inputs: Inputs for the component. - :param component_input_sockets: Input sockets of the component. - """ - for name, socket in component_input_sockets.items(): - if not socket.is_mandatory and name not in component_inputs: - if socket.is_variadic: - component_inputs[name] = [socket.default_value] - else: - component_inputs[name] = socket.default_value - - return component_inputs - def _run_component( - self, component: Dict[str, Any], inputs: Dict[str, Any], parent_span: Optional[tracing.Span] = None + self, + component: Dict[str, Any], + inputs: Dict[str, Any], + component_visits: Dict[str, int], + parent_span: Optional[tracing.Span] = None, ) -> Tuple[Dict, Dict]: """ Runs a Component with the given inputs. :param component: Component with component metadata. :param inputs: Inputs for the Component. + :param component_visits: Current state of component visits. :param parent_span: The parent span to use for the newly created span. This is to allow tracing to be correctly linked to the pipeline run. :raises PipelineRuntimeError: If Component doesn't return a dictionary. @@ -175,7 +79,7 @@ def _run_component( span.set_content_tag("haystack.component.input", deepcopy(component_inputs)) logger.info("Running component {component_name}", component_name=component_name) component_output = instance.run(**component_inputs) - component["visits"] += 1 + component_visits[component_name] += 1 if not isinstance(component_output, Mapping): raise PipelineRuntimeError( @@ -183,177 +87,10 @@ def _run_component( "Components must always return dictionaries: check the documentation." ) - span.set_tag("haystack.component.visits", component["visits"]) + span.set_tag("haystack.component.visits", component_visits[component_name]) span.set_content_tag("haystack.component.output", component_output) - return component_output, inputs - - @staticmethod - def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict) -> Tuple[Dict, Dict]: - """ - Extracts the inputs needed to run for the component and removes them from the global inputs state. - - :param component: Component with component metadata. - :param inputs: Global inputs state. - :returns: The inputs for the component and the new state of global inputs. - """ - component_inputs = inputs.get(component_name, {}) - consumed_inputs = {} - greedy_inputs_to_remove = set() - for socket_name, socket in component["input_sockets"].items(): - socket_inputs = component_inputs.get(socket_name, []) - socket_inputs = [sock["value"] for sock in socket_inputs if sock["value"] != _NO_OUTPUT_PRODUCED] - if socket_inputs: - if not socket.is_variadic: - # We only care about the first input provided to the socket. - consumed_inputs[socket_name] = socket_inputs[0] - elif socket.is_greedy: - # We need to keep track of greedy inputs because we always remove them, even if they come from - # outside the pipeline. Otherwise, a greedy input from the user would trigger a pipeline to run - # indefinitely. - greedy_inputs_to_remove.add(socket_name) - consumed_inputs[socket_name] = [socket_inputs[0]] - elif is_socket_lazy_variadic(socket): - # We use all inputs provided to the socket on a lazy variadic socket. - consumed_inputs[socket_name] = socket_inputs - - # We prune all inputs except for those that were provided from outside the pipeline (e.g. user inputs). - pruned_inputs = { - socket_name: [ - sock for sock in socket if sock["sender"] is None and not socket_name in greedy_inputs_to_remove - ] - for socket_name, socket in component_inputs.items() - } - pruned_inputs = {socket_name: socket for socket_name, socket in pruned_inputs.items() if len(socket) > 0} - - inputs[component_name] = pruned_inputs - - return consumed_inputs, inputs - - @staticmethod - def _convert_from_legacy_format(pipeline_inputs: Dict[str, Any]) -> Dict[str, Dict[str, List]]: - """ - Converts the inputs to the pipeline to the format that is needed for the internal `Pipeline.run` logic. - - :param pipeline_inputs: Inputs to the pipeline. - :returns: Converted inputs that can be used by the internal `Pipeline.run` logic. - """ - inputs: Dict[str, Dict[str, List[Dict[str, Any]]]] = {} - for component_name, socket_dict in pipeline_inputs.items(): - inputs[component_name] = {} - for socket_name, value in socket_dict.items(): - inputs[component_name][socket_name] = [{"sender": None, "value": value}] - - return inputs - - def _fill_queue(self, component_names: List[str], inputs: Dict[str, Any]) -> FIFOPriorityQueue: - """ - Calculates the execution priority for each component and inserts it into the priority queue. - - :param component_names: Names of the components to put into the queue. - :param inputs: Inputs to the components. - :returns: A prioritized queue of component names. - """ - priority_queue = FIFOPriorityQueue() - for component_name in component_names: - component = self._get_component_with_graph_metadata(component_name) - priority = self._calculate_priority(component, inputs.get(component_name, {})) - priority_queue.push(component_name, priority) - - return priority_queue - - @staticmethod - def _calculate_priority(component: Dict, inputs: Dict) -> ComponentPriority: - """ - Calculates the execution priority for a component depending on the component's inputs. - - :param component: Component metadata and component instance. - :param inputs: Inputs to the component. - :returns: Priority value for the component. - """ - if not can_component_run(component, inputs): - return ComponentPriority.BLOCKED - elif is_any_greedy_socket_ready(component, inputs) and are_all_sockets_ready(component, inputs): - return ComponentPriority.HIGHEST - elif all_predecessors_executed(component, inputs): - return ComponentPriority.READY - elif are_all_lazy_variadic_sockets_resolved(component, inputs): - return ComponentPriority.DEFER - else: - return ComponentPriority.DEFER_LAST - - def _get_component_with_graph_metadata(self, component_name: str) -> Dict[str, Any]: - return self.graph.nodes[component_name] - - def _get_next_runnable_component( - self, priority_queue: FIFOPriorityQueue - ) -> Union[Tuple[ComponentPriority, str, Dict[str, Any]], None]: - """ - Returns the next runnable component alongside its metadata from the priority queue. - - :param priority_queue: Priority queue of component names. - :returns: The next runnable component, the component name, and its priority - or None if no component in the queue can run. - :raises: PipelineMaxComponentRuns if the next runnable component has exceeded the maximum number of runs. - """ - priority_and_component_name: Union[Tuple[ComponentPriority, str], None] = priority_queue.get() - - if priority_and_component_name is not None and priority_and_component_name[0] != ComponentPriority.BLOCKED: - priority, component_name = priority_and_component_name - component = self._get_component_with_graph_metadata(component_name) - if component["visits"] > self._max_runs_per_component: - msg = f"Maximum run count {self._max_runs_per_component} reached for component '{component_name}'" - raise PipelineMaxComponentRuns(msg) - - return priority, component_name, component - - return None - - @staticmethod - def _write_component_outputs( - component_name, component_outputs, inputs, receivers, include_outputs_from - ) -> Tuple[Dict, Dict]: - """ - Distributes the outputs of a component to the input sockets that it is connected to. - - :param component_name: The name of the component. - :param component_outputs: The outputs of the component. - :param inputs: The current global input state. - :param receivers: List of receiver_name, sender_socket, receiver_socket for connected components. - :param include_outputs_from: List of component names that should always return an output from the pipeline. - """ - for receiver_name, sender_socket, receiver_socket in receivers: - # We either get the value that was produced by the actor or we use the _NO_OUTPUT_PRODUCED class to indicate - # that the sender did not produce an output for this socket. - # This allows us to track if a pre-decessor already ran but did not produce an output. - value = component_outputs.get(sender_socket.name, _NO_OUTPUT_PRODUCED) - if receiver_name not in inputs: - inputs[receiver_name] = {} - - # If we have a non-variadic or a greedy variadic receiver socket, we can just overwrite any inputs - # that might already exist (to be reconsidered but mirrors current behavior). - if not is_socket_lazy_variadic(receiver_socket): - inputs[receiver_name][receiver_socket.name] = [{"sender": component_name, "value": value}] - - # If the receiver socket is lazy variadic, and it already has an input, we need to append the new input. - # Lazy variadic sockets can collect multiple inputs. - else: - if not inputs[receiver_name].get(receiver_socket.name): - inputs[receiver_name][receiver_socket.name] = [] - - inputs[receiver_name][receiver_socket.name].append({"sender": component_name, "value": value}) - - # If we want to include all outputs from this actor in the final outputs, we don't need to prune any consumed - # outputs - if component_name in include_outputs_from: - return component_outputs, inputs - - # We prune outputs that were consumed by any receiving sockets. - # All remaining outputs will be added to the final outputs of the pipeline. - consumed_outputs = {sender_socket.name for _, sender_socket, __ in receivers} - pruned_outputs = {key: value for key, value in component_outputs.items() if key not in consumed_outputs} - - return pruned_outputs, inputs + return cast(Dict[Any, Any], component_output), inputs @staticmethod def _merge_component_and_pipeline_outputs( @@ -378,15 +115,6 @@ def _merge_component_and_pipeline_outputs( return pipeline_outputs - @staticmethod - def _is_queue_stale(priority_queue: FIFOPriorityQueue) -> bool: - """ - Checks if the priority queue needs to be recomputed because the priorities might have changed. - - :param priority_queue: Priority queue of component names. - """ - return len(priority_queue) == 0 or priority_queue.peek()[0] > ComponentPriority.READY - def run( # noqa: PLR0915, PLR0912 self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None ) -> Dict[str, Any]: @@ -470,6 +198,8 @@ def run( # noqa: PLR0915, PLR0912 will only contain the outputs of leaf components, i.e., components without outgoing connections. + :raises ValueError: + If invalid inputs are provided to the pipeline. :raises PipelineRuntimeError: If the Pipeline contains cycles with unsupported connections that would cause it to get stuck and fail running. @@ -479,9 +209,6 @@ def run( # noqa: PLR0915, PLR0912 """ pipeline_running(self) - # Reset the visits count for each component - self._init_graph() - # TODO: Remove this warmup once we can check reliably whether a component has been warmed up or not # As of now it's here to make sure we don't have failing tests that assume warm_up() is called in run() self.warm_up() @@ -489,7 +216,7 @@ def run( # noqa: PLR0915, PLR0912 # normalize `data` data = self._prepare_component_input_data(data) - # Raise if input is malformed in some way + # Raise ValueError if input is malformed in some way self._validate_input(data) if include_outputs_from is None: @@ -499,6 +226,9 @@ def run( # noqa: PLR0915, PLR0912 # and independent of insertion order into the pipeline. ordered_component_names = sorted(self.graph.nodes.keys()) + # We track component visits to decide if a component can run. + component_visits = {component_name: 0 for component_name in ordered_component_names} + # We need to access a component's receivers multiple times during a pipeline run. # We store them here for easy access. cached_receivers = {name: self._find_receivers_from(name) for name in ordered_component_names} @@ -513,14 +243,14 @@ def run( # noqa: PLR0915, PLR0912 "haystack.pipeline.max_runs_per_component": self._max_runs_per_component, }, ) as span: - inputs = self._convert_from_legacy_format(pipeline_inputs=data) - self._warn_if_ambiguous_intent( - inputs=inputs, component_names=ordered_component_names, receivers=cached_receivers - ) - priority_queue = self._fill_queue(ordered_component_names, inputs) + inputs = self._convert_to_internal_format(pipeline_inputs=data) + priority_queue = self._fill_queue(ordered_component_names, inputs, component_visits) + + # check if pipeline is blocked before execution + self.validate_pipeline(priority_queue) while True: - candidate = self._get_next_runnable_component(priority_queue) + candidate = self._get_next_runnable_component(priority_queue, component_visits) if candidate is None: break @@ -528,15 +258,18 @@ def run( # noqa: PLR0915, PLR0912 if len(priority_queue) > 0: next_priority, next_name = priority_queue.peek() - # alternative to _warn_if_ambiguous_intent - if priority in [ComponentPriority.DEFER, ComponentPriority.DEFER_LAST] and next_priority == priority: - msg = ( - f"Ambiguous running order: Components '{component_name}' and '{next_name}' are waiting for " - f"optional inputs at the same time. Component '{component_name}' executes first." - ) - warnings.warn(msg) - - component_outputs, inputs = self._run_component(component, inputs, parent_span=span) + if ( + priority in [ComponentPriority.DEFER, ComponentPriority.DEFER_LAST] + and next_priority == priority + ): + msg = ( + f"Components '{component_name}' and '{next_name}' are waiting for " + f"optional inputs at the same time. The pipeline will execute '{component_name}' " + f"first based on lexicographical ordering." + ) + warnings.warn(msg) + + component_outputs, inputs = self._run_component(component, inputs, component_visits, parent_span=span) component_pipeline_outputs, inputs = self._write_component_outputs( component_name=component_name, component_outputs=component_outputs, @@ -549,6 +282,6 @@ def run( # noqa: PLR0915, PLR0912 if component_pipeline_outputs: pipeline_outputs[component_name] = component_pipeline_outputs if self._is_queue_stale(priority_queue): - priority_queue = self._fill_queue(ordered_component_names, inputs) + priority_queue = self._fill_queue(ordered_component_names, inputs, component_visits) return pipeline_outputs diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 1f4adb05ff..980b0e4b37 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -1,73 +1,16 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 -import logging -from typing import List, Optional -from unittest.mock import patch + +from typing import Optional import pytest -from haystack import Document -from haystack.components.builders import PromptBuilder, AnswerBuilder from haystack.components.joiners import BranchJoiner from haystack.core.component import component -from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic, _empty -from haystack.core.errors import ( - DeserializationError, - PipelineConnectError, - PipelineDrawingError, - PipelineError, - PipelineRuntimeError, - PipelineMaxComponentRuns, -) -from haystack.core.pipeline import Pipeline, PredefinedPipeline -from haystack.core.pipeline.pipeline import ComponentPriority, _NO_OUTPUT_PRODUCED -from haystack.core.pipeline.utils import FIFOPriorityQueue - -from haystack.core.serialization import DeserializationCallbacks -from haystack.testing.factory import component_class -from haystack.testing.sample_components import AddFixedValue, Double, Greet - -logging.basicConfig(level=logging.DEBUG) - - -@component -class FakeComponent: - def __init__(self, an_init_param: Optional[str] = None): - pass - - @component.output_types(value=str) - def run(self, input_: str): - return {"value": input_} - - -@component -class FakeComponentSquared: - def __init__(self, an_init_param: Optional[str] = None): - self.an_init_param = an_init_param - self.inner = FakeComponent() - - @component.output_types(value=str) - def run(self, input_: str): - return {"value": input_} - - -@pytest.fixture -def regular_output_socket(): - """Output socket for a regular (non-variadic) connection with receivers""" - return OutputSocket("output1", int, receivers=["receiver1", "receiver2"]) - - -@pytest.fixture -def regular_input_socket(): - """Regular (non-variadic) input socket with a single sender""" - return InputSocket("input1", int, senders=["sender1"]) - - -@pytest.fixture -def lazy_variadic_input_socket(): - """Lazy variadic input socket with multiple senders""" - return InputSocket("variadic_input", Variadic[int], senders=["sender1", "sender2"]) +from haystack.core.component.types import InputSocket, OutputSocket, Variadic +from haystack.core.errors import PipelineRuntimeError +from haystack.core.pipeline import Pipeline class TestPipeline: @@ -76,1001 +19,6 @@ class TestPipeline: It doesn't test Pipeline.run(), that is done separately in a different way. """ - def test_pipeline_dumps(self, test_files_path): - pipeline = Pipeline(max_runs_per_component=99) - pipeline.add_component("Comp1", FakeComponent("Foo")) - pipeline.add_component("Comp2", FakeComponent()) - pipeline.connect("Comp1.value", "Comp2.input_") - result = pipeline.dumps() - with open(f"{test_files_path}/yaml/test_pipeline.yaml", "r") as f: - assert f.read() == result - - def test_pipeline_loads_invalid_data(self): - invalid_yaml = """components: - Comp1: - init_parameters: - an_init_param: null - type: test.core.pipeline.test_pipeline.FakeComponent - Comp2* - init_parameters: - an_init_param: null - type: test.core.pipeline.test_pipeline.FakeComponent - connections: - * receiver: Comp2.input_ - sender: Comp1.value - metadata: - """ - - with pytest.raises(DeserializationError, match="unmarshalling serialized"): - pipeline = Pipeline.loads(invalid_yaml) - - invalid_init_parameter_yaml = """components: - Comp1: - init_parameters: - unknown: null - type: test.core.pipeline.test_pipeline.FakeComponent - Comp2: - init_parameters: - an_init_param: null - type: test.core.pipeline.test_pipeline.FakeComponent - connections: - - receiver: Comp2.input_ - sender: Comp1.value - metadata: {} - """ - - with pytest.raises(DeserializationError, match=".*Comp1.*unknown.*"): - pipeline = Pipeline.loads(invalid_init_parameter_yaml) - - def test_pipeline_dump(self, test_files_path, tmp_path): - pipeline = Pipeline(max_runs_per_component=99) - pipeline.add_component("Comp1", FakeComponent("Foo")) - pipeline.add_component("Comp2", FakeComponent()) - pipeline.connect("Comp1.value", "Comp2.input_") - with open(tmp_path / "out.yaml", "w") as f: - pipeline.dump(f) - # re-open and ensure it's the same data as the test file - with open(f"{test_files_path}/yaml/test_pipeline.yaml", "r") as test_f, open(tmp_path / "out.yaml", "r") as f: - assert f.read() == test_f.read() - - def test_pipeline_load(self, test_files_path): - with open(f"{test_files_path}/yaml/test_pipeline.yaml", "r") as f: - pipeline = Pipeline.load(f) - assert pipeline._max_runs_per_component == 99 - assert isinstance(pipeline.get_component("Comp1"), FakeComponent) - assert isinstance(pipeline.get_component("Comp2"), FakeComponent) - - @patch("haystack.core.pipeline.base._to_mermaid_image") - @patch("haystack.core.pipeline.base.is_in_jupyter") - @patch("IPython.display.Image") - @patch("IPython.display.display") - def test_show_in_notebook( - self, mock_ipython_display, mock_ipython_image, mock_is_in_jupyter, mock_to_mermaid_image - ): - pipe = Pipeline() - - mock_to_mermaid_image.return_value = b"some_image_data" - mock_is_in_jupyter.return_value = True - - pipe.show() - mock_ipython_image.assert_called_once_with(b"some_image_data") - mock_ipython_display.assert_called_once() - - @patch("haystack.core.pipeline.base.is_in_jupyter") - def test_show_not_in_notebook(self, mock_is_in_jupyter): - pipe = Pipeline() - - mock_is_in_jupyter.return_value = False - - with pytest.raises(PipelineDrawingError): - pipe.show() - - @patch("haystack.core.pipeline.base._to_mermaid_image") - def test_draw(self, mock_to_mermaid_image, tmp_path): - pipe = Pipeline() - mock_to_mermaid_image.return_value = b"some_image_data" - - image_path = tmp_path / "test.png" - pipe.draw(path=image_path) - assert image_path.read_bytes() == mock_to_mermaid_image.return_value - - # UNIT - def test_add_component_to_different_pipelines(self): - first_pipe = Pipeline() - second_pipe = Pipeline() - some_component = component_class("Some")() - - assert some_component.__haystack_added_to_pipeline__ is None - first_pipe.add_component("some", some_component) - assert some_component.__haystack_added_to_pipeline__ is first_pipe - - with pytest.raises(PipelineError): - second_pipe.add_component("some", some_component) - - def test_remove_component_raises_if_invalid_component_name(self): - pipe = Pipeline() - component = component_class("Some")() - - pipe.add_component("1", component) - - with pytest.raises(ValueError): - pipe.remove_component("2") - - def test_remove_component_removes_component_and_its_edges(self): - pipe = Pipeline() - component_1 = component_class("Type1")() - component_2 = component_class("Type2")() - component_3 = component_class("Type3")() - component_4 = component_class("Type4")() - - pipe.add_component("1", component_1) - pipe.add_component("2", component_2) - pipe.add_component("3", component_3) - pipe.add_component("4", component_4) - - pipe.connect("1", "2") - pipe.connect("2", "3") - pipe.connect("3", "4") - - pipe.remove_component("2") - - assert ["1", "3", "4"] == sorted(pipe.graph.nodes) - assert [("3", "4")] == sorted([(u, v) for (u, v) in pipe.graph.edges()]) - - def test_remove_component_allows_you_to_reuse_the_component(self): - pipe = Pipeline() - Some = component_class("Some", input_types={"in": int}, output_types={"out": int}) - - pipe.add_component("component_1", Some()) - pipe.add_component("component_2", Some()) - pipe.add_component("component_3", Some()) - pipe.connect("component_1", "component_2") - pipe.connect("component_2", "component_3") - component_2 = pipe.remove_component("component_2") - - assert component_2.__haystack_added_to_pipeline__ is None - assert component_2.__haystack_input__._sockets_dict == {"in": InputSocket(name="in", type=int, senders=[])} - assert component_2.__haystack_output__._sockets_dict == { - "out": OutputSocket(name="out", type=int, receivers=[]) - } - - pipe2 = Pipeline() - pipe2.add_component("component_4", Some()) - pipe2.add_component("component_2", component_2) - pipe2.add_component("component_5", Some()) - - pipe2.connect("component_4", "component_2") - pipe2.connect("component_2", "component_5") - assert component_2.__haystack_added_to_pipeline__ is pipe2 - assert component_2.__haystack_input__._sockets_dict == { - "in": InputSocket(name="in", type=int, senders=["component_4"]) - } - assert component_2.__haystack_output__._sockets_dict == { - "out": OutputSocket(name="out", type=int, receivers=["component_5"]) - } - - # instance = pipe2.get_component("some") - # assert instance == component - - # UNIT - def test_get_component_name(self): - pipe = Pipeline() - some_component = component_class("Some")() - pipe.add_component("some", some_component) - - assert pipe.get_component_name(some_component) == "some" - - # UNIT - def test_get_component_name_not_added_to_pipeline(self): - pipe = Pipeline() - some_component = component_class("Some")() - - assert pipe.get_component_name(some_component) == "" - - # UNIT - def test_repr(self): - pipe = Pipeline(metadata={"test": "test"}) - pipe.add_component("add_two", AddFixedValue(add=2)) - pipe.add_component("add_default", AddFixedValue()) - pipe.add_component("double", Double()) - pipe.connect("add_two", "double") - pipe.connect("double", "add_default") - - expected_repr = ( - f"{object.__repr__(pipe)}\n" - "🧱 Metadata\n" - " - test: test\n" - "🚅 Components\n" - " - add_two: AddFixedValue\n" - " - add_default: AddFixedValue\n" - " - double: Double\n" - "🛤️ Connections\n" - " - add_two.result -> double.value (int)\n" - " - double.value -> add_default.value (int)\n" - ) - - assert repr(pipe) == expected_repr - - # UNIT - def test_to_dict(self): - add_two = AddFixedValue(add=2) - add_default = AddFixedValue() - double = Double() - pipe = Pipeline(metadata={"test": "test"}, max_runs_per_component=42) - pipe.add_component("add_two", add_two) - pipe.add_component("add_default", add_default) - pipe.add_component("double", double) - pipe.connect("add_two", "double") - pipe.connect("double", "add_default") - - res = pipe.to_dict() - expected = { - "metadata": {"test": "test"}, - "max_runs_per_component": 42, - "components": { - "add_two": { - "type": "haystack.testing.sample_components.add_value.AddFixedValue", - "init_parameters": {"add": 2}, - }, - "add_default": { - "type": "haystack.testing.sample_components.add_value.AddFixedValue", - "init_parameters": {"add": 1}, - }, - "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, - }, - "connections": [ - {"sender": "add_two.result", "receiver": "double.value"}, - {"sender": "double.value", "receiver": "add_default.value"}, - ], - } - assert res == expected - - def test_from_dict(self): - data = { - "metadata": {"test": "test"}, - "max_runs_per_component": 101, - "components": { - "add_two": { - "type": "haystack.testing.sample_components.add_value.AddFixedValue", - "init_parameters": {"add": 2}, - }, - "add_default": { - "type": "haystack.testing.sample_components.add_value.AddFixedValue", - "init_parameters": {"add": 1}, - }, - "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, - }, - "connections": [ - {"sender": "add_two.result", "receiver": "double.value"}, - {"sender": "double.value", "receiver": "add_default.value"}, - ], - } - pipe = Pipeline.from_dict(data) - - assert pipe.metadata == {"test": "test"} - assert pipe._max_runs_per_component == 101 - - # Components - assert len(pipe.graph.nodes) == 3 - ## add_two - add_two = pipe.graph.nodes["add_two"] - assert add_two["instance"].add == 2 - assert add_two["input_sockets"] == { - "value": InputSocket(name="value", type=int), - "add": InputSocket(name="add", type=Optional[int], default_value=None), - } - assert add_two["output_sockets"] == {"result": OutputSocket(name="result", type=int, receivers=["double"])} - assert add_two["visits"] == 0 - - ## add_default - add_default = pipe.graph.nodes["add_default"] - assert add_default["instance"].add == 1 - assert add_default["input_sockets"] == { - "value": InputSocket(name="value", type=int, senders=["double"]), - "add": InputSocket(name="add", type=Optional[int], default_value=None), - } - assert add_default["output_sockets"] == {"result": OutputSocket(name="result", type=int)} - assert add_default["visits"] == 0 - - ## double - double = pipe.graph.nodes["double"] - assert double["instance"] - assert double["input_sockets"] == {"value": InputSocket(name="value", type=int, senders=["add_two"])} - assert double["output_sockets"] == {"value": OutputSocket(name="value", type=int, receivers=["add_default"])} - assert double["visits"] == 0 - - # Connections - connections = list(pipe.graph.edges(data=True)) - assert len(connections) == 2 - assert connections[0] == ( - "add_two", - "double", - { - "conn_type": "int", - "from_socket": OutputSocket(name="result", type=int, receivers=["double"]), - "to_socket": InputSocket(name="value", type=int, senders=["add_two"]), - "mandatory": True, - }, - ) - assert connections[1] == ( - "double", - "add_default", - { - "conn_type": "int", - "from_socket": OutputSocket(name="value", type=int, receivers=["add_default"]), - "to_socket": InputSocket(name="value", type=int, senders=["double"]), - "mandatory": True, - }, - ) - - # TODO: Remove this, this should be a component test. - # The pipeline can't handle this in any case nor way. - def test_from_dict_with_callbacks(self): - data = { - "metadata": {"test": "test"}, - "components": { - "add_two": { - "type": "haystack.testing.sample_components.add_value.AddFixedValue", - "init_parameters": {"add": 2}, - }, - "add_default": { - "type": "haystack.testing.sample_components.add_value.AddFixedValue", - "init_parameters": {"add": 1}, - }, - "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, - "greet": { - "type": "haystack.testing.sample_components.greet.Greet", - "init_parameters": {"message": "test"}, - }, - }, - "connections": [ - {"sender": "add_two.result", "receiver": "double.value"}, - {"sender": "double.value", "receiver": "add_default.value"}, - ], - } - - components_seen_in_callback = [] - - def component_pre_init_callback(name, component_cls, init_params): - assert name in ["add_two", "add_default", "double", "greet"] - assert component_cls in [AddFixedValue, Double, Greet] - - if name == "add_two": - assert init_params == {"add": 2} - elif name == "add_default": - assert init_params == {"add": 1} - elif name == "greet": - assert init_params == {"message": "test"} - - components_seen_in_callback.append(name) - - pipe = Pipeline.from_dict( - data, callbacks=DeserializationCallbacks(component_pre_init=component_pre_init_callback) - ) - assert components_seen_in_callback == ["add_two", "add_default", "double", "greet"] - add_two = pipe.graph.nodes["add_two"]["instance"] - assert add_two.add == 2 - add_default = pipe.graph.nodes["add_default"]["instance"] - assert add_default.add == 1 - greet = pipe.graph.nodes["greet"]["instance"] - assert greet.message == "test" - assert greet.log_level == "INFO" - - def component_pre_init_callback_modify(name, component_cls, init_params): - assert name in ["add_two", "add_default", "double", "greet"] - assert component_cls in [AddFixedValue, Double, Greet] - - if name == "add_two": - init_params["add"] = 3 - elif name == "add_default": - init_params["add"] = 0 - elif name == "greet": - init_params["message"] = "modified test" - init_params["log_level"] = "DEBUG" - - pipe = Pipeline.from_dict( - data, callbacks=DeserializationCallbacks(component_pre_init=component_pre_init_callback_modify) - ) - add_two = pipe.graph.nodes["add_two"]["instance"] - assert add_two.add == 3 - add_default = pipe.graph.nodes["add_default"]["instance"] - assert add_default.add == 0 - greet = pipe.graph.nodes["greet"]["instance"] - assert greet.message == "modified test" - assert greet.log_level == "DEBUG" - - # Test with a component that internally instantiates another component - def component_pre_init_callback_check_class(name, component_cls, init_params): - assert name == "fake_component_squared" - assert component_cls == FakeComponentSquared - - pipe = Pipeline() - pipe.add_component("fake_component_squared", FakeComponentSquared()) - pipe = Pipeline.from_dict( - pipe.to_dict(), - callbacks=DeserializationCallbacks(component_pre_init=component_pre_init_callback_check_class), - ) - assert type(pipe.graph.nodes["fake_component_squared"]["instance"].inner) == FakeComponent - - # UNIT - def test_from_dict_with_empty_dict(self): - assert Pipeline() == Pipeline.from_dict({}) - - # TODO: UNIT, consider deprecating this argument - def test_from_dict_with_components_instances(self): - add_two = AddFixedValue(add=2) - add_default = AddFixedValue() - components = {"add_two": add_two, "add_default": add_default} - data = { - "metadata": {"test": "test"}, - "components": { - "add_two": {}, - "add_default": {}, - "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, - }, - "connections": [ - {"sender": "add_two.result", "receiver": "double.value"}, - {"sender": "double.value", "receiver": "add_default.value"}, - ], - } - pipe = Pipeline.from_dict(data, components=components) - assert pipe.metadata == {"test": "test"} - - # Components - assert len(pipe.graph.nodes) == 3 - ## add_two - add_two_data = pipe.graph.nodes["add_two"] - assert add_two_data["instance"] is add_two - assert add_two_data["instance"].add == 2 - assert add_two_data["input_sockets"] == { - "value": InputSocket(name="value", type=int), - "add": InputSocket(name="add", type=Optional[int], default_value=None), - } - assert add_two_data["output_sockets"] == {"result": OutputSocket(name="result", type=int, receivers=["double"])} - assert add_two_data["visits"] == 0 - - ## add_default - add_default_data = pipe.graph.nodes["add_default"] - assert add_default_data["instance"] is add_default - assert add_default_data["instance"].add == 1 - assert add_default_data["input_sockets"] == { - "value": InputSocket(name="value", type=int, senders=["double"]), - "add": InputSocket(name="add", type=Optional[int], default_value=None), - } - assert add_default_data["output_sockets"] == {"result": OutputSocket(name="result", type=int, receivers=[])} - assert add_default_data["visits"] == 0 - - ## double - double = pipe.graph.nodes["double"] - assert double["instance"] - assert double["input_sockets"] == {"value": InputSocket(name="value", type=int, senders=["add_two"])} - assert double["output_sockets"] == {"value": OutputSocket(name="value", type=int, receivers=["add_default"])} - assert double["visits"] == 0 - - # Connections - connections = list(pipe.graph.edges(data=True)) - assert len(connections) == 2 - assert connections[0] == ( - "add_two", - "double", - { - "conn_type": "int", - "from_socket": OutputSocket(name="result", type=int, receivers=["double"]), - "to_socket": InputSocket(name="value", type=int, senders=["add_two"]), - "mandatory": True, - }, - ) - assert connections[1] == ( - "double", - "add_default", - { - "conn_type": "int", - "from_socket": OutputSocket(name="value", type=int, receivers=["add_default"]), - "to_socket": InputSocket(name="value", type=int, senders=["double"]), - "mandatory": True, - }, - ) - - # UNIT - def test_from_dict_without_component_type(self): - data = { - "metadata": {"test": "test"}, - "components": {"add_two": {"init_parameters": {"add": 2}}}, - "connections": [], - } - with pytest.raises(PipelineError) as err: - Pipeline.from_dict(data) - - err.match("Missing 'type' in component 'add_two'") - - # UNIT - def test_from_dict_without_registered_component_type(self): - data = { - "metadata": {"test": "test"}, - "components": {"add_two": {"type": "foo.bar.baz", "init_parameters": {"add": 2}}}, - "connections": [], - } - with pytest.raises(PipelineError) as err: - Pipeline.from_dict(data) - - err.match(r"Component .+ not imported.") - - def test_from_dict_with_invalid_type(self): - data = { - "metadata": {"test": "test"}, - "components": {"add_two": {"type": "", "init_parameters": {"add": 2}}}, - "connections": [], - } - with pytest.raises(PipelineError) as err: - Pipeline.from_dict(data) - - err.match(r"Component '' \(name: 'add_two'\) not imported.") - - # UNIT - def test_from_dict_without_connection_sender(self): - data = {"metadata": {"test": "test"}, "components": {}, "connections": [{"receiver": "some.receiver"}]} - with pytest.raises(PipelineError) as err: - Pipeline.from_dict(data) - - err.match("Missing sender in connection: {'receiver': 'some.receiver'}") - - # UNIT - def test_from_dict_without_connection_receiver(self): - data = {"metadata": {"test": "test"}, "components": {}, "connections": [{"sender": "some.sender"}]} - with pytest.raises(PipelineError) as err: - Pipeline.from_dict(data) - - err.match("Missing receiver in connection: {'sender': 'some.sender'}") - - def test_describe_input_only_no_inputs_components(self): - A = component_class("A", input_types={}, output={"x": 0}) - B = component_class("B", input_types={}, output={"y": 0}) - C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) - p = Pipeline() - p.add_component("a", A()) - p.add_component("b", B()) - p.add_component("c", C()) - p.connect("a.x", "c.x") - p.connect("b.y", "c.y") - assert p.inputs() == {} - assert p.inputs(include_components_with_connected_inputs=True) == { - "c": {"x": {"type": int, "is_mandatory": True}, "y": {"type": int, "is_mandatory": True}} - } - - def test_describe_input_some_components_with_no_inputs(self): - A = component_class("A", input_types={}, output={"x": 0}) - B = component_class("B", input_types={"y": int}, output={"y": 0}) - C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) - p = Pipeline() - p.add_component("a", A()) - p.add_component("b", B()) - p.add_component("c", C()) - p.connect("a.x", "c.x") - p.connect("b.y", "c.y") - assert p.inputs() == {"b": {"y": {"type": int, "is_mandatory": True}}} - assert p.inputs(include_components_with_connected_inputs=True) == { - "b": {"y": {"type": int, "is_mandatory": True}}, - "c": {"x": {"type": int, "is_mandatory": True}, "y": {"type": int, "is_mandatory": True}}, - } - - def test_describe_input_all_components_have_inputs(self): - A = component_class("A", input_types={"x": Optional[int]}, output={"x": 0}) - B = component_class("B", input_types={"y": int}, output={"y": 0}) - C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) - p = Pipeline() - p.add_component("a", A()) - p.add_component("b", B()) - p.add_component("c", C()) - p.connect("a.x", "c.x") - p.connect("b.y", "c.y") - assert p.inputs() == { - "a": {"x": {"type": Optional[int], "is_mandatory": True}}, - "b": {"y": {"type": int, "is_mandatory": True}}, - } - assert p.inputs(include_components_with_connected_inputs=True) == { - "a": {"x": {"type": Optional[int], "is_mandatory": True}}, - "b": {"y": {"type": int, "is_mandatory": True}}, - "c": {"x": {"type": int, "is_mandatory": True}, "y": {"type": int, "is_mandatory": True}}, - } - - def test_describe_output_multiple_possible(self): - """ - This pipeline has two outputs: - {"b": {"output_b": {"type": str}}, "a": {"output_a": {"type": str}}} - """ - A = component_class("A", input_types={"input_a": str}, output={"output_a": "str", "output_b": "str"}) - B = component_class("B", input_types={"input_b": str}, output={"output_b": "str"}) - - pipe = Pipeline() - pipe.add_component("a", A()) - pipe.add_component("b", B()) - pipe.connect("a.output_b", "b.input_b") - - assert pipe.outputs() == {"b": {"output_b": {"type": str}}, "a": {"output_a": {"type": str}}} - assert pipe.outputs(include_components_with_connected_outputs=True) == { - "a": {"output_a": {"type": str}, "output_b": {"type": str}}, - "b": {"output_b": {"type": str}}, - } - - def test_describe_output_single(self): - """ - This pipeline has one output: - {"c": {"z": {"type": int}}} - """ - A = component_class("A", input_types={"x": Optional[int]}, output={"x": 0}) - B = component_class("B", input_types={"y": int}, output={"y": 0}) - C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) - p = Pipeline() - p.add_component("a", A()) - p.add_component("b", B()) - p.add_component("c", C()) - p.connect("a.x", "c.x") - p.connect("b.y", "c.y") - - assert p.outputs() == {"c": {"z": {"type": int}}} - assert p.outputs(include_components_with_connected_outputs=True) == { - "a": {"x": {"type": int}}, - "b": {"y": {"type": int}}, - "c": {"z": {"type": int}}, - } - - def test_describe_no_outputs(self): - """ - This pipeline sets up elaborate connections between three components but in fact it has no outputs: - Check that p.outputs() == {} - """ - A = component_class("A", input_types={"x": Optional[int]}, output={"x": 0}) - B = component_class("B", input_types={"y": int}, output={"y": 0}) - C = component_class("C", input_types={"x": int, "y": int}, output={}) - p = Pipeline() - p.add_component("a", A()) - p.add_component("b", B()) - p.add_component("c", C()) - p.connect("a.x", "c.x") - p.connect("b.y", "c.y") - assert p.outputs() == {} - assert p.outputs(include_components_with_connected_outputs=True) == { - "a": {"x": {"type": int}}, - "b": {"y": {"type": int}}, - } - - def test_from_template(self, monkeypatch): - monkeypatch.setenv("OPENAI_API_KEY", "fake_key") - pipe = Pipeline.from_template(PredefinedPipeline.INDEXING) - assert pipe.get_component("cleaner") - - def test_walk_pipeline_with_no_cycles(self): - """ - This pipeline has two source nodes, source1 and source2, one hello3 node in between, and one sink node, joiner. - pipeline.walk() should return each component exactly once. The order is not guaranteed. - """ - - @component - class Hello: - @component.output_types(output=str) - def run(self, word: str): - """ - Takes a string in input and returns "Hello, !" in output. - """ - return {"output": f"Hello, {word}!"} - - @component - class Joiner: - @component.output_types(output=str) - def run(self, word1: str, word2: str): - """ - Takes two strings in input and returns "Hello, and !" in output. - """ - return {"output": f"Hello, {word1} and {word2}!"} - - pipeline = Pipeline() - source1 = Hello() - source2 = Hello() - hello3 = Hello() - joiner = Joiner() - pipeline.add_component("source1", source1) - pipeline.add_component("source2", source2) - pipeline.add_component("hello3", hello3) - pipeline.add_component("joiner", joiner) - - pipeline.connect("source1", "joiner.word1") - pipeline.connect("source2", "hello3") - pipeline.connect("hello3", "joiner.word2") - - expected_components = [("source1", source1), ("source2", source2), ("joiner", joiner), ("hello3", hello3)] - assert sorted(expected_components) == sorted(pipeline.walk()) - - def test_walk_pipeline_with_cycles(self): - """ - This pipeline consists of two components, which would run three times in a loop. - pipeline.walk() should return these components exactly once. The order is not guaranteed. - """ - - @component - class Hello: - def __init__(self): - self.iteration_counter = 0 - - @component.output_types(intermediate=str, final=str) - def run(self, word: str, intermediate: Optional[str] = None): - """ - Takes a string in input and returns "Hello, !" in output. - """ - if self.iteration_counter < 3: - self.iteration_counter += 1 - return {"intermediate": f"Hello, {intermediate or word}!"} - return {"final": f"Hello, {intermediate or word}!"} - - pipeline = Pipeline() - hello = Hello() - hello_again = Hello() - pipeline.add_component("hello", hello) - pipeline.add_component("hello_again", hello_again) - pipeline.connect("hello.intermediate", "hello_again.intermediate") - pipeline.connect("hello_again.intermediate", "hello.intermediate") - assert {("hello", hello), ("hello_again", hello_again)} == set(pipeline.walk()) - - def test__init_graph(self): - pipe = Pipeline() - pipe.add_component("greet", Greet()) - pipe.add_component("adder", AddFixedValue()) - pipe.connect("greet", "adder") - pipe._init_graph() - for node in pipe.graph.nodes: - assert pipe.graph.nodes[node]["visits"] == 0 - - def test__prepare_component_input_data(self): - MockComponent = component_class("MockComponent", input_types={"x": List[str], "y": str}) - pipe = Pipeline() - pipe.add_component("first_mock", MockComponent()) - pipe.add_component("second_mock", MockComponent()) - - res = pipe._prepare_component_input_data({"x": ["some data"], "y": "some other data"}) - assert res == { - "first_mock": {"x": ["some data"], "y": "some other data"}, - "second_mock": {"x": ["some data"], "y": "some other data"}, - } - assert id(res["first_mock"]["x"]) != id(res["second_mock"]["x"]) - - def test__prepare_component_input_data_with_connected_inputs(self): - MockComponent = component_class( - "MockComponent", input_types={"x": List[str], "y": str}, output_types={"z": str} - ) - pipe = Pipeline() - pipe.add_component("first_mock", MockComponent()) - pipe.add_component("second_mock", MockComponent()) - pipe.connect("first_mock.z", "second_mock.y") - - res = pipe._prepare_component_input_data({"x": ["some data"], "y": "some other data"}) - assert res == {"first_mock": {"x": ["some data"], "y": "some other data"}, "second_mock": {"x": ["some data"]}} - assert id(res["first_mock"]["x"]) != id(res["second_mock"]["x"]) - - def test__prepare_component_input_data_with_non_existing_input(self, caplog): - pipe = Pipeline() - res = pipe._prepare_component_input_data({"input_name": 1}) - assert res == {} - assert ( - "Inputs ['input_name'] were not matched to any component inputs, " - "please check your run parameters." in caplog.text - ) - - def test_connect(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - assert pipe.connect("comp1.value", "comp2.value") is pipe - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_already_connected(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.connect("comp1.value", "comp2.value") - pipe.connect("comp1.value", "comp2.value") - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_with_sender_component_name(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.connect("comp1", "comp2.value") - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_with_receiver_component_name(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.connect("comp1.value", "comp2") - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_with_sender_and_receiver_component_name(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.connect("comp1", "comp2") - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_with_sender_not_in_pipeline(self): - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp2", comp2) - with pytest.raises(ValueError): - pipe.connect("comp1.value", "comp2.value") - - def test_connect_with_receiver_not_in_pipeline(self): - comp1 = component_class("Comp1", output_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - with pytest.raises(ValueError): - pipe.connect("comp1.value", "comp2.value") - - def test_connect_with_sender_socket_name_not_in_pipeline(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - with pytest.raises(PipelineConnectError): - pipe.connect("comp1.non_existing", "comp2.value") - - def test_connect_with_receiver_socket_name_not_in_pipeline(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - with pytest.raises(PipelineConnectError): - pipe.connect("comp1.value", "comp2.non_existing") - - def test_connect_with_no_matching_types_and_same_names(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": str})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - with pytest.raises(PipelineConnectError): - pipe.connect("comp1", "comp2") - - def test_connect_with_multiple_sender_connections_with_same_type_and_differing_name(self): - comp1 = component_class("Comp1", output_types={"val1": int, "val2": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - with pytest.raises(PipelineConnectError): - pipe.connect("comp1", "comp2") - - def test_connect_with_multiple_receiver_connections_with_same_type_and_differing_name(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"val1": int, "val2": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - with pytest.raises(PipelineConnectError): - pipe.connect("comp1", "comp2") - - def test_connect_with_multiple_sender_connections_with_same_type_and_same_name(self): - comp1 = component_class("Comp1", output_types={"value": int, "other": int})() - comp2 = component_class("Comp2", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.connect("comp1", "comp2") - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_with_multiple_receiver_connections_with_same_type_and_same_name(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", input_types={"value": int, "other": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.connect("comp1", "comp2") - - assert comp1.__haystack_output__.value.receivers == ["comp2"] - assert comp2.__haystack_input__.value.senders == ["comp1"] - assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] - - def test_connect_multiple_outputs_to_non_variadic_input(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", output_types={"value": int})() - comp3 = component_class("Comp3", input_types={"value": int})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.add_component("comp3", comp3) - pipe.connect("comp1.value", "comp3.value") - with pytest.raises(PipelineConnectError): - pipe.connect("comp2.value", "comp3.value") - - def test_connect_multiple_outputs_to_variadic_input(self): - comp1 = component_class("Comp1", output_types={"value": int})() - comp2 = component_class("Comp2", output_types={"value": int})() - comp3 = component_class("Comp3", input_types={"value": Variadic[int]})() - pipe = Pipeline() - pipe.add_component("comp1", comp1) - pipe.add_component("comp2", comp2) - pipe.add_component("comp3", comp3) - pipe.connect("comp1.value", "comp3.value") - pipe.connect("comp2.value", "comp3.value") - - assert comp1.__haystack_output__.value.receivers == ["comp3"] - assert comp2.__haystack_output__.value.receivers == ["comp3"] - assert comp3.__haystack_input__.value.senders == ["comp1", "comp2"] - assert list(pipe.graph.edges) == [("comp1", "comp3", "value/value"), ("comp2", "comp3", "value/value")] - - def test_connect_same_component_as_sender_and_receiver(self): - """ - This pipeline consists of one component, which would be connected to itself. - Connecting a component to itself is raises PipelineConnectError. - """ - pipe = Pipeline() - single_component = FakeComponent() - pipe.add_component("single_component", single_component) - with pytest.raises(PipelineConnectError): - pipe.connect("single_component.out", "single_component.in") - - @pytest.mark.parametrize( - "component_inputs,sockets,expected_inputs", - [ - ({"mandatory": 1}, {"mandatory": InputSocket("mandatory", int)}, {"mandatory": 1}), - ({}, {"optional": InputSocket("optional", str, default_value="test")}, {"optional": "test"}), - ( - {"mandatory": 1}, - { - "mandatory": InputSocket("mandatory", int), - "optional": InputSocket("optional", str, default_value="test"), - }, - {"mandatory": 1, "optional": "test"}, - ), - ( - {}, - {"optional_variadic": InputSocket("optional_variadic", Variadic[str], default_value="test")}, - {"optional_variadic": ["test"]}, - ), - ( - {}, - { - "optional_1": InputSocket("optional_1", int, default_value=1), - "optional_2": InputSocket("optional_2", int, default_value=2), - }, - {"optional_1": 1, "optional_2": 2}, - ), - ], - ids=["no-defaults", "only-default", "mixed-default", "variadic-default", "multiple_defaults"], - ) - def test__add_missing_defaults(self, component_inputs, sockets, expected_inputs): - filled_inputs = Pipeline._add_missing_input_defaults(component_inputs, sockets) - - assert filled_inputs == expected_inputs - def test__run_component_success(self): """Test successful component execution""" joiner_1 = BranchJoiner(type_=str) @@ -1082,7 +30,9 @@ def test__run_component_success(self): inputs = {"joiner_1": {"value": [{"sender": None, "value": "test_value"}]}} outputs, updated_inputs = pp._run_component( - component=pp._get_component_with_graph_metadata("joiner_1"), inputs=inputs + component=pp._get_component_with_graph_metadata_and_visits("joiner_1", 0), + inputs=inputs, + component_visits={"joiner_1": 0, "joiner_2": 0}, ) assert outputs == {"value": "test_value"} @@ -1105,568 +55,10 @@ def run(self, value: str): inputs = {"wrong": {"value": [{"sender": None, "value": "test_value"}]}} with pytest.raises(PipelineRuntimeError) as exc_info: - pp._run_component(component=pp._get_component_with_graph_metadata("wrong"), inputs=inputs) - - assert "didn't return a dictionary" in str(exc_info.value) - - def test__find_receivers_from(self): - sentence_builder = component_class( - "SentenceBuilder", input_types={"words": List[str]}, output_types={"text": str} - )() - document_builder = component_class( - "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} - )() - conditional_document_builder = component_class( - "ConditionalDocumentBuilder", output_types={"doc": Document, "noop": None} - )() - - document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() - - pipe = Pipeline() - pipe.add_component("sentence_builder", sentence_builder) - pipe.add_component("document_builder", document_builder) - pipe.add_component("document_joiner", document_joiner) - pipe.add_component("conditional_document_builder", conditional_document_builder) - pipe.connect("sentence_builder.text", "document_builder.text") - pipe.connect("document_builder.doc", "document_joiner.docs") - pipe.connect("conditional_document_builder.doc", "document_joiner.docs") - - res = pipe._find_receivers_from("sentence_builder") - assert res == [ - ( - "document_builder", - OutputSocket(name="text", type=str, receivers=["document_builder"]), - InputSocket(name="text", type=str, default_value=_empty, senders=["sentence_builder"]), - ) - ] - - res = pipe._find_receivers_from("document_builder") - assert res == [ - ( - "document_joiner", - OutputSocket(name="doc", type=Document, receivers=["document_joiner"]), - InputSocket( - name="docs", - type=Variadic[Document], - default_value=_empty, - senders=["document_builder", "conditional_document_builder"], - ), + pp._run_component( + component=pp._get_component_with_graph_metadata_and_visits("wrong", 0), + inputs=inputs, + component_visits={"wrong": 0}, ) - ] - - res = pipe._find_receivers_from("document_joiner") - assert res == [] - - res = pipe._find_receivers_from("conditional_document_builder") - assert res == [ - ( - "document_joiner", - OutputSocket(name="doc", type=Document, receivers=["document_joiner"]), - InputSocket( - name="docs", - type=Variadic[Document], - default_value=_empty, - senders=["document_builder", "conditional_document_builder"], - ), - ) - ] - - @pytest.mark.parametrize( - "component, inputs, expected_priority, test_description", - [ - # Test case 1: BLOCKED - Missing mandatory input - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": { - "mandatory_input": InputSocket("mandatory_input", int), - "optional_input": InputSocket( - "optional_input", str, default_value="default", senders=["previous_component"] - ), - }, - }, - {"optional_input": [{"sender": "previous_component", "value": "test"}]}, - ComponentPriority.BLOCKED, - "Component should be BLOCKED when mandatory input is missing", - ), - # Test case 2: BLOCKED - No trigger after first visit - ( - { - "instance": "mock_instance", - "visits": 1, # Already visited - "input_sockets": { - "mandatory_input": InputSocket("mandatory_input", int), - "optional_input": InputSocket("optional_input", str, default_value="default"), - }, - }, - {"mandatory_input": [{"sender": None, "value": 42}]}, - ComponentPriority.BLOCKED, - "Component should be BLOCKED when there's no new trigger after first visit", - ), - # Test case 3: HIGHEST - Greedy socket ready - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": { - "greedy_input": InputSocket("greedy_input", GreedyVariadic[int], senders=["component1"]), - "normal_input": InputSocket("normal_input", str, senders=["component2"]), - }, - }, - { - "greedy_input": [{"sender": "component1", "value": 42}], - "normal_input": [{"sender": "component2", "value": "test"}], - }, - ComponentPriority.HIGHEST, - "Component should have HIGHEST priority when greedy socket has valid input", - ), - # Test case 4: DEFER - Greedy socket ready but optional missing - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": { - "greedy_input": InputSocket("greedy_input", GreedyVariadic[int], senders=["component1"]), - "optional_input": InputSocket( - "optional_input", str, senders=["component2"], default_value="test" - ), - }, - }, - {"greedy_input": [{"sender": "component1", "value": 42}]}, - ComponentPriority.DEFER, - "Component should DEFER when greedy socket has valid input but expected optional input is missing", - ), - # Test case 4: READY - All predecessors executed - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": { - "mandatory_input": InputSocket("mandatory_input", int, senders=["previous_component"]), - "optional_input": InputSocket( - "optional_input", str, senders=["another_component"], default_value="default" - ), - }, - }, - { - "mandatory_input": [{"sender": "previous_component", "value": 42}], - "optional_input": [{"sender": "another_component", "value": "test"}], - }, - ComponentPriority.READY, - "Component should be READY when all predecessors have executed", - ), - # Test case 5: DEFER - Lazy variadic sockets resolved and optional missing. - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": { - "variadic_input": InputSocket( - "variadic_input", Variadic[int], senders=["component1", "component2"] - ), - "normal_input": InputSocket("normal_input", str, senders=["component3"]), - "optional_input": InputSocket( - "optional_input", str, default_value="default", senders=["component4"] - ), - }, - }, - { - "variadic_input": [ - {"sender": "component1", "value": "test"}, - {"sender": "component2", "value": _NO_OUTPUT_PRODUCED}, - ], - "normal_input": [{"sender": "component3", "value": "test"}], - }, - ComponentPriority.DEFER, - "Component should DEFER when all lazy variadic sockets are resolved", - ), - # Test case 6: DEFER_LAST - Incomplete variadic inputs - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": { - "variadic_input": InputSocket( - "variadic_input", Variadic[int], senders=["component1", "component2"] - ), - "normal_input": InputSocket("normal_input", str), - }, - }, - { - "variadic_input": [{"sender": "component1", "value": 42}], # Missing component2 - "normal_input": [{"sender": "component3", "value": "test"}], - }, - ComponentPriority.DEFER_LAST, - "Component should be DEFER_LAST when not all variadic senders have produced output", - ), - # Test case 7: READY - No input sockets, first visit - ( - { - "instance": "mock_instance", - "visits": 0, - "input_sockets": {"optional_input": InputSocket("optional_input", str, default_value="default")}, - }, - {}, # no inputs - ComponentPriority.READY, - "Component should be READY on first visit when it has no input sockets", - ), - # Test case 8: BLOCKED - No connected input sockets, subsequent visit - ( - { - "instance": "mock_instance", - "visits": 1, - "input_sockets": {"optional_input": InputSocket("optional_input", str, default_value="default")}, - }, - {}, # no inputs - ComponentPriority.BLOCKED, - "Component should be BLOCKED on subsequent visits when it has no input sockets", - ), - ], - ids=lambda p: p.name if isinstance(p, ComponentPriority) else str(p), - ) - def test__calculate_priority(self, component, inputs, expected_priority, test_description): - """Test priority calculation for various component and input combinations.""" - # For variadic inputs, set up senders if needed - for socket in component["input_sockets"].values(): - if socket.is_variadic and not hasattr(socket, "senders"): - socket.senders = ["component1", "component2"] - - assert Pipeline._calculate_priority(component, inputs) == expected_priority - - @pytest.mark.parametrize( - "pipeline_inputs,expected_output", - [ - # Test case 1: Empty input - ({}, {}), - # Test case 2: Single component, multiple inputs - ( - {"component1": {"input1": 42, "input2": "test", "input3": True}}, - { - "component1": { - "input1": [{"sender": None, "value": 42}], - "input2": [{"sender": None, "value": "test"}], - "input3": [{"sender": None, "value": True}], - } - }, - ), - # Test case 3: Multiple components - ( - { - "component1": {"input1": 42, "input2": "test"}, - "component2": {"input3": [1, 2, 3], "input4": {"key": "value"}}, - }, - { - "component1": { - "input1": [{"sender": None, "value": 42}], - "input2": [{"sender": None, "value": "test"}], - }, - "component2": { - "input3": [{"sender": None, "value": [1, 2, 3]}], - "input4": [{"sender": None, "value": {"key": "value"}}], - }, - }, - ), - ], - ids=["empty_input", "single_component_multiple_inputs", "multiple_components"], - ) - def test__convert_from_legacy_format(self, pipeline_inputs, expected_output): - """Test conversion of legacy pipeline inputs to internal format.""" - result = Pipeline._convert_from_legacy_format(pipeline_inputs) - assert result == expected_output - - @pytest.mark.parametrize( - "socket_type,existing_inputs,expected_count", - [ - ("regular", None, 1), # Regular socket should overwrite - ("regular", [{"sender": "other", "value": 24}], 1), # Should still overwrite - ("lazy_variadic", None, 1), # First input to lazy variadic - ("lazy_variadic", [{"sender": "other", "value": 24}], 2), # Should append - ], - ids=["regular-new", "regular-existing", "variadic-new", "variadic-existing"], - ) - def test__write_component_outputs_different_sockets( - self, - socket_type, - existing_inputs, - expected_count, - regular_output_socket, - regular_input_socket, - lazy_variadic_input_socket, - ): - """Test writing to different socket types with various existing input states""" - receiver_socket = lazy_variadic_input_socket if socket_type == "lazy_variadic" else regular_input_socket - socket_name = receiver_socket.name - receivers = [("receiver1", regular_output_socket, receiver_socket)] - - inputs = {} - if existing_inputs: - inputs = {"receiver1": {socket_name: existing_inputs}} - - component_outputs = {"output1": 42} - - pruned_outputs, updated_inputs = Pipeline._write_component_outputs( - component_name="sender1", - component_outputs=component_outputs, - inputs=inputs, - receivers=receivers, - include_outputs_from=[], - ) - - assert len(updated_inputs["receiver1"][socket_name]) == expected_count - assert {"sender": "sender1", "value": 42} in updated_inputs["receiver1"][socket_name] - - @pytest.mark.parametrize( - "component_outputs,include_outputs,expected_pruned", - [ - ({"output1": 42, "output2": 24}, [], {"output2": 24}), # Prune consumed outputs only - ({"output1": 42, "output2": 24}, ["sender1"], {"output1": 42, "output2": 24}), # Keep all outputs - ({}, [], {}), # No outputs case - ], - ids=["prune-consumed", "keep-all", "no-outputs"], - ) - def test__write_component_outputs_output_pruning( - self, component_outputs, include_outputs, expected_pruned, regular_output_socket, regular_input_socket - ): - """Test output pruning behavior under different scenarios""" - receivers = [("receiver1", regular_output_socket, regular_input_socket)] - - pruned_outputs, _ = Pipeline._write_component_outputs( - component_name="sender1", - component_outputs=component_outputs, - inputs={}, - receivers=receivers, - include_outputs_from=include_outputs, - ) - - assert pruned_outputs == expected_pruned - - @pytest.mark.parametrize( - "output_value", - [42, None, _NO_OUTPUT_PRODUCED, "string_value", 3.14], - ids=["int", "none", "no-output", "string", "float"], - ) - def test__write_component_outputs_different_output_values( - self, output_value, regular_output_socket, regular_input_socket - ): - """Test handling of different output values""" - receivers = [("receiver1", regular_output_socket, regular_input_socket)] - component_outputs = {"output1": output_value} - - _, updated_inputs = Pipeline._write_component_outputs( - component_name="sender1", - component_outputs=component_outputs, - inputs={}, - receivers=receivers, - include_outputs_from=[], - ) - - assert updated_inputs["receiver1"]["input1"] == [{"sender": "sender1", "value": output_value}] - - @pytest.mark.parametrize("receivers_count", [1, 2, 3], ids=["single-receiver", "two-receivers", "three-receivers"]) - def test__write_component_outputs_multiple_receivers( - self, receivers_count, regular_output_socket, regular_input_socket - ): - """Test writing to multiple receivers""" - receivers = [(f"receiver{i}", regular_output_socket, regular_input_socket) for i in range(receivers_count)] - component_outputs = {"output1": 42} - - _, updated_inputs = Pipeline._write_component_outputs( - component_name="sender1", - component_outputs=component_outputs, - inputs={}, - receivers=receivers, - include_outputs_from=[], - ) - - for i in range(receivers_count): - receiver_name = f"receiver{i}" - assert receiver_name in updated_inputs - assert updated_inputs[receiver_name]["input1"] == [{"sender": "sender1", "value": 42}] - - def test__get_next_runnable_component_empty(self): - """Test with empty queue returns None""" - queue = FIFOPriorityQueue() - pipeline = Pipeline() - result = pipeline._get_next_runnable_component(queue) - assert result is None - - def test__get_next_runnable_component_blocked(self): - """Test component with BLOCKED priority returns None""" - pipeline = Pipeline() - queue = FIFOPriorityQueue() - queue.push("blocked_component", ComponentPriority.BLOCKED) - result = pipeline._get_next_runnable_component(queue) - assert result is None - - @patch("haystack.core.pipeline.Pipeline._get_component_with_graph_metadata") - def test__get_next_runnable_component_max_visits(self, mock_get_component_with_graph_metadata): - """Test component exceeding max visits raises exception""" - pipeline = Pipeline(max_runs_per_component=2) - queue = FIFOPriorityQueue() - queue.push("ready_component", ComponentPriority.READY) - mock_get_component_with_graph_metadata.return_value = {"instance": "test", "visits": 3} - - with pytest.raises(PipelineMaxComponentRuns) as exc_info: - pipeline._get_next_runnable_component(queue) - - assert "Maximum run count 2 reached for component 'ready_component'" in str(exc_info.value) - - @patch("haystack.core.pipeline.Pipeline._get_component_with_graph_metadata") - def test__get_next_runnable_component_ready(self, mock_get_component_with_graph_metadata): - """Test component that is READY""" - pipeline = Pipeline() - queue = FIFOPriorityQueue() - queue.push("ready_component", ComponentPriority.READY) - mock_get_component_with_graph_metadata.return_value = {"instance": "test", "visits": 1} - - priority, component_name, component = pipeline._get_next_runnable_component(queue) - assert priority == ComponentPriority.READY - assert component_name == "ready_component" - assert component == {"instance": "test", "visits": 1} - - @pytest.mark.parametrize( - "queue_setup,expected_stale", - [ - # Empty queue case - (None, True), - # READY priority case - ((ComponentPriority.READY, "component1"), False), - # DEFER priority case - ((ComponentPriority.DEFER, "component1"), True), - ], - ids=["empty-queue", "ready-component", "deferred-component"], - ) - def test__is_queue_stale(self, queue_setup, expected_stale): - queue = FIFOPriorityQueue() - if queue_setup: - priority, component_name = queue_setup - queue.push(component_name, priority) - - result = Pipeline._is_queue_stale(queue) - assert result == expected_stale - - @patch("haystack.core.pipeline.Pipeline._calculate_priority") - @patch("haystack.core.pipeline.Pipeline._get_component_with_graph_metadata") - def test_fill_queue(self, mock_get_metadata, mock_calc_priority): - pipeline = Pipeline() - component_names = ["comp1", "comp2"] - inputs = {"comp1": {"input1": "value1"}, "comp2": {"input2": "value2"}} - - mock_get_metadata.side_effect = lambda name: {"component": f"mock_{name}"} - mock_calc_priority.side_effect = [1, 2] # Different priorities for testing - - queue = pipeline._fill_queue(component_names, inputs) - - assert mock_get_metadata.call_count == 2 - assert mock_calc_priority.call_count == 2 - - # Verify correct calls for first component - mock_get_metadata.assert_any_call("comp1") - mock_calc_priority.assert_any_call({"component": "mock_comp1"}, {"input1": "value1"}) - - # Verify correct calls for second component - mock_get_metadata.assert_any_call("comp2") - mock_calc_priority.assert_any_call({"component": "mock_comp2"}, {"input2": "value2"}) - - assert queue.pop() == (1, "comp1") - assert queue.pop() == (2, "comp2") - - @pytest.mark.parametrize( - "input_sockets,component_inputs,expected_consumed,expected_remaining", - [ - # Regular socket test - ( - {"input1": InputSocket("input1", int)}, - {"input1": [{"sender": "comp1", "value": 42}, {"sender": "comp2", "value": 24}]}, - {"input1": 42}, # Should take first valid input - {}, # All pipeline inputs should be removed - ), - # Regular socket with user input - ( - {"input1": InputSocket("input1", int)}, - { - "input1": [ - {"sender": "comp1", "value": 42}, - {"sender": None, "value": 24}, # User input - ] - }, - {"input1": 42}, - {"input1": [{"sender": None, "value": 24}]}, # User input should remain - ), - # Greedy variadic socket - ( - {"greedy": InputSocket("greedy", GreedyVariadic[int])}, - { - "greedy": [ - {"sender": "comp1", "value": 42}, - {"sender": None, "value": 24}, # User input - {"sender": "comp2", "value": 33}, - ] - }, - {"greedy": [42]}, # Takes first valid input - {}, # All inputs removed for greedy sockets - ), - # Lazy variadic socket - ( - {"lazy": InputSocket("lazy", Variadic[int])}, - { - "lazy": [ - {"sender": "comp1", "value": 42}, - {"sender": "comp2", "value": 24}, - {"sender": None, "value": 33}, # User input - ] - }, - {"lazy": [42, 24, 33]}, # Takes all valid inputs - {"lazy": [{"sender": None, "value": 33}]}, # User input remains - ), - # Mixed socket types - ( - { - "regular": InputSocket("regular", int), - "greedy": InputSocket("greedy", GreedyVariadic[int]), - "lazy": InputSocket("lazy", Variadic[int]), - }, - { - "regular": [{"sender": "comp1", "value": 42}, {"sender": None, "value": 24}], - "greedy": [{"sender": "comp2", "value": 33}, {"sender": None, "value": 15}], - "lazy": [{"sender": "comp3", "value": 55}, {"sender": "comp4", "value": 66}], - }, - {"regular": 42, "greedy": [33], "lazy": [55, 66]}, - {"regular": [{"sender": None, "value": 24}]}, # Only non-greedy user input remains - ), - # Filtering _NO_OUTPUT_PRODUCED - ( - {"input1": InputSocket("input1", int)}, - { - "input1": [ - {"sender": "comp1", "value": _NO_OUTPUT_PRODUCED}, - {"sender": "comp2", "value": 42}, - {"sender": "comp2", "value": _NO_OUTPUT_PRODUCED}, - ] - }, - {"input1": 42}, # Should skip _NO_OUTPUT_PRODUCED values - {}, # All inputs consumed - ), - ], - ids=[ - "regular-socket", - "regular-with-user-input", - "greedy-variadic", - "lazy-variadic", - "mixed-sockets", - "no-output-filtering", - ], - ) - def test__consume_component_inputs(self, input_sockets, component_inputs, expected_consumed, expected_remaining): - # Setup - component = {"input_sockets": input_sockets} - inputs = {"test_component": component_inputs} - - # Run - consumed, updated_inputs = Pipeline._consume_component_inputs("test_component", component, inputs) - - # Verify - assert consumed == expected_consumed - assert updated_inputs["test_component"] == expected_remaining + assert "didn't return a dictionary" in str(exc_info.value) diff --git a/test/core/pipeline/test_pipeline_base.py b/test/core/pipeline/test_pipeline_base.py new file mode 100644 index 0000000000..4a9c09ac0a --- /dev/null +++ b/test/core/pipeline/test_pipeline_base.py @@ -0,0 +1,1637 @@ +# SPDX-FileCopyrightText: 2022-present deepset GmbH +# +# SPDX-License-Identifier: Apache-2.0 +import logging +from typing import List, Optional +from unittest.mock import patch + +import pytest + +from haystack import Document +from haystack.components.builders import PromptBuilder, AnswerBuilder +from haystack.components.joiners import BranchJoiner +from haystack.core.component import component +from haystack.core.component.types import InputSocket, OutputSocket, Variadic, GreedyVariadic, _empty +from haystack.core.errors import ( + DeserializationError, + PipelineConnectError, + PipelineDrawingError, + PipelineError, + PipelineRuntimeError, + PipelineMaxComponentRuns, +) +from haystack.core.pipeline import PredefinedPipeline +from haystack.core.pipeline.base import PipelineBase +from haystack.core.pipeline.base import ComponentPriority, _NO_OUTPUT_PRODUCED +from haystack.core.pipeline.utils import FIFOPriorityQueue + +from haystack.core.serialization import DeserializationCallbacks +from haystack.testing.factory import component_class +from haystack.testing.sample_components import AddFixedValue, Double, Greet + +logging.basicConfig(level=logging.DEBUG) + + +@component +class FakeComponent: + def __init__(self, an_init_param: Optional[str] = None): + pass + + @component.output_types(value=str) + def run(self, input_: str): + return {"value": input_} + + +@component +class FakeComponentSquared: + def __init__(self, an_init_param: Optional[str] = None): + self.an_init_param = an_init_param + self.inner = FakeComponent() + + @component.output_types(value=str) + def run(self, input_: str): + return {"value": input_} + + +@pytest.fixture +def regular_output_socket(): + """Output socket for a regular (non-variadic) connection with receivers""" + return OutputSocket("output1", int, receivers=["receiver1", "receiver2"]) + + +@pytest.fixture +def regular_input_socket(): + """Regular (non-variadic) input socket with a single sender""" + return InputSocket("input1", int, senders=["sender1"]) + + +@pytest.fixture +def lazy_variadic_input_socket(): + """Lazy variadic input socket with multiple senders""" + return InputSocket("variadic_input", Variadic[int], senders=["sender1", "sender2"]) + + +class TestPipelineBase: + """ + This class contains only unit tests for the PipelineBase class. + It doesn't test Pipeline.run(), that is done separately in a different way. + """ + + def test_pipeline_dumps(self, test_files_path): + pipeline = PipelineBase(max_runs_per_component=99) + pipeline.add_component("Comp1", FakeComponent("Foo")) + pipeline.add_component("Comp2", FakeComponent()) + pipeline.connect("Comp1.value", "Comp2.input_") + result = pipeline.dumps() + with open(f"{test_files_path}/yaml/test_pipeline.yaml", "r") as f: + assert f.read() == result + + def test_pipeline_loads_invalid_data(self): + invalid_yaml = """components: + Comp1: + init_parameters: + an_init_param: null + type: test.core.pipeline.test_pipeline_base.FakeComponent + Comp2* + init_parameters: + an_init_param: null + type: test.core.pipeline.test_pipeline_base.FakeComponent + connections: + * receiver: Comp2.input_ + sender: Comp1.value + metadata: + """ + + with pytest.raises(DeserializationError, match="unmarshalling serialized"): + pipeline = PipelineBase.loads(invalid_yaml) + + invalid_init_parameter_yaml = """components: + Comp1: + init_parameters: + unknown: null + type: test.core.pipeline.test_pipeline_base.FakeComponent + Comp2: + init_parameters: + an_init_param: null + type: test.core.pipeline.test_pipeline_base.FakeComponent + connections: + - receiver: Comp2.input_ + sender: Comp1.value + metadata: {} + """ + + with pytest.raises(DeserializationError, match=".*Comp1.*unknown.*"): + pipeline = PipelineBase.loads(invalid_init_parameter_yaml) + + def test_pipeline_dump(self, test_files_path, tmp_path): + pipeline = PipelineBase(max_runs_per_component=99) + pipeline.add_component("Comp1", FakeComponent("Foo")) + pipeline.add_component("Comp2", FakeComponent()) + pipeline.connect("Comp1.value", "Comp2.input_") + with open(tmp_path / "out.yaml", "w") as f: + pipeline.dump(f) + # re-open and ensure it's the same data as the test file + with open(f"{test_files_path}/yaml/test_pipeline.yaml", "r") as test_f, open(tmp_path / "out.yaml", "r") as f: + assert f.read() == test_f.read() + + def test_pipeline_load(self, test_files_path): + with open(f"{test_files_path}/yaml/test_pipeline.yaml", "r") as f: + pipeline = PipelineBase.load(f) + assert pipeline._max_runs_per_component == 99 + assert isinstance(pipeline.get_component("Comp1"), FakeComponent) + assert isinstance(pipeline.get_component("Comp2"), FakeComponent) + + @patch("haystack.core.pipeline.base._to_mermaid_image") + @patch("haystack.core.pipeline.base.is_in_jupyter") + @patch("IPython.display.Image") + @patch("IPython.display.display") + def test_show_in_notebook( + self, mock_ipython_display, mock_ipython_image, mock_is_in_jupyter, mock_to_mermaid_image + ): + pipe = PipelineBase() + + mock_to_mermaid_image.return_value = b"some_image_data" + mock_is_in_jupyter.return_value = True + + pipe.show() + mock_ipython_image.assert_called_once_with(b"some_image_data") + mock_ipython_display.assert_called_once() + + @patch("haystack.core.pipeline.base.is_in_jupyter") + def test_show_not_in_notebook(self, mock_is_in_jupyter): + pipe = PipelineBase() + + mock_is_in_jupyter.return_value = False + + with pytest.raises(PipelineDrawingError): + pipe.show() + + @patch("haystack.core.pipeline.base._to_mermaid_image") + def test_draw(self, mock_to_mermaid_image, tmp_path): + pipe = PipelineBase() + mock_to_mermaid_image.return_value = b"some_image_data" + + image_path = tmp_path / "test.png" + pipe.draw(path=image_path) + assert image_path.read_bytes() == mock_to_mermaid_image.return_value + + # UNIT + def test_add_component_to_different_pipelines(self): + first_pipe = PipelineBase() + second_pipe = PipelineBase() + some_component = component_class("Some")() + + assert some_component.__haystack_added_to_pipeline__ is None + first_pipe.add_component("some", some_component) + assert some_component.__haystack_added_to_pipeline__ is first_pipe + + with pytest.raises(PipelineError): + second_pipe.add_component("some", some_component) + + def test_remove_component_raises_if_invalid_component_name(self): + pipe = PipelineBase() + component = component_class("Some")() + + pipe.add_component("1", component) + + with pytest.raises(ValueError): + pipe.remove_component("2") + + def test_remove_component_removes_component_and_its_edges(self): + pipe = PipelineBase() + component_1 = component_class("Type1")() + component_2 = component_class("Type2")() + component_3 = component_class("Type3")() + component_4 = component_class("Type4")() + + pipe.add_component("1", component_1) + pipe.add_component("2", component_2) + pipe.add_component("3", component_3) + pipe.add_component("4", component_4) + + pipe.connect("1", "2") + pipe.connect("2", "3") + pipe.connect("3", "4") + + pipe.remove_component("2") + + assert ["1", "3", "4"] == sorted(pipe.graph.nodes) + assert [("3", "4")] == sorted([(u, v) for (u, v) in pipe.graph.edges()]) + + def test_remove_component_allows_you_to_reuse_the_component(self): + pipe = PipelineBase() + Some = component_class("Some", input_types={"in": int}, output_types={"out": int}) + + pipe.add_component("component_1", Some()) + pipe.add_component("component_2", Some()) + pipe.add_component("component_3", Some()) + pipe.connect("component_1", "component_2") + pipe.connect("component_2", "component_3") + component_2 = pipe.remove_component("component_2") + + assert component_2.__haystack_added_to_pipeline__ is None + assert component_2.__haystack_input__._sockets_dict == {"in": InputSocket(name="in", type=int, senders=[])} + assert component_2.__haystack_output__._sockets_dict == { + "out": OutputSocket(name="out", type=int, receivers=[]) + } + + pipe2 = PipelineBase() + pipe2.add_component("component_4", Some()) + pipe2.add_component("component_2", component_2) + pipe2.add_component("component_5", Some()) + + pipe2.connect("component_4", "component_2") + pipe2.connect("component_2", "component_5") + assert component_2.__haystack_added_to_pipeline__ is pipe2 + assert component_2.__haystack_input__._sockets_dict == { + "in": InputSocket(name="in", type=int, senders=["component_4"]) + } + assert component_2.__haystack_output__._sockets_dict == { + "out": OutputSocket(name="out", type=int, receivers=["component_5"]) + } + + # instance = pipe2.get_component("some") + # assert instance == component + + # UNIT + def test_get_component_name(self): + pipe = PipelineBase() + some_component = component_class("Some")() + pipe.add_component("some", some_component) + + assert pipe.get_component_name(some_component) == "some" + + # UNIT + def test_get_component_name_not_added_to_pipeline(self): + pipe = PipelineBase() + some_component = component_class("Some")() + + assert pipe.get_component_name(some_component) == "" + + # UNIT + def test_repr(self): + pipe = PipelineBase(metadata={"test": "test"}) + pipe.add_component("add_two", AddFixedValue(add=2)) + pipe.add_component("add_default", AddFixedValue()) + pipe.add_component("double", Double()) + pipe.connect("add_two", "double") + pipe.connect("double", "add_default") + + expected_repr = ( + f"{object.__repr__(pipe)}\n" + "🧱 Metadata\n" + " - test: test\n" + "🚅 Components\n" + " - add_two: AddFixedValue\n" + " - add_default: AddFixedValue\n" + " - double: Double\n" + "🛤️ Connections\n" + " - add_two.result -> double.value (int)\n" + " - double.value -> add_default.value (int)\n" + ) + + assert repr(pipe) == expected_repr + + # UNIT + def test_to_dict(self): + add_two = AddFixedValue(add=2) + add_default = AddFixedValue() + double = Double() + pipe = PipelineBase(metadata={"test": "test"}, max_runs_per_component=42) + pipe.add_component("add_two", add_two) + pipe.add_component("add_default", add_default) + pipe.add_component("double", double) + pipe.connect("add_two", "double") + pipe.connect("double", "add_default") + + res = pipe.to_dict() + expected = { + "metadata": {"test": "test"}, + "max_runs_per_component": 42, + "components": { + "add_two": { + "type": "haystack.testing.sample_components.add_value.AddFixedValue", + "init_parameters": {"add": 2}, + }, + "add_default": { + "type": "haystack.testing.sample_components.add_value.AddFixedValue", + "init_parameters": {"add": 1}, + }, + "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, + }, + "connections": [ + {"sender": "add_two.result", "receiver": "double.value"}, + {"sender": "double.value", "receiver": "add_default.value"}, + ], + } + assert res == expected + + def test_from_dict(self): + data = { + "metadata": {"test": "test"}, + "max_runs_per_component": 101, + "components": { + "add_two": { + "type": "haystack.testing.sample_components.add_value.AddFixedValue", + "init_parameters": {"add": 2}, + }, + "add_default": { + "type": "haystack.testing.sample_components.add_value.AddFixedValue", + "init_parameters": {"add": 1}, + }, + "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, + }, + "connections": [ + {"sender": "add_two.result", "receiver": "double.value"}, + {"sender": "double.value", "receiver": "add_default.value"}, + ], + } + pipe = PipelineBase.from_dict(data) + + assert pipe.metadata == {"test": "test"} + assert pipe._max_runs_per_component == 101 + + # Components + assert len(pipe.graph.nodes) == 3 + ## add_two + add_two = pipe.graph.nodes["add_two"] + assert add_two["instance"].add == 2 + assert add_two["input_sockets"] == { + "value": InputSocket(name="value", type=int), + "add": InputSocket(name="add", type=Optional[int], default_value=None), + } + assert add_two["output_sockets"] == {"result": OutputSocket(name="result", type=int, receivers=["double"])} + assert add_two["visits"] == 0 + + ## add_default + add_default = pipe.graph.nodes["add_default"] + assert add_default["instance"].add == 1 + assert add_default["input_sockets"] == { + "value": InputSocket(name="value", type=int, senders=["double"]), + "add": InputSocket(name="add", type=Optional[int], default_value=None), + } + assert add_default["output_sockets"] == {"result": OutputSocket(name="result", type=int)} + assert add_default["visits"] == 0 + + ## double + double = pipe.graph.nodes["double"] + assert double["instance"] + assert double["input_sockets"] == {"value": InputSocket(name="value", type=int, senders=["add_two"])} + assert double["output_sockets"] == {"value": OutputSocket(name="value", type=int, receivers=["add_default"])} + assert double["visits"] == 0 + + # Connections + connections = list(pipe.graph.edges(data=True)) + assert len(connections) == 2 + assert connections[0] == ( + "add_two", + "double", + { + "conn_type": "int", + "from_socket": OutputSocket(name="result", type=int, receivers=["double"]), + "to_socket": InputSocket(name="value", type=int, senders=["add_two"]), + "mandatory": True, + }, + ) + assert connections[1] == ( + "double", + "add_default", + { + "conn_type": "int", + "from_socket": OutputSocket(name="value", type=int, receivers=["add_default"]), + "to_socket": InputSocket(name="value", type=int, senders=["double"]), + "mandatory": True, + }, + ) + + # TODO: Remove this, this should be a component test. + # The pipeline can't handle this in any case nor way. + def test_from_dict_with_callbacks(self): + data = { + "metadata": {"test": "test"}, + "components": { + "add_two": { + "type": "haystack.testing.sample_components.add_value.AddFixedValue", + "init_parameters": {"add": 2}, + }, + "add_default": { + "type": "haystack.testing.sample_components.add_value.AddFixedValue", + "init_parameters": {"add": 1}, + }, + "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, + "greet": { + "type": "haystack.testing.sample_components.greet.Greet", + "init_parameters": {"message": "test"}, + }, + }, + "connections": [ + {"sender": "add_two.result", "receiver": "double.value"}, + {"sender": "double.value", "receiver": "add_default.value"}, + ], + } + + components_seen_in_callback = [] + + def component_pre_init_callback(name, component_cls, init_params): + assert name in ["add_two", "add_default", "double", "greet"] + assert component_cls in [AddFixedValue, Double, Greet] + + if name == "add_two": + assert init_params == {"add": 2} + elif name == "add_default": + assert init_params == {"add": 1} + elif name == "greet": + assert init_params == {"message": "test"} + + components_seen_in_callback.append(name) + + pipe = PipelineBase.from_dict( + data, callbacks=DeserializationCallbacks(component_pre_init=component_pre_init_callback) + ) + assert components_seen_in_callback == ["add_two", "add_default", "double", "greet"] + add_two = pipe.graph.nodes["add_two"]["instance"] + assert add_two.add == 2 + add_default = pipe.graph.nodes["add_default"]["instance"] + assert add_default.add == 1 + greet = pipe.graph.nodes["greet"]["instance"] + assert greet.message == "test" + assert greet.log_level == "INFO" + + def component_pre_init_callback_modify(name, component_cls, init_params): + assert name in ["add_two", "add_default", "double", "greet"] + assert component_cls in [AddFixedValue, Double, Greet] + + if name == "add_two": + init_params["add"] = 3 + elif name == "add_default": + init_params["add"] = 0 + elif name == "greet": + init_params["message"] = "modified test" + init_params["log_level"] = "DEBUG" + + pipe = PipelineBase.from_dict( + data, callbacks=DeserializationCallbacks(component_pre_init=component_pre_init_callback_modify) + ) + add_two = pipe.graph.nodes["add_two"]["instance"] + assert add_two.add == 3 + add_default = pipe.graph.nodes["add_default"]["instance"] + assert add_default.add == 0 + greet = pipe.graph.nodes["greet"]["instance"] + assert greet.message == "modified test" + assert greet.log_level == "DEBUG" + + # Test with a component that internally instantiates another component + def component_pre_init_callback_check_class(name, component_cls, init_params): + assert name == "fake_component_squared" + assert component_cls == FakeComponentSquared + + pipe = PipelineBase() + pipe.add_component("fake_component_squared", FakeComponentSquared()) + pipe = PipelineBase.from_dict( + pipe.to_dict(), + callbacks=DeserializationCallbacks(component_pre_init=component_pre_init_callback_check_class), + ) + assert type(pipe.graph.nodes["fake_component_squared"]["instance"].inner) == FakeComponent + + # UNIT + def test_from_dict_with_empty_dict(self): + assert PipelineBase() == PipelineBase.from_dict({}) + + # TODO: UNIT, consider deprecating this argument + def test_from_dict_with_components_instances(self): + add_two = AddFixedValue(add=2) + add_default = AddFixedValue() + components = {"add_two": add_two, "add_default": add_default} + data = { + "metadata": {"test": "test"}, + "components": { + "add_two": {}, + "add_default": {}, + "double": {"type": "haystack.testing.sample_components.double.Double", "init_parameters": {}}, + }, + "connections": [ + {"sender": "add_two.result", "receiver": "double.value"}, + {"sender": "double.value", "receiver": "add_default.value"}, + ], + } + pipe = PipelineBase.from_dict(data, components=components) + assert pipe.metadata == {"test": "test"} + + # Components + assert len(pipe.graph.nodes) == 3 + ## add_two + add_two_data = pipe.graph.nodes["add_two"] + assert add_two_data["instance"] is add_two + assert add_two_data["instance"].add == 2 + assert add_two_data["input_sockets"] == { + "value": InputSocket(name="value", type=int), + "add": InputSocket(name="add", type=Optional[int], default_value=None), + } + assert add_two_data["output_sockets"] == {"result": OutputSocket(name="result", type=int, receivers=["double"])} + assert add_two_data["visits"] == 0 + + ## add_default + add_default_data = pipe.graph.nodes["add_default"] + assert add_default_data["instance"] is add_default + assert add_default_data["instance"].add == 1 + assert add_default_data["input_sockets"] == { + "value": InputSocket(name="value", type=int, senders=["double"]), + "add": InputSocket(name="add", type=Optional[int], default_value=None), + } + assert add_default_data["output_sockets"] == {"result": OutputSocket(name="result", type=int, receivers=[])} + assert add_default_data["visits"] == 0 + + ## double + double = pipe.graph.nodes["double"] + assert double["instance"] + assert double["input_sockets"] == {"value": InputSocket(name="value", type=int, senders=["add_two"])} + assert double["output_sockets"] == {"value": OutputSocket(name="value", type=int, receivers=["add_default"])} + assert double["visits"] == 0 + + # Connections + connections = list(pipe.graph.edges(data=True)) + assert len(connections) == 2 + assert connections[0] == ( + "add_two", + "double", + { + "conn_type": "int", + "from_socket": OutputSocket(name="result", type=int, receivers=["double"]), + "to_socket": InputSocket(name="value", type=int, senders=["add_two"]), + "mandatory": True, + }, + ) + assert connections[1] == ( + "double", + "add_default", + { + "conn_type": "int", + "from_socket": OutputSocket(name="value", type=int, receivers=["add_default"]), + "to_socket": InputSocket(name="value", type=int, senders=["double"]), + "mandatory": True, + }, + ) + + # UNIT + def test_from_dict_without_component_type(self): + data = { + "metadata": {"test": "test"}, + "components": {"add_two": {"init_parameters": {"add": 2}}}, + "connections": [], + } + with pytest.raises(PipelineError) as err: + PipelineBase.from_dict(data) + + err.match("Missing 'type' in component 'add_two'") + + # UNIT + def test_from_dict_without_registered_component_type(self): + data = { + "metadata": {"test": "test"}, + "components": {"add_two": {"type": "foo.bar.baz", "init_parameters": {"add": 2}}}, + "connections": [], + } + with pytest.raises(PipelineError) as err: + PipelineBase.from_dict(data) + + err.match(r"Component .+ not imported.") + + def test_from_dict_with_invalid_type(self): + data = { + "metadata": {"test": "test"}, + "components": {"add_two": {"type": "", "init_parameters": {"add": 2}}}, + "connections": [], + } + with pytest.raises(PipelineError) as err: + PipelineBase.from_dict(data) + + err.match(r"Component '' \(name: 'add_two'\) not imported.") + + # UNIT + def test_from_dict_without_connection_sender(self): + data = {"metadata": {"test": "test"}, "components": {}, "connections": [{"receiver": "some.receiver"}]} + with pytest.raises(PipelineError) as err: + PipelineBase.from_dict(data) + + err.match("Missing sender in connection: {'receiver': 'some.receiver'}") + + # UNIT + def test_from_dict_without_connection_receiver(self): + data = {"metadata": {"test": "test"}, "components": {}, "connections": [{"sender": "some.sender"}]} + with pytest.raises(PipelineError) as err: + PipelineBase.from_dict(data) + + err.match("Missing receiver in connection: {'sender': 'some.sender'}") + + def test_describe_input_only_no_inputs_components(self): + A = component_class("A", input_types={}, output={"x": 0}) + B = component_class("B", input_types={}, output={"y": 0}) + C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) + p = PipelineBase() + p.add_component("a", A()) + p.add_component("b", B()) + p.add_component("c", C()) + p.connect("a.x", "c.x") + p.connect("b.y", "c.y") + assert p.inputs() == {} + assert p.inputs(include_components_with_connected_inputs=True) == { + "c": {"x": {"type": int, "is_mandatory": True}, "y": {"type": int, "is_mandatory": True}} + } + + def test_describe_input_some_components_with_no_inputs(self): + A = component_class("A", input_types={}, output={"x": 0}) + B = component_class("B", input_types={"y": int}, output={"y": 0}) + C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) + p = PipelineBase() + p.add_component("a", A()) + p.add_component("b", B()) + p.add_component("c", C()) + p.connect("a.x", "c.x") + p.connect("b.y", "c.y") + assert p.inputs() == {"b": {"y": {"type": int, "is_mandatory": True}}} + assert p.inputs(include_components_with_connected_inputs=True) == { + "b": {"y": {"type": int, "is_mandatory": True}}, + "c": {"x": {"type": int, "is_mandatory": True}, "y": {"type": int, "is_mandatory": True}}, + } + + def test_describe_input_all_components_have_inputs(self): + A = component_class("A", input_types={"x": Optional[int]}, output={"x": 0}) + B = component_class("B", input_types={"y": int}, output={"y": 0}) + C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) + p = PipelineBase() + p.add_component("a", A()) + p.add_component("b", B()) + p.add_component("c", C()) + p.connect("a.x", "c.x") + p.connect("b.y", "c.y") + assert p.inputs() == { + "a": {"x": {"type": Optional[int], "is_mandatory": True}}, + "b": {"y": {"type": int, "is_mandatory": True}}, + } + assert p.inputs(include_components_with_connected_inputs=True) == { + "a": {"x": {"type": Optional[int], "is_mandatory": True}}, + "b": {"y": {"type": int, "is_mandatory": True}}, + "c": {"x": {"type": int, "is_mandatory": True}, "y": {"type": int, "is_mandatory": True}}, + } + + def test_describe_output_multiple_possible(self): + """ + This pipeline has two outputs: + {"b": {"output_b": {"type": str}}, "a": {"output_a": {"type": str}}} + """ + A = component_class("A", input_types={"input_a": str}, output={"output_a": "str", "output_b": "str"}) + B = component_class("B", input_types={"input_b": str}, output={"output_b": "str"}) + + pipe = PipelineBase() + pipe.add_component("a", A()) + pipe.add_component("b", B()) + pipe.connect("a.output_b", "b.input_b") + + assert pipe.outputs() == {"b": {"output_b": {"type": str}}, "a": {"output_a": {"type": str}}} + assert pipe.outputs(include_components_with_connected_outputs=True) == { + "a": {"output_a": {"type": str}, "output_b": {"type": str}}, + "b": {"output_b": {"type": str}}, + } + + def test_describe_output_single(self): + """ + This pipeline has one output: + {"c": {"z": {"type": int}}} + """ + A = component_class("A", input_types={"x": Optional[int]}, output={"x": 0}) + B = component_class("B", input_types={"y": int}, output={"y": 0}) + C = component_class("C", input_types={"x": int, "y": int}, output={"z": 0}) + p = PipelineBase() + p.add_component("a", A()) + p.add_component("b", B()) + p.add_component("c", C()) + p.connect("a.x", "c.x") + p.connect("b.y", "c.y") + + assert p.outputs() == {"c": {"z": {"type": int}}} + assert p.outputs(include_components_with_connected_outputs=True) == { + "a": {"x": {"type": int}}, + "b": {"y": {"type": int}}, + "c": {"z": {"type": int}}, + } + + def test_describe_no_outputs(self): + """ + This pipeline sets up elaborate connections between three components but in fact it has no outputs: + Check that p.outputs() == {} + """ + A = component_class("A", input_types={"x": Optional[int]}, output={"x": 0}) + B = component_class("B", input_types={"y": int}, output={"y": 0}) + C = component_class("C", input_types={"x": int, "y": int}, output={}) + p = PipelineBase() + p.add_component("a", A()) + p.add_component("b", B()) + p.add_component("c", C()) + p.connect("a.x", "c.x") + p.connect("b.y", "c.y") + assert p.outputs() == {} + assert p.outputs(include_components_with_connected_outputs=True) == { + "a": {"x": {"type": int}}, + "b": {"y": {"type": int}}, + } + + def test_from_template(self, monkeypatch): + monkeypatch.setenv("OPENAI_API_KEY", "fake_key") + pipe = PipelineBase.from_template(PredefinedPipeline.INDEXING) + assert pipe.get_component("cleaner") + + def test_walk_pipeline_with_no_cycles(self): + """ + This pipeline has two source nodes, source1 and source2, one hello3 node in between, and one sink node, joiner. + pipeline.walk() should return each component exactly once. The order is not guaranteed. + """ + + @component + class Hello: + @component.output_types(output=str) + def run(self, word: str): + """ + Takes a string in input and returns "Hello, !" in output. + """ + return {"output": f"Hello, {word}!"} + + @component + class Joiner: + @component.output_types(output=str) + def run(self, word1: str, word2: str): + """ + Takes two strings in input and returns "Hello, and !" in output. + """ + return {"output": f"Hello, {word1} and {word2}!"} + + pipeline = PipelineBase() + source1 = Hello() + source2 = Hello() + hello3 = Hello() + joiner = Joiner() + pipeline.add_component("source1", source1) + pipeline.add_component("source2", source2) + pipeline.add_component("hello3", hello3) + pipeline.add_component("joiner", joiner) + + pipeline.connect("source1", "joiner.word1") + pipeline.connect("source2", "hello3") + pipeline.connect("hello3", "joiner.word2") + + expected_components = [("source1", source1), ("source2", source2), ("joiner", joiner), ("hello3", hello3)] + assert sorted(expected_components) == sorted(pipeline.walk()) + + def test_walk_pipeline_with_cycles(self): + """ + This pipeline consists of two components, which would run three times in a loop. + pipeline.walk() should return these components exactly once. The order is not guaranteed. + """ + + @component + class Hello: + def __init__(self): + self.iteration_counter = 0 + + @component.output_types(intermediate=str, final=str) + def run(self, word: str, intermediate: Optional[str] = None): + """ + Takes a string in input and returns "Hello, !" in output. + """ + if self.iteration_counter < 3: + self.iteration_counter += 1 + return {"intermediate": f"Hello, {intermediate or word}!"} + return {"final": f"Hello, {intermediate or word}!"} + + pipeline = PipelineBase() + hello = Hello() + hello_again = Hello() + pipeline.add_component("hello", hello) + pipeline.add_component("hello_again", hello_again) + pipeline.connect("hello.intermediate", "hello_again.intermediate") + pipeline.connect("hello_again.intermediate", "hello.intermediate") + assert {("hello", hello), ("hello_again", hello_again)} == set(pipeline.walk()) + + def test__init_graph(self): + pipe = PipelineBase() + pipe.add_component("greet", Greet()) + pipe.add_component("adder", AddFixedValue()) + pipe.connect("greet", "adder") + pipe._init_graph() + for node in pipe.graph.nodes: + assert pipe.graph.nodes[node]["visits"] == 0 + + def test__prepare_component_input_data(self): + MockComponent = component_class("MockComponent", input_types={"x": List[str], "y": str}) + pipe = PipelineBase() + pipe.add_component("first_mock", MockComponent()) + pipe.add_component("second_mock", MockComponent()) + + res = pipe._prepare_component_input_data({"x": ["some data"], "y": "some other data"}) + assert res == { + "first_mock": {"x": ["some data"], "y": "some other data"}, + "second_mock": {"x": ["some data"], "y": "some other data"}, + } + assert id(res["first_mock"]["x"]) != id(res["second_mock"]["x"]) + + def test__prepare_component_input_data_with_connected_inputs(self): + MockComponent = component_class( + "MockComponent", input_types={"x": List[str], "y": str}, output_types={"z": str} + ) + pipe = PipelineBase() + pipe.add_component("first_mock", MockComponent()) + pipe.add_component("second_mock", MockComponent()) + pipe.connect("first_mock.z", "second_mock.y") + + res = pipe._prepare_component_input_data({"x": ["some data"], "y": "some other data"}) + assert res == {"first_mock": {"x": ["some data"], "y": "some other data"}, "second_mock": {"x": ["some data"]}} + assert id(res["first_mock"]["x"]) != id(res["second_mock"]["x"]) + + def test__prepare_component_input_data_with_non_existing_input(self, caplog): + pipe = PipelineBase() + res = pipe._prepare_component_input_data({"input_name": 1}) + assert res == {} + assert ( + "Inputs ['input_name'] were not matched to any component inputs, " + "please check your run parameters." in caplog.text + ) + + def test_connect(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + assert pipe.connect("comp1.value", "comp2.value") is pipe + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_already_connected(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.connect("comp1.value", "comp2.value") + pipe.connect("comp1.value", "comp2.value") + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_with_sender_component_name(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.connect("comp1", "comp2.value") + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_with_receiver_component_name(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.connect("comp1.value", "comp2") + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_with_sender_and_receiver_component_name(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.connect("comp1", "comp2") + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_with_sender_not_in_pipeline(self): + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp2", comp2) + with pytest.raises(ValueError): + pipe.connect("comp1.value", "comp2.value") + + def test_connect_with_receiver_not_in_pipeline(self): + comp1 = component_class("Comp1", output_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + with pytest.raises(ValueError): + pipe.connect("comp1.value", "comp2.value") + + def test_connect_with_sender_socket_name_not_in_pipeline(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + with pytest.raises(PipelineConnectError): + pipe.connect("comp1.non_existing", "comp2.value") + + def test_connect_with_receiver_socket_name_not_in_pipeline(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + with pytest.raises(PipelineConnectError): + pipe.connect("comp1.value", "comp2.non_existing") + + def test_connect_with_no_matching_types_and_same_names(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": str})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + with pytest.raises(PipelineConnectError): + pipe.connect("comp1", "comp2") + + def test_connect_with_multiple_sender_connections_with_same_type_and_differing_name(self): + comp1 = component_class("Comp1", output_types={"val1": int, "val2": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + with pytest.raises(PipelineConnectError): + pipe.connect("comp1", "comp2") + + def test_connect_with_multiple_receiver_connections_with_same_type_and_differing_name(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"val1": int, "val2": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + with pytest.raises(PipelineConnectError): + pipe.connect("comp1", "comp2") + + def test_connect_with_multiple_sender_connections_with_same_type_and_same_name(self): + comp1 = component_class("Comp1", output_types={"value": int, "other": int})() + comp2 = component_class("Comp2", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.connect("comp1", "comp2") + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_with_multiple_receiver_connections_with_same_type_and_same_name(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", input_types={"value": int, "other": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.connect("comp1", "comp2") + + assert comp1.__haystack_output__.value.receivers == ["comp2"] + assert comp2.__haystack_input__.value.senders == ["comp1"] + assert list(pipe.graph.edges) == [("comp1", "comp2", "value/value")] + + def test_connect_multiple_outputs_to_non_variadic_input(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", output_types={"value": int})() + comp3 = component_class("Comp3", input_types={"value": int})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.add_component("comp3", comp3) + pipe.connect("comp1.value", "comp3.value") + with pytest.raises(PipelineConnectError): + pipe.connect("comp2.value", "comp3.value") + + def test_connect_multiple_outputs_to_variadic_input(self): + comp1 = component_class("Comp1", output_types={"value": int})() + comp2 = component_class("Comp2", output_types={"value": int})() + comp3 = component_class("Comp3", input_types={"value": Variadic[int]})() + pipe = PipelineBase() + pipe.add_component("comp1", comp1) + pipe.add_component("comp2", comp2) + pipe.add_component("comp3", comp3) + pipe.connect("comp1.value", "comp3.value") + pipe.connect("comp2.value", "comp3.value") + + assert comp1.__haystack_output__.value.receivers == ["comp3"] + assert comp2.__haystack_output__.value.receivers == ["comp3"] + assert comp3.__haystack_input__.value.senders == ["comp1", "comp2"] + assert list(pipe.graph.edges) == [("comp1", "comp3", "value/value"), ("comp2", "comp3", "value/value")] + + def test_connect_same_component_as_sender_and_receiver(self): + """ + This pipeline consists of one component, which would be connected to itself. + Connecting a component to itself is raises PipelineConnectError. + """ + pipe = PipelineBase() + single_component = FakeComponent() + pipe.add_component("single_component", single_component) + with pytest.raises(PipelineConnectError): + pipe.connect("single_component.out", "single_component.in") + + @pytest.mark.parametrize( + "component_inputs,sockets,expected_inputs", + [ + ({"mandatory": 1}, {"mandatory": InputSocket("mandatory", int)}, {"mandatory": 1}), + ({}, {"optional": InputSocket("optional", str, default_value="test")}, {"optional": "test"}), + ( + {"mandatory": 1}, + { + "mandatory": InputSocket("mandatory", int), + "optional": InputSocket("optional", str, default_value="test"), + }, + {"mandatory": 1, "optional": "test"}, + ), + ( + {}, + {"optional_variadic": InputSocket("optional_variadic", Variadic[str], default_value="test")}, + {"optional_variadic": ["test"]}, + ), + ( + {}, + { + "optional_1": InputSocket("optional_1", int, default_value=1), + "optional_2": InputSocket("optional_2", int, default_value=2), + }, + {"optional_1": 1, "optional_2": 2}, + ), + ], + ids=["no-defaults", "only-default", "mixed-default", "variadic-default", "multiple_defaults"], + ) + def test__add_missing_defaults(self, component_inputs, sockets, expected_inputs): + filled_inputs = PipelineBase._add_missing_input_defaults(component_inputs, sockets) + + assert filled_inputs == expected_inputs + + def test__find_receivers_from(self): + sentence_builder = component_class( + "SentenceBuilder", input_types={"words": List[str]}, output_types={"text": str} + )() + document_builder = component_class( + "DocumentBuilder", input_types={"text": str}, output_types={"doc": Document} + )() + conditional_document_builder = component_class( + "ConditionalDocumentBuilder", output_types={"doc": Document, "noop": None} + )() + + document_joiner = component_class("DocumentJoiner", input_types={"docs": Variadic[Document]})() + + pipe = PipelineBase() + pipe.add_component("sentence_builder", sentence_builder) + pipe.add_component("document_builder", document_builder) + pipe.add_component("document_joiner", document_joiner) + pipe.add_component("conditional_document_builder", conditional_document_builder) + pipe.connect("sentence_builder.text", "document_builder.text") + pipe.connect("document_builder.doc", "document_joiner.docs") + pipe.connect("conditional_document_builder.doc", "document_joiner.docs") + + res = pipe._find_receivers_from("sentence_builder") + assert res == [ + ( + "document_builder", + OutputSocket(name="text", type=str, receivers=["document_builder"]), + InputSocket(name="text", type=str, default_value=_empty, senders=["sentence_builder"]), + ) + ] + + res = pipe._find_receivers_from("document_builder") + assert res == [ + ( + "document_joiner", + OutputSocket(name="doc", type=Document, receivers=["document_joiner"]), + InputSocket( + name="docs", + type=Variadic[Document], + default_value=_empty, + senders=["document_builder", "conditional_document_builder"], + ), + ) + ] + + res = pipe._find_receivers_from("document_joiner") + assert res == [] + + res = pipe._find_receivers_from("conditional_document_builder") + assert res == [ + ( + "document_joiner", + OutputSocket(name="doc", type=Document, receivers=["document_joiner"]), + InputSocket( + name="docs", + type=Variadic[Document], + default_value=_empty, + senders=["document_builder", "conditional_document_builder"], + ), + ) + ] + + @pytest.mark.parametrize( + "component, inputs, expected_priority, test_description", + [ + # Test case 1: BLOCKED - Missing mandatory input + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int), + "optional_input": InputSocket( + "optional_input", str, default_value="default", senders=["previous_component"] + ), + }, + }, + {"optional_input": [{"sender": "previous_component", "value": "test"}]}, + ComponentPriority.BLOCKED, + "Component should be BLOCKED when mandatory input is missing", + ), + # Test case 2: BLOCKED - No trigger after first visit + ( + { + "instance": "mock_instance", + "visits": 1, # Already visited + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int), + "optional_input": InputSocket("optional_input", str, default_value="default"), + }, + }, + {"mandatory_input": [{"sender": None, "value": 42}]}, + ComponentPriority.BLOCKED, + "Component should be BLOCKED when there's no new trigger after first visit", + ), + # Test case 3: HIGHEST - Greedy socket ready + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "greedy_input": InputSocket("greedy_input", GreedyVariadic[int], senders=["component1"]), + "normal_input": InputSocket("normal_input", str, senders=["component2"]), + }, + }, + { + "greedy_input": [{"sender": "component1", "value": 42}], + "normal_input": [{"sender": "component2", "value": "test"}], + }, + ComponentPriority.HIGHEST, + "Component should have HIGHEST priority when greedy socket has valid input", + ), + # Test case 4: DEFER - Greedy socket ready but optional missing + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "greedy_input": InputSocket("greedy_input", GreedyVariadic[int], senders=["component1"]), + "optional_input": InputSocket( + "optional_input", str, senders=["component2"], default_value="test" + ), + }, + }, + {"greedy_input": [{"sender": "component1", "value": 42}]}, + ComponentPriority.DEFER, + "Component should DEFER when greedy socket has valid input but expected optional input is missing", + ), + # Test case 4: READY - All predecessors executed + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "mandatory_input": InputSocket("mandatory_input", int, senders=["previous_component"]), + "optional_input": InputSocket( + "optional_input", str, senders=["another_component"], default_value="default" + ), + }, + }, + { + "mandatory_input": [{"sender": "previous_component", "value": 42}], + "optional_input": [{"sender": "another_component", "value": "test"}], + }, + ComponentPriority.READY, + "Component should be READY when all predecessors have executed", + ), + # Test case 5: DEFER - Lazy variadic sockets resolved and optional missing. + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "variadic_input": InputSocket( + "variadic_input", Variadic[int], senders=["component1", "component2"] + ), + "normal_input": InputSocket("normal_input", str, senders=["component3"]), + "optional_input": InputSocket( + "optional_input", str, default_value="default", senders=["component4"] + ), + }, + }, + { + "variadic_input": [ + {"sender": "component1", "value": "test"}, + {"sender": "component2", "value": _NO_OUTPUT_PRODUCED}, + ], + "normal_input": [{"sender": "component3", "value": "test"}], + }, + ComponentPriority.DEFER, + "Component should DEFER when all lazy variadic sockets are resolved", + ), + # Test case 6: DEFER_LAST - Incomplete variadic inputs + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": { + "variadic_input": InputSocket( + "variadic_input", Variadic[int], senders=["component1", "component2"] + ), + "normal_input": InputSocket("normal_input", str), + }, + }, + { + "variadic_input": [{"sender": "component1", "value": 42}], # Missing component2 + "normal_input": [{"sender": "component3", "value": "test"}], + }, + ComponentPriority.DEFER_LAST, + "Component should be DEFER_LAST when not all variadic senders have produced output", + ), + # Test case 7: READY - No input sockets, first visit + ( + { + "instance": "mock_instance", + "visits": 0, + "input_sockets": {"optional_input": InputSocket("optional_input", str, default_value="default")}, + }, + {}, # no inputs + ComponentPriority.READY, + "Component should be READY on first visit when it has no input sockets", + ), + # Test case 8: BLOCKED - No connected input sockets, subsequent visit + ( + { + "instance": "mock_instance", + "visits": 1, + "input_sockets": {"optional_input": InputSocket("optional_input", str, default_value="default")}, + }, + {}, # no inputs + ComponentPriority.BLOCKED, + "Component should be BLOCKED on subsequent visits when it has no input sockets", + ), + ], + ids=lambda p: p.name if isinstance(p, ComponentPriority) else str(p), + ) + def test__calculate_priority(self, component, inputs, expected_priority, test_description): + """Test priority calculation for various component and input combinations.""" + # For variadic inputs, set up senders if needed + for socket in component["input_sockets"].values(): + if socket.is_variadic and not hasattr(socket, "senders"): + socket.senders = ["component1", "component2"] + + assert PipelineBase._calculate_priority(component, inputs) == expected_priority + + @pytest.mark.parametrize( + "pipeline_inputs,expected_output", + [ + # Test case 1: Empty input + ({}, {}), + # Test case 2: Single component, multiple inputs + ( + {"component1": {"input1": 42, "input2": "test", "input3": True}}, + { + "component1": { + "input1": [{"sender": None, "value": 42}], + "input2": [{"sender": None, "value": "test"}], + "input3": [{"sender": None, "value": True}], + } + }, + ), + # Test case 3: Multiple components + ( + { + "component1": {"input1": 42, "input2": "test"}, + "component2": {"input3": [1, 2, 3], "input4": {"key": "value"}}, + }, + { + "component1": { + "input1": [{"sender": None, "value": 42}], + "input2": [{"sender": None, "value": "test"}], + }, + "component2": { + "input3": [{"sender": None, "value": [1, 2, 3]}], + "input4": [{"sender": None, "value": {"key": "value"}}], + }, + }, + ), + ], + ids=["empty_input", "single_component_multiple_inputs", "multiple_components"], + ) + def test__convert_to_internal_format(self, pipeline_inputs, expected_output): + """Test conversion of legacy pipeline inputs to internal format.""" + result = PipelineBase._convert_to_internal_format(pipeline_inputs) + assert result == expected_output + + @pytest.mark.parametrize( + "socket_type,existing_inputs,expected_count", + [ + ("regular", None, 1), # Regular socket should overwrite + ("regular", [{"sender": "other", "value": 24}], 1), # Should still overwrite + ("lazy_variadic", None, 1), # First input to lazy variadic + ("lazy_variadic", [{"sender": "other", "value": 24}], 2), # Should append + ], + ids=["regular-new", "regular-existing", "variadic-new", "variadic-existing"], + ) + def test__write_component_outputs_different_sockets( + self, + socket_type, + existing_inputs, + expected_count, + regular_output_socket, + regular_input_socket, + lazy_variadic_input_socket, + ): + """Test writing to different socket types with various existing input states""" + receiver_socket = lazy_variadic_input_socket if socket_type == "lazy_variadic" else regular_input_socket + socket_name = receiver_socket.name + receivers = [("receiver1", regular_output_socket, receiver_socket)] + + inputs = {} + if existing_inputs: + inputs = {"receiver1": {socket_name: existing_inputs}} + + component_outputs = {"output1": 42} + + pruned_outputs, updated_inputs = PipelineBase._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs=inputs, + receivers=receivers, + include_outputs_from=[], + ) + + assert len(updated_inputs["receiver1"][socket_name]) == expected_count + assert {"sender": "sender1", "value": 42} in updated_inputs["receiver1"][socket_name] + + @pytest.mark.parametrize( + "component_outputs,include_outputs,expected_pruned", + [ + ({"output1": 42, "output2": 24}, [], {"output2": 24}), # Prune consumed outputs only + ({"output1": 42, "output2": 24}, ["sender1"], {"output1": 42, "output2": 24}), # Keep all outputs + ({}, [], {}), # No outputs case + ], + ids=["prune-consumed", "keep-all", "no-outputs"], + ) + def test__write_component_outputs_output_pruning( + self, component_outputs, include_outputs, expected_pruned, regular_output_socket, regular_input_socket + ): + """Test output pruning behavior under different scenarios""" + receivers = [("receiver1", regular_output_socket, regular_input_socket)] + + pruned_outputs, _ = PipelineBase._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs={}, + receivers=receivers, + include_outputs_from=include_outputs, + ) + + assert pruned_outputs == expected_pruned + + @pytest.mark.parametrize( + "output_value", + [42, None, _NO_OUTPUT_PRODUCED, "string_value", 3.14], + ids=["int", "none", "no-output", "string", "float"], + ) + def test__write_component_outputs_different_output_values( + self, output_value, regular_output_socket, regular_input_socket + ): + """Test handling of different output values""" + receivers = [("receiver1", regular_output_socket, regular_input_socket)] + component_outputs = {"output1": output_value} + + _, updated_inputs = PipelineBase._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs={}, + receivers=receivers, + include_outputs_from=[], + ) + + assert updated_inputs["receiver1"]["input1"] == [{"sender": "sender1", "value": output_value}] + + @pytest.mark.parametrize("receivers_count", [1, 2, 3], ids=["single-receiver", "two-receivers", "three-receivers"]) + def test__write_component_outputs_multiple_receivers( + self, receivers_count, regular_output_socket, regular_input_socket + ): + """Test writing to multiple receivers""" + receivers = [(f"receiver{i}", regular_output_socket, regular_input_socket) for i in range(receivers_count)] + component_outputs = {"output1": 42} + + _, updated_inputs = PipelineBase._write_component_outputs( + component_name="sender1", + component_outputs=component_outputs, + inputs={}, + receivers=receivers, + include_outputs_from=[], + ) + + for i in range(receivers_count): + receiver_name = f"receiver{i}" + assert receiver_name in updated_inputs + assert updated_inputs[receiver_name]["input1"] == [{"sender": "sender1", "value": 42}] + + def test__get_next_runnable_component_empty(self): + """Test with empty queue returns None""" + queue = FIFOPriorityQueue() + pipeline = PipelineBase() + result = pipeline._get_next_runnable_component(queue, component_visits={}) + assert result is None + + def test__get_next_runnable_component_blocked(self): + """Test component with BLOCKED priority returns None""" + pipeline = PipelineBase() + queue = FIFOPriorityQueue() + queue.push("blocked_component", ComponentPriority.BLOCKED) + result = pipeline._get_next_runnable_component(queue, component_visits={"blocked_component": 0}) + assert result is None + + @patch("haystack.core.pipeline.base.PipelineBase._get_component_with_graph_metadata_and_visits") + def test__get_next_runnable_component_max_visits(self, mock_get_component_with_graph_metadata_and_visits): + """Test component exceeding max visits raises exception""" + pipeline = PipelineBase(max_runs_per_component=2) + queue = FIFOPriorityQueue() + queue.push("ready_component", ComponentPriority.READY) + mock_get_component_with_graph_metadata_and_visits.return_value = {"instance": "test", "visits": 3} + + with pytest.raises(PipelineMaxComponentRuns) as exc_info: + pipeline._get_next_runnable_component(queue, component_visits={"ready_component": 3}) + + assert "Maximum run count 2 reached for component 'ready_component'" in str(exc_info.value) + + @patch("haystack.core.pipeline.base.PipelineBase._get_component_with_graph_metadata_and_visits") + def test__get_next_runnable_component_ready(self, mock_get_component_with_graph_metadata_and_visits): + """Test component that is READY""" + pipeline = PipelineBase() + queue = FIFOPriorityQueue() + queue.push("ready_component", ComponentPriority.READY) + mock_get_component_with_graph_metadata_and_visits.return_value = {"instance": "test", "visits": 1} + + priority, component_name, component = pipeline._get_next_runnable_component( + queue, component_visits={"ready_component": 1} + ) + + assert priority == ComponentPriority.READY + assert component_name == "ready_component" + assert component == {"instance": "test", "visits": 1} + + @pytest.mark.parametrize( + "queue_setup,expected_stale", + [ + # Empty queue case + (None, True), + # READY priority case + ((ComponentPriority.READY, "component1"), False), + # DEFER priority case + ((ComponentPriority.DEFER, "component1"), True), + ], + ids=["empty-queue", "ready-component", "deferred-component"], + ) + def test__is_queue_stale(self, queue_setup, expected_stale): + queue = FIFOPriorityQueue() + if queue_setup: + priority, component_name = queue_setup + queue.push(component_name, priority) + + result = PipelineBase._is_queue_stale(queue) + assert result == expected_stale + + @patch("haystack.core.pipeline.base.PipelineBase._calculate_priority") + @patch("haystack.core.pipeline.base.PipelineBase._get_component_with_graph_metadata_and_visits") + def test_fill_queue(self, mock_get_metadata, mock_calc_priority): + pipeline = PipelineBase() + component_names = ["comp1", "comp2"] + inputs = {"comp1": {"input1": "value1"}, "comp2": {"input2": "value2"}} + + mock_get_metadata.side_effect = lambda name, _: {"component": f"mock_{name}"} + mock_calc_priority.side_effect = [1, 2] # Different priorities for testing + + queue = pipeline._fill_queue(component_names, inputs, component_visits={"comp1": 1, "comp2": 1}) + + assert mock_get_metadata.call_count == 2 + assert mock_calc_priority.call_count == 2 + + # Verify correct calls for first component + mock_get_metadata.assert_any_call("comp1", 1) + mock_calc_priority.assert_any_call({"component": "mock_comp1"}, {"input1": "value1"}) + + # Verify correct calls for second component + mock_get_metadata.assert_any_call("comp2", 1) + mock_calc_priority.assert_any_call({"component": "mock_comp2"}, {"input2": "value2"}) + + assert queue.pop() == (1, "comp1") + assert queue.pop() == (2, "comp2") + + @pytest.mark.parametrize( + "input_sockets,component_inputs,expected_consumed,expected_remaining", + [ + # Regular socket test + ( + {"input1": InputSocket("input1", int)}, + {"input1": [{"sender": "comp1", "value": 42}, {"sender": "comp2", "value": 24}]}, + {"input1": 42}, # Should take first valid input + {}, # All pipeline inputs should be removed + ), + # Regular socket with user input + ( + {"input1": InputSocket("input1", int)}, + { + "input1": [ + {"sender": "comp1", "value": 42}, + {"sender": None, "value": 24}, # User input + ] + }, + {"input1": 42}, + {"input1": [{"sender": None, "value": 24}]}, # User input should remain + ), + # Greedy variadic socket + ( + {"greedy": InputSocket("greedy", GreedyVariadic[int])}, + { + "greedy": [ + {"sender": "comp1", "value": 42}, + {"sender": None, "value": 24}, # User input + {"sender": "comp2", "value": 33}, + ] + }, + {"greedy": [42]}, # Takes first valid input + {}, # All inputs removed for greedy sockets + ), + # Lazy variadic socket + ( + {"lazy": InputSocket("lazy", Variadic[int])}, + { + "lazy": [ + {"sender": "comp1", "value": 42}, + {"sender": "comp2", "value": 24}, + {"sender": None, "value": 33}, # User input + ] + }, + {"lazy": [42, 24, 33]}, # Takes all valid inputs + {"lazy": [{"sender": None, "value": 33}]}, # User input remains + ), + # Mixed socket types + ( + { + "regular": InputSocket("regular", int), + "greedy": InputSocket("greedy", GreedyVariadic[int]), + "lazy": InputSocket("lazy", Variadic[int]), + }, + { + "regular": [{"sender": "comp1", "value": 42}, {"sender": None, "value": 24}], + "greedy": [{"sender": "comp2", "value": 33}, {"sender": None, "value": 15}], + "lazy": [{"sender": "comp3", "value": 55}, {"sender": "comp4", "value": 66}], + }, + {"regular": 42, "greedy": [33], "lazy": [55, 66]}, + {"regular": [{"sender": None, "value": 24}]}, # Only non-greedy user input remains + ), + # Filtering _NO_OUTPUT_PRODUCED + ( + {"input1": InputSocket("input1", int)}, + { + "input1": [ + {"sender": "comp1", "value": _NO_OUTPUT_PRODUCED}, + {"sender": "comp2", "value": 42}, + {"sender": "comp2", "value": _NO_OUTPUT_PRODUCED}, + ] + }, + {"input1": 42}, # Should skip _NO_OUTPUT_PRODUCED values + {}, # All inputs consumed + ), + ], + ids=[ + "regular-socket", + "regular-with-user-input", + "greedy-variadic", + "lazy-variadic", + "mixed-sockets", + "no-output-filtering", + ], + ) + def test__consume_component_inputs(self, input_sockets, component_inputs, expected_consumed, expected_remaining): + # Setup + component = {"input_sockets": input_sockets} + inputs = {"test_component": component_inputs} + + # Run + consumed, updated_inputs = PipelineBase._consume_component_inputs("test_component", component, inputs) + + # Verify + assert consumed == expected_consumed + assert updated_inputs["test_component"] == expected_remaining diff --git a/test/test_files/yaml/test_pipeline.yaml b/test/test_files/yaml/test_pipeline.yaml index 945ef018ae..53c281d30c 100644 --- a/test/test_files/yaml/test_pipeline.yaml +++ b/test/test_files/yaml/test_pipeline.yaml @@ -2,11 +2,11 @@ components: Comp1: init_parameters: an_init_param: null - type: test.core.pipeline.test_pipeline.FakeComponent + type: test.core.pipeline.test_pipeline_base.FakeComponent Comp2: init_parameters: an_init_param: null - type: test.core.pipeline.test_pipeline.FakeComponent + type: test.core.pipeline.test_pipeline_base.FakeComponent connections: - receiver: Comp2.input_ sender: Comp1.value From 3b8d8862561c7ce217f045bbd87c3a9c844815f1 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 14:21:32 +0100 Subject: [PATCH 52/88] refactor: remove unused method --- haystack/core/pipeline/base.py | 5 ----- test/core/pipeline/test_pipeline_base.py | 8 -------- 2 files changed, 13 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 9794433711..3f506b8cd0 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -811,11 +811,6 @@ def from_template( msg += f"Source:\n{rendered}" raise PipelineUnmarshalError(msg) - def _init_graph(self): - """Resets the visits count for each component""" - for node in self.graph.nodes: - self.graph.nodes[node]["visits"] = 0 - def _find_receivers_from(self, component_name: str) -> List[Tuple[str, OutputSocket, InputSocket]]: """ Utility function to find all Components that receive input form `component_name`. diff --git a/test/core/pipeline/test_pipeline_base.py b/test/core/pipeline/test_pipeline_base.py index 4a9c09ac0a..69fc5c7502 100644 --- a/test/core/pipeline/test_pipeline_base.py +++ b/test/core/pipeline/test_pipeline_base.py @@ -811,14 +811,6 @@ def run(self, word: str, intermediate: Optional[str] = None): pipeline.connect("hello_again.intermediate", "hello.intermediate") assert {("hello", hello), ("hello_again", hello_again)} == set(pipeline.walk()) - def test__init_graph(self): - pipe = PipelineBase() - pipe.add_component("greet", Greet()) - pipe.add_component("adder", AddFixedValue()) - pipe.connect("greet", "adder") - pipe._init_graph() - for node in pipe.graph.nodes: - assert pipe.graph.nodes[node]["visits"] == 0 def test__prepare_component_input_data(self): MockComponent = component_class("MockComponent", input_types={"x": List[str], "y": str}) From 00ed49ded421d6df10934edf811071e8f7185bde Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 14:22:22 +0100 Subject: [PATCH 53/88] refactor: remove unused method --- haystack/core/pipeline/pipeline.py | 23 ----------------------- 1 file changed, 23 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 8bf59771ec..c067a58515 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -92,29 +92,6 @@ def _run_component( return cast(Dict[Any, Any], component_output), inputs - @staticmethod - def _merge_component_and_pipeline_outputs( - component_name: str, component_outputs: Dict, pipeline_outputs: Dict - ) -> Dict: - """ - Merges the outputs of a component with the current pipeline outputs. - - :param component_name: The name of the component. - :param component_outputs: The outputs of the component. - :param pipeline_outputs: The pipeline outputs. - :returns: New pipeline outputs. - """ - if not component_outputs: - return pipeline_outputs - elif component_name not in pipeline_outputs: - pipeline_outputs[component_name] = component_outputs - else: - for key, value in component_outputs.items(): - if key not in pipeline_outputs[component_name]: - pipeline_outputs[component_name][key] = value - - return pipeline_outputs - def run( # noqa: PLR0915, PLR0912 self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None ) -> Dict[str, Any]: From f1d325a6ec1e3a288ddcb791eaa7dfc6f54a5067 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 14:23:48 +0100 Subject: [PATCH 54/88] refactor: outdated comment --- haystack/core/pipeline/pipeline.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index c067a58515..c20f828a87 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -254,8 +254,7 @@ def run( # noqa: PLR0915, PLR0912 receivers=cached_receivers[component_name], include_outputs_from=include_outputs_from, ) - # TODO check original logic in pipeline, it looks like we don't want to override existing outputs - # e.g. for cycles but the tests check if intermediate outputs from components in cycles are overwritten + if component_pipeline_outputs: pipeline_outputs[component_name] = component_pipeline_outputs if self._is_queue_stale(priority_queue): From 711c84636ed0d594da0faa5f0721591010613732 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 14:38:56 +0100 Subject: [PATCH 55/88] refactor: inputs state is updated as side effect - to prepare for AsyncPipeline implementation --- haystack/core/pipeline/base.py | 12 +++++----- haystack/core/pipeline/pipeline.py | 15 +++++++----- test/core/pipeline/test_pipeline_base.py | 29 ++++++++++++------------ 3 files changed, 30 insertions(+), 26 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 3f506b8cd0..e5485a2b06 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -852,14 +852,14 @@ def _convert_to_internal_format(pipeline_inputs: Dict[str, Any]) -> Dict[str, Di return inputs @staticmethod - def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict) -> Tuple[Dict, Dict]: + def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict) -> Dict[str, Any]: """ Extracts the inputs needed to run for the component and removes them from the global inputs state. :param component_name: The name of a component. :param component: Component with component metadata. :param inputs: Global inputs state. - :returns: The inputs for the component and the new state of global inputs. + :returns: The inputs for the component. """ component_inputs = inputs.get(component_name, {}) consumed_inputs = {} @@ -892,7 +892,7 @@ def _consume_component_inputs(component_name: str, component: Dict, inputs: Dict inputs[component_name] = pruned_inputs - return consumed_inputs, inputs + return consumed_inputs def _fill_queue( self, component_names: List[str], inputs: Dict[str, Any], component_visits: Dict[str, int] @@ -998,7 +998,7 @@ def _add_missing_input_defaults(component_inputs: Dict[str, Any], component_inpu @staticmethod def _write_component_outputs( component_name, component_outputs, inputs, receivers, include_outputs_from - ) -> Tuple[Dict, Dict]: + ) -> Dict[str, Any]: """ Distributes the outputs of a component to the input sockets that it is connected to. @@ -1032,14 +1032,14 @@ def _write_component_outputs( # If we want to include all outputs from this actor in the final outputs, we don't need to prune any consumed # outputs if component_name in include_outputs_from: - return component_outputs, inputs + return component_outputs # We prune outputs that were consumed by any receiving sockets. # All remaining outputs will be added to the final outputs of the pipeline. consumed_outputs = {sender_socket.name for _, sender_socket, __ in receivers} pruned_outputs = {key: value for key, value in component_outputs.items() if key not in consumed_outputs} - return pruned_outputs, inputs + return pruned_outputs @staticmethod def _is_queue_stale(priority_queue: FIFOPriorityQueue) -> bool: diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index c20f828a87..6b2a9a19fe 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -29,7 +29,7 @@ def _run_component( inputs: Dict[str, Any], component_visits: Dict[str, int], parent_span: Optional[tracing.Span] = None, - ) -> Tuple[Dict, Dict]: + ) -> Dict[str, Any]: """ Runs a Component with the given inputs. @@ -39,11 +39,11 @@ def _run_component( :param parent_span: The parent span to use for the newly created span. This is to allow tracing to be correctly linked to the pipeline run. :raises PipelineRuntimeError: If Component doesn't return a dictionary. - :return: The output of the Component and the new state of inputs. + :return: The output of the Component. """ instance: Component = component["instance"] component_name = self.get_component_name(instance) - component_inputs, inputs = self._consume_component_inputs( + component_inputs = self._consume_component_inputs( component_name=component_name, component=component, inputs=inputs ) @@ -90,7 +90,7 @@ def _run_component( span.set_tag("haystack.component.visits", component_visits[component_name]) span.set_content_tag("haystack.component.output", component_output) - return cast(Dict[Any, Any], component_output), inputs + return cast(Dict[Any, Any], component_output) def run( # noqa: PLR0915, PLR0912 self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None @@ -246,8 +246,11 @@ def run( # noqa: PLR0915, PLR0912 ) warnings.warn(msg) - component_outputs, inputs = self._run_component(component, inputs, component_visits, parent_span=span) - component_pipeline_outputs, inputs = self._write_component_outputs( + component_outputs = self._run_component(component, inputs, component_visits, parent_span=span) + + # Updates global input state with component outputs and returns outputs that should go to + # pipeline outputs. + component_pipeline_outputs = self._write_component_outputs( component_name=component_name, component_outputs=component_outputs, inputs=inputs, diff --git a/test/core/pipeline/test_pipeline_base.py b/test/core/pipeline/test_pipeline_base.py index 69fc5c7502..3227fac552 100644 --- a/test/core/pipeline/test_pipeline_base.py +++ b/test/core/pipeline/test_pipeline_base.py @@ -1360,7 +1360,7 @@ def test__write_component_outputs_different_sockets( component_outputs = {"output1": 42} - pruned_outputs, updated_inputs = PipelineBase._write_component_outputs( + PipelineBase._write_component_outputs( component_name="sender1", component_outputs=component_outputs, inputs=inputs, @@ -1368,8 +1368,8 @@ def test__write_component_outputs_different_sockets( include_outputs_from=[], ) - assert len(updated_inputs["receiver1"][socket_name]) == expected_count - assert {"sender": "sender1", "value": 42} in updated_inputs["receiver1"][socket_name] + assert len(inputs["receiver1"][socket_name]) == expected_count + assert {"sender": "sender1", "value": 42} in inputs["receiver1"][socket_name] @pytest.mark.parametrize( "component_outputs,include_outputs,expected_pruned", @@ -1386,7 +1386,7 @@ def test__write_component_outputs_output_pruning( """Test output pruning behavior under different scenarios""" receivers = [("receiver1", regular_output_socket, regular_input_socket)] - pruned_outputs, _ = PipelineBase._write_component_outputs( + pruned_outputs = PipelineBase._write_component_outputs( component_name="sender1", component_outputs=component_outputs, inputs={}, @@ -1407,16 +1407,16 @@ def test__write_component_outputs_different_output_values( """Test handling of different output values""" receivers = [("receiver1", regular_output_socket, regular_input_socket)] component_outputs = {"output1": output_value} - - _, updated_inputs = PipelineBase._write_component_outputs( + inputs = {} + PipelineBase._write_component_outputs( component_name="sender1", component_outputs=component_outputs, - inputs={}, + inputs=inputs, receivers=receivers, include_outputs_from=[], ) - assert updated_inputs["receiver1"]["input1"] == [{"sender": "sender1", "value": output_value}] + assert inputs["receiver1"]["input1"] == [{"sender": "sender1", "value": output_value}] @pytest.mark.parametrize("receivers_count", [1, 2, 3], ids=["single-receiver", "two-receivers", "three-receivers"]) def test__write_component_outputs_multiple_receivers( @@ -1426,18 +1426,19 @@ def test__write_component_outputs_multiple_receivers( receivers = [(f"receiver{i}", regular_output_socket, regular_input_socket) for i in range(receivers_count)] component_outputs = {"output1": 42} - _, updated_inputs = PipelineBase._write_component_outputs( + inputs = {} + PipelineBase._write_component_outputs( component_name="sender1", component_outputs=component_outputs, - inputs={}, + inputs=inputs, receivers=receivers, include_outputs_from=[], ) for i in range(receivers_count): receiver_name = f"receiver{i}" - assert receiver_name in updated_inputs - assert updated_inputs[receiver_name]["input1"] == [{"sender": "sender1", "value": 42}] + assert receiver_name in inputs + assert inputs[receiver_name]["input1"] == [{"sender": "sender1", "value": 42}] def test__get_next_runnable_component_empty(self): """Test with empty queue returns None""" @@ -1622,8 +1623,8 @@ def test__consume_component_inputs(self, input_sockets, component_inputs, expect inputs = {"test_component": component_inputs} # Run - consumed, updated_inputs = PipelineBase._consume_component_inputs("test_component", component, inputs) + consumed = PipelineBase._consume_component_inputs("test_component", component, inputs) # Verify assert consumed == expected_consumed - assert updated_inputs["test_component"] == expected_remaining + assert inputs["test_component"] == expected_remaining From fa2b65c7fbed3b0e2f5ced2ef28ad69c9f8c7d0b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 14:39:08 +0100 Subject: [PATCH 56/88] format --- test/core/pipeline/test_pipeline_base.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/core/pipeline/test_pipeline_base.py b/test/core/pipeline/test_pipeline_base.py index 3227fac552..e92fbc089c 100644 --- a/test/core/pipeline/test_pipeline_base.py +++ b/test/core/pipeline/test_pipeline_base.py @@ -811,7 +811,6 @@ def run(self, word: str, intermediate: Optional[str] = None): pipeline.connect("hello_again.intermediate", "hello.intermediate") assert {("hello", hello), ("hello_again", hello_again)} == set(pipeline.walk()) - def test__prepare_component_input_data(self): MockComponent = component_class("MockComponent", input_types={"x": List[str], "y": str}) pipe = PipelineBase() From 8b7d761c8a818715281c20e550c4160144745ed6 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:28:06 +0100 Subject: [PATCH 57/88] test: add file conversion test --- .../pipeline/features/pipeline_run.feature | 1 + test/core/pipeline/features/test_run.py | 117 +++++++++++++++++- 2 files changed, 114 insertions(+), 4 deletions(-) diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index 22375c7691..8724dd3e1e 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -51,6 +51,7 @@ Feature: Pipeline running | that passes outputs that are consumed in cycle to outside the cycle | | with a component that has dynamic default inputs | | with a component that has variadic dynamic default inputs | + | that is a file conversion pipeline with two joiners | Scenario Outline: Running a bad Pipeline Given a pipeline diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index c307fe0105..5fdd2f3fd3 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -7,11 +7,11 @@ from haystack import Pipeline, Document, component from haystack.document_stores.types import DuplicatePolicy -from haystack.dataclasses import ChatMessage, GeneratedAnswer, TextContent -from haystack.components.routers import ConditionalRouter +from haystack.dataclasses import ChatMessage, GeneratedAnswer, TextContent, ByteStream +from haystack.components.routers import ConditionalRouter, FileTypeRouter from haystack.components.builders import PromptBuilder, AnswerBuilder, ChatPromptBuilder -from haystack.components.converters.output_adapter import OutputAdapter -from haystack.components.preprocessors import DocumentCleaner +from haystack.components.converters import OutputAdapter, JSONConverter, TextFileToDocument, CSVToDocument +from haystack.components.preprocessors import DocumentCleaner, DocumentSplitter from haystack.components.retrievers.in_memory import InMemoryBM25Retriever from haystack.document_stores.in_memory import InMemoryDocumentStore from haystack.components.joiners import BranchJoiner, DocumentJoiner, AnswerJoiner, StringJoiner @@ -4960,3 +4960,112 @@ def run(self, **kwargs): ), ], ) + +@given("a pipeline that is a file conversion pipeline with two joiners", target_fixture="pipeline_data") +def pipeline_that_converts_files(): + csv_data = """ +some,header,row +0,1,0 + """ + + txt_data = "Text file content for testing this." + + json_data = '{"content": "Some test content"}' + + sources = [ + ByteStream.from_string(text=csv_data, mime_type="text/csv", meta={"file_type": "csv"}), + ByteStream.from_string(text=txt_data, mime_type="text/plain", meta={"file_type": "txt"}), + ByteStream.from_string(text=json_data, mime_type="application/json", meta={"file_type": "json"}), + ] + + router = FileTypeRouter(mime_types=["text/csv", "text/plain", "application/json"]) + splitter = DocumentSplitter(split_by="word", split_length=3, split_overlap=0) + txt_converter = TextFileToDocument() + csv_converter = CSVToDocument() + json_converter = JSONConverter(content_key="content") + + b_joiner = DocumentJoiner() + a_joiner = DocumentJoiner() + + pp = Pipeline(max_runs_per_component=1) + + pp.add_component("router", router) + pp.add_component("splitter", splitter) + pp.add_component("txt_converter", txt_converter) + pp.add_component("csv_converter", csv_converter) + pp.add_component("json_converter", json_converter) + pp.add_component("b_joiner", b_joiner) + pp.add_component("a_joiner", a_joiner) + + pp.connect("router.text/plain", "txt_converter.sources") + pp.connect("router.application/json", "json_converter.sources") + pp.connect("router.text/csv", "csv_converter.sources") + pp.connect("txt_converter.documents", "b_joiner.documents") + pp.connect("json_converter.documents", "b_joiner.documents") + pp.connect("csv_converter.documents", "a_joiner.documents") + pp.connect("b_joiner.documents", "splitter.documents") + pp.connect("splitter.documents", "a_joiner.documents") + + expected_pre_split_docs = [ + Document(content="Some test content", meta={"file_type": "json"}), + Document(content=txt_data, meta={"file_type": "txt"}), + ] + expected_splits_docs = [ + Document( + content="Some test content", + meta={ + "file_type": "json", + "source_id": "0c6c5951d18da2935c7af3e24d417a9f94ca85403866dcfee1de93922504e1e5", + "page_number": 1, + "split_id": 0, + "split_idx_start": 0, + }, + ), + Document( + content="Text file content ", + meta={ + "file_type": "txt", + "source_id": "41cb91740f6e64ab542122936ea746c238ae0a92fd29b698efabbe23d0ba4c42", + "page_number": 1, + "split_id": 0, + "split_idx_start": 0, + }, + ), + Document( + content="for testing this.", + meta={ + "file_type": "txt", + "source_id": "41cb91740f6e64ab542122936ea746c238ae0a92fd29b698efabbe23d0ba4c42", + "page_number": 1, + "split_id": 1, + "split_idx_start": 18, + }, + ), + ] + + expected_csv_docs = [ + Document(content=csv_data, meta={"file_type": "csv"}) + ] + + return ( + pp, + [ + PipelineRunData( + inputs={"router": {"sources": sources}}, + expected_outputs={ + "a_joiner": { + "documents": expected_csv_docs + expected_splits_docs, + } + }, + expected_component_calls={ + ("router", 1): {"sources": sources, "meta": None}, + ("csv_converter", 1): {"sources": [sources[0]], "meta": None}, + ("txt_converter", 1): {"sources": [sources[1]], "meta": None}, + ("json_converter", 1): {"sources": [sources[2]], "meta": None}, + ("b_joiner", 1): {"documents": [[expected_pre_split_docs[0]], [expected_pre_split_docs[1]]], "top_k": None}, + ("splitter", 1): {"documents": expected_pre_split_docs}, + ("a_joiner", 1): {"documents": [expected_csv_docs, expected_splits_docs], "top_k": None}, + }, + ) + ], + ) From dc6acb982b2e308247a2a7cc7d6ca60970f0a83b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:28:36 +0100 Subject: [PATCH 58/88] format --- test/core/pipeline/features/test_run.py | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 5fdd2f3fd3..30c415f68d 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -4961,6 +4961,7 @@ def run(self, **kwargs): ], ) + @given("a pipeline that is a file conversion pipeline with two joiners", target_fixture="pipeline_data") def pipeline_that_converts_files(): csv_data = """ @@ -5043,26 +5044,23 @@ def pipeline_that_converts_files(): ), ] - expected_csv_docs = [ - Document(content=csv_data, meta={"file_type": "csv"}) - ] + expected_csv_docs = [Document(content=csv_data, meta={"file_type": "csv"})] return ( pp, [ PipelineRunData( inputs={"router": {"sources": sources}}, - expected_outputs={ - "a_joiner": { - "documents": expected_csv_docs + expected_splits_docs, - } - }, + expected_outputs={"a_joiner": {"documents": expected_csv_docs + expected_splits_docs}}, expected_component_calls={ ("router", 1): {"sources": sources, "meta": None}, ("csv_converter", 1): {"sources": [sources[0]], "meta": None}, ("txt_converter", 1): {"sources": [sources[1]], "meta": None}, ("json_converter", 1): {"sources": [sources[2]], "meta": None}, - ("b_joiner", 1): {"documents": [[expected_pre_split_docs[0]], [expected_pre_split_docs[1]]], "top_k": None}, + ("b_joiner", 1): { + "documents": [[expected_pre_split_docs[0]], [expected_pre_split_docs[1]]], + "top_k": None, + }, ("splitter", 1): {"documents": expected_pre_split_docs}, ("a_joiner", 1): {"documents": [expected_csv_docs, expected_splits_docs], "top_k": None}, }, From ba81a247ba11df462d6afc19c67f534701ac9bd4 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:33:44 +0100 Subject: [PATCH 59/88] fix: original implementation deepcopies outputs --- haystack/core/pipeline/pipeline.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 6b2a9a19fe..8beeed58b2 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -259,7 +259,7 @@ def run( # noqa: PLR0915, PLR0912 ) if component_pipeline_outputs: - pipeline_outputs[component_name] = component_pipeline_outputs + pipeline_outputs[component_name] = deepcopy(component_pipeline_outputs) if self._is_queue_stale(priority_queue): priority_queue = self._fill_queue(ordered_component_names, inputs, component_visits) From c9058819970ac6c5646c84180fc5765fde04868f Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:39:03 +0100 Subject: [PATCH 60/88] lint --- haystack/core/pipeline/base.py | 21 ++++++++++----------- haystack/core/pipeline/pipeline.py | 5 ++--- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index e5485a2b06..2fd638558f 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -10,6 +10,8 @@ from pathlib import Path from typing import Any, Dict, Iterator, List, Optional, TextIO, Tuple, Type, TypeVar, Union +from networkx import MultiDiGraph # type:ignore + from haystack import logging from haystack.core.component import Component, InputSocket, OutputSocket, component from haystack.core.errors import ( @@ -22,16 +24,6 @@ PipelineUnmarshalError, PipelineValidationError, ) -from haystack.core.pipeline.descriptions import find_pipeline_inputs, find_pipeline_outputs -from haystack.core.pipeline.draw import _to_mermaid_image -from haystack.core.pipeline.template import PipelineTemplate, PredefinedPipeline -from haystack.core.pipeline.utils import parse_connect_string -from haystack.core.serialization import DeserializationCallbacks, component_from_dict, component_to_dict -from haystack.core.type_utils import _type_name, _types_are_compatible -from haystack.marshal import Marshaller, YamlMarshaller -from haystack.utils import is_in_jupyter, type_serialization -from networkx import MultiDiGraph # type:ignore - from haystack.core.pipeline.component_checks import ( _NO_OUTPUT_PRODUCED, all_predecessors_executed, @@ -41,7 +33,14 @@ is_any_greedy_socket_ready, is_socket_lazy_variadic, ) -from haystack.core.pipeline.utils import FIFOPriorityQueue +from haystack.core.pipeline.descriptions import find_pipeline_inputs, find_pipeline_outputs +from haystack.core.pipeline.draw import _to_mermaid_image +from haystack.core.pipeline.template import PipelineTemplate, PredefinedPipeline +from haystack.core.pipeline.utils import FIFOPriorityQueue, parse_connect_string +from haystack.core.serialization import DeserializationCallbacks, component_from_dict, component_to_dict +from haystack.core.type_utils import _type_name, _types_are_compatible +from haystack.marshal import Marshaller, YamlMarshaller +from haystack.utils import is_in_jupyter, type_serialization DEFAULT_MARSHALLER = YamlMarshaller() diff --git a/haystack/core/pipeline/pipeline.py b/haystack/core/pipeline/pipeline.py index 8beeed58b2..e85daaf36c 100644 --- a/haystack/core/pipeline/pipeline.py +++ b/haystack/core/pipeline/pipeline.py @@ -4,14 +4,13 @@ import warnings from copy import deepcopy -from typing import Any, Dict, Mapping, Optional, Set, Tuple, cast +from typing import Any, Dict, Mapping, Optional, Set, cast from haystack import logging, tracing from haystack.core.component import Component from haystack.core.errors import PipelineRuntimeError -from haystack.telemetry import pipeline_running - from haystack.core.pipeline.base import ComponentPriority, PipelineBase +from haystack.telemetry import pipeline_running logger = logging.getLogger(__name__) From e045e90ac42316d6a7760fa21bfe54ff1cf655d5 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:52:29 +0100 Subject: [PATCH 61/88] fix: from_dict was updated --- haystack/core/pipeline/base.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 2fd638558f..df2e8fd60f 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -144,8 +144,8 @@ def to_dict(self) -> Dict[str, Any]: } @classmethod - def from_dict( # noqa: PLR0912 - cls: Type[T], data: Dict[str, Any], callbacks: Optional[DeserializationCallbacks] = None, **kwargs + def from_dict( + cls: Type[T], data: Dict[str, Any], callbacks: Optional[DeserializationCallbacks] = None, **kwargs ) -> T: """ Deserializes the pipeline from a dictionary. @@ -185,8 +185,10 @@ def from_dict( # noqa: PLR0912 f"Successfully imported module {module} but can't find it in the component registry." "This is unexpected and most likely a bug." ) - except (ImportError, PipelineError) as e: - raise PipelineError(f"Component '{component_data['type']}' not imported.") from e + except (ImportError, PipelineError, ValueError) as e: + raise PipelineError( + f"Component '{component_data['type']}' (name: '{name}') not imported." + ) from e # Create a new one component_class = component.registry[component_data["type"]] From ed7c7002de01a1026dba9f05cfdcc3545134fb57 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:53:57 +0100 Subject: [PATCH 62/88] fix: format --- haystack/core/pipeline/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index df2e8fd60f..c4686f8aac 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -145,7 +145,7 @@ def to_dict(self) -> Dict[str, Any]: @classmethod def from_dict( - cls: Type[T], data: Dict[str, Any], callbacks: Optional[DeserializationCallbacks] = None, **kwargs + cls: Type[T], data: Dict[str, Any], callbacks: Optional[DeserializationCallbacks] = None, **kwargs ) -> T: """ Deserializes the pipeline from a dictionary. From 558050fb9a49ba0bb826ea050e483c7d47ec8184 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 16:58:42 +0100 Subject: [PATCH 63/88] fix: test --- test/core/pipeline/test_pipeline.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 980b0e4b37..0cda1ca185 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -29,7 +29,7 @@ def test__run_component_success(self): pp.connect("joiner_1", "joiner_2") inputs = {"joiner_1": {"value": [{"sender": None, "value": "test_value"}]}} - outputs, updated_inputs = pp._run_component( + outputs = pp._run_component( component=pp._get_component_with_graph_metadata_and_visits("joiner_1", 0), inputs=inputs, component_visits={"joiner_1": 0, "joiner_2": 0}, @@ -37,7 +37,7 @@ def test__run_component_success(self): assert outputs == {"value": "test_value"} # We remove input in greedy variadic sockets, even if they are from the user - assert "value" not in updated_inputs["joiner_1"] + assert "value" not in inputs["joiner_1"] def test__run_component_fail(self): """Test error when component doesn't return a dictionary""" From 09965ba594dafc9ee7d7e7fd8db640f5215fde36 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:03:39 +0100 Subject: [PATCH 64/88] test: add test for thread safety --- test/conftest.py | 22 ++++++++++++++++++-- test/core/pipeline/test_pipeline.py | 32 +++++++++++++++++++++++++++++ 2 files changed, 52 insertions(+), 2 deletions(-) diff --git a/test/conftest.py b/test/conftest.py index 92d16eb310..ed8d80ab52 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -4,14 +4,17 @@ from datetime import datetime from pathlib import Path from test.tracing.utils import SpyingTracer -from typing import Generator +from typing import Generator, Dict from unittest.mock import Mock, patch import pytest +import time +import asyncio + from openai.types.chat import ChatCompletion, ChatCompletionMessage from openai.types.chat.chat_completion import Choice -from haystack import tracing +from haystack import tracing, component from haystack.testing.test_utils import set_all_seeds set_all_seeds(0) @@ -19,6 +22,21 @@ # Tracing is disable by default to avoid failures in CI tracing.disable_tracing() +@pytest.fixture() +def waiting_component(): + @component + class Waiter: + @component.output_types(waited_for=int) + def run(self, wait_for: int) -> Dict[str, int]: + time.sleep(wait_for) + return {'waited_for': wait_for} + + @component.output_types(waited_for=int) + async def run_async(self, wait_for: int) -> Dict[str, int]: + await asyncio.sleep(wait_for) + return {'waited_for': wait_for} + + return Waiter @pytest.fixture() def mock_tokenizer(): diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index 0cda1ca185..dbe8d52a71 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -3,6 +3,7 @@ # SPDX-License-Identifier: Apache-2.0 from typing import Optional +from concurrent.futures import ThreadPoolExecutor import pytest @@ -19,6 +20,37 @@ class TestPipeline: It doesn't test Pipeline.run(), that is done separately in a different way. """ + def test_pipeline_thread_safety(self, waiting_component, spying_tracer): + # Initialize pipeline with synchronous components + pp = Pipeline() + pp.add_component("wait", waiting_component()) + + run_data = [ + {"wait_for": 1}, + {"wait_for": 2}, + ] + + # Use ThreadPoolExecutor to run pipeline calls in parallel + with ThreadPoolExecutor(max_workers=len(run_data)) as executor: + # Submit pipeline runs to the executor + futures = [ + executor.submit(pp.run, data) + for data in run_data + ] + + # Wait for all futures to complete + for future in futures: + future.result() + + # Verify component visits using tracer + component_spans = [ + sp for sp in spying_tracer.spans + if sp.operation_name == "haystack.component.run" + ] + + for span in component_spans: + assert span.tags["haystack.component.visits"] == 1 + def test__run_component_success(self): """Test successful component execution""" joiner_1 = BranchJoiner(type_=str) From a3eb8da741168a334c3d4f0c9897bdcf11825730 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:04:01 +0100 Subject: [PATCH 65/88] remove unused imports --- test/conftest.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/test/conftest.py b/test/conftest.py index ed8d80ab52..d954cddbea 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -11,8 +11,6 @@ import time import asyncio -from openai.types.chat import ChatCompletion, ChatCompletionMessage -from openai.types.chat.chat_completion import Choice from haystack import tracing, component from haystack.testing.test_utils import set_all_seeds From 4d9845acfed1b30d773b923488400ad8bdb690b7 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:04:17 +0100 Subject: [PATCH 66/88] format --- test/conftest.py | 6 ++++-- test/core/pipeline/test_pipeline.py | 15 +++------------ 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/test/conftest.py b/test/conftest.py index d954cddbea..a496043309 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -20,6 +20,7 @@ # Tracing is disable by default to avoid failures in CI tracing.disable_tracing() + @pytest.fixture() def waiting_component(): @component @@ -27,15 +28,16 @@ class Waiter: @component.output_types(waited_for=int) def run(self, wait_for: int) -> Dict[str, int]: time.sleep(wait_for) - return {'waited_for': wait_for} + return {"waited_for": wait_for} @component.output_types(waited_for=int) async def run_async(self, wait_for: int) -> Dict[str, int]: await asyncio.sleep(wait_for) - return {'waited_for': wait_for} + return {"waited_for": wait_for} return Waiter + @pytest.fixture() def mock_tokenizer(): """ diff --git a/test/core/pipeline/test_pipeline.py b/test/core/pipeline/test_pipeline.py index dbe8d52a71..7aa36ca17f 100644 --- a/test/core/pipeline/test_pipeline.py +++ b/test/core/pipeline/test_pipeline.py @@ -25,28 +25,19 @@ def test_pipeline_thread_safety(self, waiting_component, spying_tracer): pp = Pipeline() pp.add_component("wait", waiting_component()) - run_data = [ - {"wait_for": 1}, - {"wait_for": 2}, - ] + run_data = [{"wait_for": 1}, {"wait_for": 2}] # Use ThreadPoolExecutor to run pipeline calls in parallel with ThreadPoolExecutor(max_workers=len(run_data)) as executor: # Submit pipeline runs to the executor - futures = [ - executor.submit(pp.run, data) - for data in run_data - ] + futures = [executor.submit(pp.run, data) for data in run_data] # Wait for all futures to complete for future in futures: future.result() # Verify component visits using tracer - component_spans = [ - sp for sp in spying_tracer.spans - if sp.operation_name == "haystack.component.run" - ] + component_spans = [sp for sp in spying_tracer.spans if sp.operation_name == "haystack.component.run"] for span in component_spans: assert span.tags["haystack.component.visits"] == 1 From 2d11923626d34cb9f58251863409c8aac78c67be Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:11:00 +0100 Subject: [PATCH 67/88] test: FIFOPriorityQueue --- test/core/pipeline/test_utils.py | 170 ++++++++++++++++++++++++++++++- 1 file changed, 169 insertions(+), 1 deletion(-) diff --git a/test/core/pipeline/test_utils.py b/test/core/pipeline/test_utils.py index fcb5734f50..e2cc26cac3 100644 --- a/test/core/pipeline/test_utils.py +++ b/test/core/pipeline/test_utils.py @@ -1,9 +1,177 @@ # SPDX-FileCopyrightText: 2022-present deepset GmbH # # SPDX-License-Identifier: Apache-2.0 -from haystack.core.pipeline.utils import parse_connect_string + +import pytest + +from haystack.core.pipeline.utils import parse_connect_string, FIFOPriorityQueue def test_parse_connection(): assert parse_connect_string("foobar") == ("foobar", None) assert parse_connect_string("foo.bar") == ("foo", "bar") + + +@pytest.fixture +def empty_queue(): + """Fixture providing a fresh empty queue for each test.""" + return FIFOPriorityQueue() + + +def test_empty_queue_initialization(empty_queue): + """Test that a new queue is empty.""" + assert len(empty_queue) == 0 + assert not bool(empty_queue) + + +def test_push_single_item(empty_queue): + """Test pushing a single item.""" + empty_queue.push("item1", 1) + assert len(empty_queue) == 1 + assert bool(empty_queue) + assert empty_queue.peek() == (1, "item1") + + +def test_push_multiple_items_different_priorities(empty_queue): + """Test pushing multiple items with different priorities.""" + items = [("item3", 3), ("item1", 1), ("item2", 2)] + for item, priority in items: + empty_queue.push(item, priority) + + # Items should come out in priority order + assert empty_queue.pop() == (1, "item1") + assert empty_queue.pop() == (2, "item2") + assert empty_queue.pop() == (3, "item3") + + +def test_push_multiple_items_same_priority(empty_queue): + """Test FIFO behavior for items with equal priority.""" + items = [("first", 1), ("second", 1), ("third", 1)] + for item, priority in items: + empty_queue.push(item, priority) + + # Items should come out in insertion order + assert empty_queue.pop() == (1, "first") + assert empty_queue.pop() == (1, "second") + assert empty_queue.pop() == (1, "third") + + +def test_mixed_priority_and_fifo(empty_queue): + """Test mixed priority levels with some equal priorities.""" + empty_queue.push("medium1", 2) + empty_queue.push("high", 1) + empty_queue.push("medium2", 2) + empty_queue.push("low", 3) + + # Check extraction order + assert empty_queue.pop() == (1, "high") + assert empty_queue.pop() == (2, "medium1") + assert empty_queue.pop() == (2, "medium2") + assert empty_queue.pop() == (3, "low") + + +def test_peek_behavior(empty_queue): + """Test that peek returns items without removing them.""" + empty_queue.push("item1", 1) + empty_queue.push("item2", 2) + + # Peek multiple times + for _ in range(3): + assert empty_queue.peek() == (1, "item1") + assert len(empty_queue) == 2 + + +def test_get_behavior(empty_queue): + """Test the get method with both empty and non-empty queues.""" + # Test on empty queue + assert empty_queue.get() is None + + # Test with items + empty_queue.push("item1", 1) + assert empty_queue.get() == (1, "item1") + assert empty_queue.get() is None # Queue should be empty again + + +def test_pop_empty_queue(empty_queue): + """Test that pop raises IndexError on empty queue.""" + with pytest.raises(IndexError, match="pop from empty queue"): + empty_queue.pop() + + +def test_peek_empty_queue(empty_queue): + """Test that peek raises IndexError on empty queue.""" + with pytest.raises(IndexError, match="peek at empty queue"): + empty_queue.peek() + + +def test_length_updates(empty_queue): + """Test that length updates correctly with pushes and pops.""" + initial_len = len(empty_queue) + assert initial_len == 0 + + # Test length increases + empty_queue.push("item1", 1) + assert len(empty_queue) == 1 + empty_queue.push("item2", 2) + assert len(empty_queue) == 2 + + # Test length decreases + empty_queue.pop() + assert len(empty_queue) == 1 + empty_queue.pop() + assert len(empty_queue) == 0 + + +def test_bool_conversion(empty_queue): + """Test boolean conversion in various states.""" + # Empty queue should be False + assert not bool(empty_queue) + + # Queue with items should be True + empty_queue.push("item", 1) + assert bool(empty_queue) + + # Queue should be False again after removing item + empty_queue.pop() + assert not bool(empty_queue) + + +def test_large_number_of_items(empty_queue): + """Test handling of a large number of items with mixed priorities.""" + # Add 1000 items with 10 different priority levels + for i in range(1000): + priority = i % 10 + empty_queue.push(f"item{i}", priority) + + # Verify FIFO order within same priority + last_priority = -1 + last_index = -1 + for _ in range(1000): + priority, item = empty_queue.pop() + current_index = int(item[4:]) # Extract index from "itemX" + + if priority == last_priority: + assert current_index > last_index, "FIFO order violated within same priority" + else: + assert priority > last_priority, "Priority order violated" + + last_priority = priority + last_index = current_index + + +@pytest.mark.parametrize( + "items", + [ + [(1, "A"), (1, "B"), (1, "C")], # Same priority + [(3, "A"), (2, "B"), (1, "C")], # Different priorities + [(2, "A"), (1, "B"), (2, "C")], # Mixed priorities + ], +) +def test_queue_ordering_parametrized(empty_queue, items): + """Parametrized test for different ordering scenarios.""" + for priority, item in items: + empty_queue.push(item, priority) + + sorted_items = sorted(items, key=lambda x: (x[0], items.index(x))) + for priority, item in sorted_items: + assert empty_queue.pop() == (priority, item) From d6673fe80c6a06da0b279376d4b9c12b852313de Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:18:58 +0100 Subject: [PATCH 68/88] chore: add release note --- releasenotes/notes/fix-pipeline-run-2fefeafc705a6d91.yaml | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 releasenotes/notes/fix-pipeline-run-2fefeafc705a6d91.yaml diff --git a/releasenotes/notes/fix-pipeline-run-2fefeafc705a6d91.yaml b/releasenotes/notes/fix-pipeline-run-2fefeafc705a6d91.yaml new file mode 100644 index 0000000000..faad6ad57b --- /dev/null +++ b/releasenotes/notes/fix-pipeline-run-2fefeafc705a6d91.yaml @@ -0,0 +1,8 @@ +--- +fixes: + - | + Fixes multiple bugs in the pipeline-run-method: + - acyclic pipelines with multiple lazy variadic components not running all components + - cyclic pipelines not passing intermediate outputs to components outside the cycle + - cyclic pipelines with two or more optional or greedy variadic edges showing unexpected execution behavior + - cyclic pipelines with two cycles sharing an edge raising errors From a48d8028371916f2300d1171fc4ba7a63186a54e Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:52:06 +0100 Subject: [PATCH 69/88] feat: add AsyncPipeline --- haystack/__init__.py | 3 +- haystack/core/pipeline/__init__.py | 3 +- haystack/core/pipeline/async_pipeline.py | 578 ++++++++++++++++++++++ test/core/pipeline/features/conftest.py | 64 ++- test/core/pipeline/features/test_run.py | 204 ++++---- test/core/pipeline/test_async_pipeline.py | 26 + 6 files changed, 778 insertions(+), 100 deletions(-) create mode 100644 haystack/core/pipeline/async_pipeline.py create mode 100644 test/core/pipeline/test_async_pipeline.py diff --git a/haystack/__init__.py b/haystack/__init__.py index 2f275d3a46..7d521b433e 100644 --- a/haystack/__init__.py +++ b/haystack/__init__.py @@ -6,7 +6,7 @@ import haystack.tracing from haystack.core.component import component from haystack.core.errors import ComponentError, DeserializationError -from haystack.core.pipeline import Pipeline, PredefinedPipeline +from haystack.core.pipeline import AsyncPipeline, Pipeline, PredefinedPipeline from haystack.core.serialization import default_from_dict, default_to_dict from haystack.dataclasses import Answer, Document, ExtractedAnswer, GeneratedAnswer @@ -23,6 +23,7 @@ "default_to_dict", "DeserializationError", "ComponentError", + "AsyncPipeline", "Pipeline", "PredefinedPipeline", "Document", diff --git a/haystack/core/pipeline/__init__.py b/haystack/core/pipeline/__init__.py index 744c282f8c..1d84ac1c74 100644 --- a/haystack/core/pipeline/__init__.py +++ b/haystack/core/pipeline/__init__.py @@ -2,7 +2,8 @@ # # SPDX-License-Identifier: Apache-2.0 +from .async_pipeline import AsyncPipeline from .pipeline import Pipeline from .template import PredefinedPipeline -__all__ = ["Pipeline", "PredefinedPipeline"] +__all__ = ["AsyncPipeline", "Pipeline", "PredefinedPipeline"] diff --git a/haystack/core/pipeline/async_pipeline.py b/haystack/core/pipeline/async_pipeline.py new file mode 100644 index 0000000000..9b31d7ce1f --- /dev/null +++ b/haystack/core/pipeline/async_pipeline.py @@ -0,0 +1,578 @@ +# SPDX-FileCopyrightText: 2022-present deepset GmbH +# +# SPDX-License-Identifier: Apache-2.0 + +import asyncio +from copy import deepcopy +from typing import Any, AsyncIterator, Dict, List, Optional, Set + +from haystack import logging, tracing +from haystack.core.component import Component +from haystack.core.errors import PipelineMaxComponentRuns, PipelineRuntimeError +from haystack.core.pipeline.base import ComponentPriority, PipelineBase +from haystack.telemetry import pipeline_running + +logger = logging.getLogger(__name__) + + +class AsyncPipeline(PipelineBase): + """ + Asynchronous version of the orchestration engine. + + Orchestrates component execution and runs components concurrently if the execution graph allows it. + """ + + async def run_async_generator( # noqa: PLR0915,C901 + self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None, concurrency_limit: int = 4 + ) -> AsyncIterator[Dict[str, Any]]: + """ + Execute this pipeline asynchronously, yielding partial outputs when any component finishes. + + :param data: Initial input data to the pipeline. + :param concurrency_limit: The maximum number of components that are allowed to run concurrently. + :param include_outputs_from: + Set of component names whose individual outputs are to be + included in the pipeline's output. For components that are + invoked multiple times (in a loop), only the last-produced + output is included. + :return: An async iterator of partial (and final) outputs. + """ + if include_outputs_from is None: + include_outputs_from = set() + + # 0) Basic pipeline init + pipeline_running(self) # telemetry + self.warm_up() # optional warm-up (if needed) + + # 1) Prepare ephemeral state + ready_sem = asyncio.Semaphore(max(1, concurrency_limit)) + inputs_state: Dict[str, Dict[str, List[Dict[str, Any]]]] = {} + pipeline_outputs: Dict[str, Any] = {} + running_tasks: Dict[asyncio.Task, str] = {} + + # A set of component names that have been scheduled but not finished: + scheduled_components: Set[str] = set() + + # 2) Convert input data + prepared_data = self._prepare_component_input_data(data) + self._validate_input(prepared_data) + inputs_state = self._convert_to_internal_format(prepared_data) + + # For quick lookup of downstream receivers + ordered_names = sorted(self.graph.nodes.keys()) + cached_receivers = {n: self._find_receivers_from(n) for n in ordered_names} + component_visits = {component_name: 0 for component_name in ordered_names} + + # We fill the queue once and raise if all components are BLOCKED + self.validate_pipeline(self._fill_queue(ordered_names, inputs_state, component_visits)) + + # Single parent span for entire pipeline execution + with tracing.tracer.trace( + "haystack.async_pipeline.run", + tags={ + "haystack.pipeline.input_data": data, + "haystack.pipeline.output_data": pipeline_outputs, + "haystack.pipeline.metadata": self.metadata, + "haystack.pipeline.max_runs_per_component": self._max_runs_per_component, + }, + ) as parent_span: + # ------------------------------------------------- + # We define some functions here so that they have access to local runtime state + # (inputs, tasks, scheduled components) via closures. + # ------------------------------------------------- + async def _run_component_async(component_name: str, component_inputs: Dict[str, Any]) -> Dict[str, Any]: + """ + Runs one component. + + If the component supports async, await directly it will run async; otherwise offload to executor. + Updates visits count, writes outputs to `inputs_state`, + and returns pruned outputs that get stored in `pipeline_outputs`. + + :param component_name: The name of the component. + :param component_inputs: Inputs for the component. + :returns: Outputs from the component that can be yielded from run_async_generator. + """ + if component_visits[component_name] > self._max_runs_per_component: + raise PipelineMaxComponentRuns(f"Max runs for '{component_name}' reached.") + + instance: Component = self.get_component(component_name) + with tracing.tracer.trace( + "haystack.component.run", + tags={ + "haystack.component.name": component_name, + "haystack.component.type": instance.__class__.__name__, + "haystack.component.input_types": {k: type(v).__name__ for k, v in component_inputs.items()}, + "haystack.component.input_spec": { + key: { + "type": (value.type.__name__ if isinstance(value.type, type) else str(value.type)), + "senders": value.senders, + } + for key, value in instance.__haystack_input__._sockets_dict.items() # type: ignore + }, + "haystack.component.output_spec": { + key: { + "type": (value.type.__name__ if isinstance(value.type, type) else str(value.type)), + "receivers": value.receivers, + } + for key, value in instance.__haystack_output__._sockets_dict.items() # type: ignore + }, + }, + parent_span=parent_span, + ) as span: + span.set_content_tag("haystack.component.input", deepcopy(component_inputs)) + logger.info("Running component {name}", name=component_name) + + if getattr(instance, "__haystack_supports_async__", False): + outputs = await instance.run_async(**component_inputs) # type: ignore + else: + loop = asyncio.get_running_loop() + outputs = await loop.run_in_executor(None, lambda: instance.run(**component_inputs)) + + component_visits[component_name] += 1 + + if not isinstance(outputs, dict): + raise PipelineRuntimeError( + f"Component '{component_name}' returned an invalid output type. " + f"Expected a dict, but got {type(outputs).__name__} instead. " + ) + + span.set_tag("haystack.component.visits", component_visits[component_name]) + span.set_content_tag("haystack.component.outputs", deepcopy(outputs)) + + # Distribute outputs to downstream inputs; also prune outputs based on `include_outputs_from` + pruned = self._write_component_outputs( + component_name=component_name, + component_outputs=outputs, + inputs=inputs_state, + receivers=cached_receivers[component_name], + include_outputs_from=include_outputs_from, + ) + if pruned: + pipeline_outputs[component_name] = pruned + + return pruned + + async def _run_highest_in_isolation(component_name: str) -> AsyncIterator[Dict[str, Any]]: + """ + Runs a component with HIGHEST priority in isolation. + + We need to run components with HIGHEST priority (i.e. components with GreedyVariadic input socket) + because otherwise, downstream components could produce additional inputs for the GreedyVariadic socket. + + :param component_name: The name of the component. + :return: An async iterator of partial outputs. + """ + # 1) Wait for all in-flight tasks to finish + while running_tasks: + done, _pending = await asyncio.wait(running_tasks.keys(), return_when=asyncio.ALL_COMPLETED) + for finished in done: + finished_component_name = running_tasks.pop(finished) + partial_result = finished.result() + scheduled_components.discard(finished_component_name) + if partial_result: + yield_dict = {finished_component_name: deepcopy(partial_result)} + yield yield_dict # partial outputs + + if component_name in scheduled_components: + # If it's already scheduled for some reason, skip + return + + # 2) Run the HIGHEST component by itself + scheduled_components.add(component_name) + comp_dict = self._get_component_with_graph_metadata_and_visits( + component_name, component_visits[component_name] + ) + component_inputs = self._consume_component_inputs(component_name, comp_dict, inputs_state) + component_inputs = self._add_missing_input_defaults(component_inputs, comp_dict["input_sockets"]) + result = await _run_component_async(component_name, component_inputs) + scheduled_components.remove(component_name) + if result: + yield {component_name: deepcopy(result)} + + async def _schedule_ready_task(component_name: str) -> None: + """ + Schedule a component that is considered READY (or just turned READY). + + We do NOT wait for it to finish here. This allows us to run other components concurrently. + + :param component_name: The name of the component. + """ + + if component_name in scheduled_components: + return # already scheduled, do nothing + + scheduled_components.add(component_name) + + comp_dict = self._get_component_with_graph_metadata_and_visits( + component_name, component_visits[component_name] + ) + component_inputs = self._consume_component_inputs(component_name, comp_dict, inputs_state) + component_inputs = self._add_missing_input_defaults(component_inputs, comp_dict["input_sockets"]) + + async def _runner(): + async with ready_sem: + result = await _run_component_async(component_name, component_inputs) + + scheduled_components.remove(component_name) + return result + + task = asyncio.create_task(_runner()) + running_tasks[task] = component_name + + async def _wait_for_one_task_to_complete() -> AsyncIterator[Dict[str, Any]]: + """ + Wait for exactly one running task to finish, yield partial outputs. + + If no tasks are running, does nothing. + """ + if running_tasks: + done, _ = await asyncio.wait(running_tasks.keys(), return_when=asyncio.FIRST_COMPLETED) + for finished in done: + finished_component_name = running_tasks.pop(finished) + partial_result = finished.result() + scheduled_components.discard(finished_component_name) + if partial_result: + yield {finished_component_name: deepcopy(partial_result)} + + async def _wait_for_all_tasks_to_complete() -> AsyncIterator[Dict[str, Any]]: + """ + Wait for all running tasks to finish, yield partial outputs. + """ + if running_tasks: + done, _ = await asyncio.wait(running_tasks.keys(), return_when=asyncio.ALL_COMPLETED) + for finished in done: + finished_component_name = running_tasks.pop(finished) + partial_result = finished.result() + scheduled_components.discard(finished_component_name) + if partial_result: + yield {finished_component_name: deepcopy(partial_result)} + + async def _schedule_defer_incrementally(component_name: str) -> AsyncIterator[Dict[str, Any]]: + """ + Schedule a component that has priority DEFER or DEFER_LAST. + + Waits for tasks to complete one-by-one. Schedules the component as soon as it turns READY. + If the component does not turn READY, it drains the queue completely before scheduling the component. + + :param component_name: The name of the component. + :returns: An async iterator of partial outputs. + """ + comp_dict = self._get_component_with_graph_metadata_and_visits( + component_name, component_visits[component_name] + ) + while True: + # Already scheduled => stop + if component_name in scheduled_components: + return + # Priority is recalculated after each completed task + + new_prio = self._calculate_priority(comp_dict, inputs_state.get(component_name, {})) + if new_prio == ComponentPriority.READY: + # It's now ready => schedule it + await _schedule_ready_task(component_name) + return + + elif new_prio == ComponentPriority.HIGHEST: + # Edge case: somehow became HIGHEST => run in isolation + async for partial_out in _run_highest_in_isolation(component_name): + yield partial_out + return + + # else it remains DEFER or DEFER_LAST, keep waiting + if running_tasks: + # Wait for just one task to finish + async for part in _wait_for_one_task_to_complete(): + yield part + else: + # No tasks left => schedule anyway (end of pipeline) + # This ensures we don't deadlock forever. + await _schedule_ready_task(component_name) + return + + # ------------------------------------------------- + # MAIN SCHEDULING LOOP + # ------------------------------------------------- + while True: + # 2) Build the priority queue of candidates + priority_queue = self._fill_queue(ordered_names, inputs_state, component_visits) + candidate = self._get_next_runnable_component(priority_queue, component_visits) + if candidate is None and running_tasks: + # We need to wait for one task to finish to make progress and potentially unblock the priority_queue + async for partial_result in _wait_for_one_task_to_complete(): + yield partial_result + continue + + if candidate is None and not running_tasks: + # done + break + + priority, component_name, _ = candidate # type: ignore + + if component_name in scheduled_components: + # We need to wait for one task to finish to make progress + async for partial_result in _wait_for_one_task_to_complete(): + yield partial_result + continue + + if priority == ComponentPriority.HIGHEST: + # 1) run alone + async for partial_result in _run_highest_in_isolation(component_name): + yield partial_result + # then continue the loop + continue + + if priority == ComponentPriority.READY: + # 1) schedule this one + await _schedule_ready_task(component_name) + + # 2) Possibly schedule more READY tasks if concurrency not fully used + while len(priority_queue) > 0 and not ready_sem.locked(): + peek_prio, peek_name = priority_queue.peek() + if peek_prio in (ComponentPriority.BLOCKED, ComponentPriority.HIGHEST): + # can't run or must run alone => skip + break + if peek_prio == ComponentPriority.READY: + priority_queue.pop() + await _schedule_ready_task(peek_name) + # keep adding while concurrency is not locked + continue + + # The next is DEFER/DEFER_LAST => we only schedule it if it "becomes READY" + # We'll handle it in the next iteration or with incremental waiting + break + + # 3) Wait for at least 1 task to finish => yield partial + async for partial_result in _wait_for_one_task_to_complete(): + yield partial_result + + elif priority in (ComponentPriority.DEFER, ComponentPriority.DEFER_LAST): + # We do incremental waiting + async for partial_result in _schedule_defer_incrementally(component_name): + yield partial_result + + # End main loop + + # 3) Drain leftover tasks + async for partial_result in _wait_for_all_tasks_to_complete(): + yield partial_result + + # 4) Yield final pipeline outputs + yield deepcopy(pipeline_outputs) + + async def run_async( + self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None, concurrency_limit: int = 4 + ) -> Dict[str, Any]: + """ + Runs the Pipeline with given input data. + + Usage: + ```python + from haystack import Document + from haystack.utils import Secret + from haystack.document_stores.in_memory import InMemoryDocumentStore + from haystack.components.retrievers.in_memory import InMemoryBM25Retriever + from haystack.components.generators import OpenAIGenerator + from haystack.components.builders.answer_builder import AnswerBuilder + from haystack.components.builders.prompt_builder import PromptBuilder + + from haystack_experimental import AsyncPipeline + + import asyncio + + # Write documents to InMemoryDocumentStore + document_store = InMemoryDocumentStore() + document_store.write_documents([ + Document(content="My name is Jean and I live in Paris."), + Document(content="My name is Mark and I live in Berlin."), + Document(content="My name is Giorgio and I live in Rome.") + ]) + + prompt_template = \"\"\" + Given these documents, answer the question. + Documents: + {% for doc in documents %} + {{ doc.content }} + {% endfor %} + Question: {{question}} + Answer: + \"\"\" + + retriever = InMemoryBM25Retriever(document_store=document_store) + prompt_builder = PromptBuilder(template=prompt_template) + llm = OpenAIGenerator(api_key=Secret.from_token(api_key)) + + rag_pipeline = AsyncPipeline() + rag_pipeline.add_component("retriever", retriever) + rag_pipeline.add_component("prompt_builder", prompt_builder) + rag_pipeline.add_component("llm", llm) + rag_pipeline.connect("retriever", "prompt_builder.documents") + rag_pipeline.connect("prompt_builder", "llm") + + # Ask a question + question = "Who lives in Paris?" + + + async def run_inner(data, include_outputs_from): + return await rag_pipeline.run_async(data=data, include_outputs_from=include_outputs_from) + + data = { + "retriever": {"query": question}, + "prompt_builder": {"question": question}, + } + async_loop = asyncio.new_event_loop() + asyncio.set_event_loop(async_loop) + results = async_loop.run_until_complete(run_inner(data)) + async_loop.close() + + print(results["llm"]["replies"]) + # Jean lives in Paris + ``` + + :param data: + A dictionary of inputs for the pipeline's components. Each key is a component name + and its value is a dictionary of that component's input parameters: + ``` + data = { + "comp1": {"input1": 1, "input2": 2}, + } + ``` + For convenience, this format is also supported when input names are unique: + ``` + data = { + "input1": 1, "input2": 2, + } + ``` + :param include_outputs_from: + Set of component names whose individual outputs are to be + included in the pipeline's output. For components that are + invoked multiple times (in a loop), only the last-produced + output is included. + :param concurrency_limit: The maximum number of components that should be allowed to run concurrently. + :returns: + A dictionary where each entry corresponds to a component name + and its output. If `include_outputs_from` is `None`, this dictionary + will only contain the outputs of leaf components, i.e., components + without outgoing connections. + + :raises ValueError: + If invalid inputs are provided to the pipeline. + :raises PipelineRuntimeError: + If the Pipeline contains cycles with unsupported connections that would cause + it to get stuck and fail running. + Or if a Component fails or returns output in an unsupported type. + :raises PipelineMaxComponentRuns: + If a Component reaches the maximum number of times it can be run in this Pipeline. + """ + final: Dict[str, Any] = {} + async for partial in self.run_async_generator( + data=data, concurrency_limit=concurrency_limit, include_outputs_from=include_outputs_from + ): + final = partial + return final or {} + + def run( + self, data: Dict[str, Any], include_outputs_from: Optional[Set[str]] = None, concurrency_limit: int = 4 + ) -> Dict[str, Any]: + """ + Runs the pipeline with given input data. + + This method is synchronous, but it runs components asynchronously internally. + Check out `run_async` or `run_async_generator` if you are looking for async-methods. + + Usage: + ```python + from haystack import Document + from haystack.utils import Secret + from haystack.document_stores.in_memory import InMemoryDocumentStore + from haystack.components.retrievers.in_memory import InMemoryBM25Retriever + from haystack.components.generators import OpenAIGenerator + from haystack.components.builders.answer_builder import AnswerBuilder + from haystack.components.builders.prompt_builder import PromptBuilder + + from haystack_experimental import AsyncPipeline + + # Write documents to InMemoryDocumentStore + document_store = InMemoryDocumentStore() + document_store.write_documents([ + Document(content="My name is Jean and I live in Paris."), + Document(content="My name is Mark and I live in Berlin."), + Document(content="My name is Giorgio and I live in Rome.") + ]) + + prompt_template = \"\"\" + Given these documents, answer the question. + Documents: + {% for doc in documents %} + {{ doc.content }} + {% endfor %} + Question: {{question}} + Answer: + \"\"\" + + retriever = InMemoryBM25Retriever(document_store=document_store) + prompt_builder = PromptBuilder(template=prompt_template) + llm = OpenAIGenerator(api_key=Secret.from_token(api_key)) + + rag_pipeline = AsyncPipeline() + rag_pipeline.add_component("retriever", retriever) + rag_pipeline.add_component("prompt_builder", prompt_builder) + rag_pipeline.add_component("llm", llm) + rag_pipeline.connect("retriever", "prompt_builder.documents") + rag_pipeline.connect("prompt_builder", "llm") + + # Ask a question + question = "Who lives in Paris?" + + + async def run_inner(data, include_outputs_from): + return await rag_pipeline.run_async(data=data, include_outputs_from=include_outputs_from) + + data = { + "retriever": {"query": question}, + "prompt_builder": {"question": question}, + } + + results = rag_pipeline.run(data) + + print(results["llm"]["replies"]) + # Jean lives in Paris + ``` + + :param data: + A dictionary of inputs for the pipeline's components. Each key is a component name + and its value is a dictionary of that component's input parameters: + ``` + data = { + "comp1": {"input1": 1, "input2": 2}, + } + ``` + For convenience, this format is also supported when input names are unique: + ``` + data = { + "input1": 1, "input2": 2, + } + ``` + :param include_outputs_from: + Set of component names whose individual outputs are to be + included in the pipeline's output. For components that are + invoked multiple times (in a loop), only the last-produced + output is included. + :param concurrency_limit: The maximum number of components that should be allowed to run concurrently. + :returns: + A dictionary where each entry corresponds to a component name + and its output. If `include_outputs_from` is `None`, this dictionary + will only contain the outputs of leaf components, i.e., components + without outgoing connections. + + :raises ValueError: + If invalid inputs are provided to the pipeline. + :raises PipelineRuntimeError: + If the Pipeline contains cycles with unsupported connections that would cause + it to get stuck and fail running. + Or if a Component fails or returns output in an unsupported type. + :raises PipelineMaxComponentRuns: + If a Component reaches the maximum number of times it can be run in this Pipeline. + """ + return asyncio.run( + self.run_async(data=data, include_outputs_from=include_outputs_from, concurrency_limit=concurrency_limit) + ) diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index e1e411df1e..73b19bc205 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -2,14 +2,25 @@ from typing import Tuple, List, Dict, Any, Set, Union from pathlib import Path import re +import pytest +import asyncio from pytest_bdd import when, then, parsers -from haystack import Pipeline +from haystack import Pipeline, AsyncPipeline PIPELINE_NAME_REGEX = re.compile(r"\[(.*)\]") +@pytest.fixture(params=[AsyncPipeline, Pipeline]) +def pipeline_class(request): + """ + A parametrized fixture that will yield AsyncPipeline for one test run + and Pipeline for the next test run. + """ + return request.param + + @dataclass class PipelineRunData: """ @@ -34,6 +45,56 @@ class _PipelineResult: @when("I run the Pipeline", target_fixture="pipeline_result") def run_pipeline( + pipeline_data: Tuple[Union[AsyncPipeline, Pipeline], List[PipelineRunData]], spying_tracer +) -> Union[List[Tuple[_PipelineResult, PipelineRunData]], Exception]: + if isinstance(pipeline_data[0], AsyncPipeline): + return run_async_pipeline(pipeline_data, spying_tracer) + else: + return run_sync_pipeline(pipeline_data, spying_tracer) + + +def run_async_pipeline( + pipeline_data: Tuple[Union[AsyncPipeline], List[PipelineRunData]], spying_tracer +) -> Union[List[Tuple[_PipelineResult, PipelineRunData]], Exception]: + """ + Attempts to run a pipeline with the given inputs. + `pipeline_data` is a tuple that must contain: + * A Pipeline instance + * The data to run the pipeline with + + If successful returns a tuple of the run outputs and the expected outputs. + In case an exceptions is raised returns that. + """ + pipeline, pipeline_run_data = pipeline_data[0], pipeline_data[1] + + results: List[_PipelineResult] = [] + + async def run_inner(data, include_outputs_from): + return await pipeline.run_async(data=data.inputs, include_outputs_from=include_outputs_from) + + for data in pipeline_run_data: + try: + async_loop = asyncio.new_event_loop() + asyncio.set_event_loop(async_loop) + outputs = async_loop.run_until_complete(run_inner(data, data.include_outputs_from)) + + component_calls = { + (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ + "haystack.component.input" + ] + for span in spying_tracer.spans + if "haystack.component.name" in span.tags and "haystack.component.visits" in span.tags + } + results.append(_PipelineResult(outputs=outputs, component_calls=component_calls)) + spying_tracer.spans.clear() + except Exception as e: + return e + finally: + async_loop.close() + return [e for e in zip(results, pipeline_run_data)] + + +def run_sync_pipeline( pipeline_data: Tuple[Pipeline, List[PipelineRunData]], spying_tracer ) -> Union[List[Tuple[_PipelineResult, PipelineRunData]], Exception]: """ @@ -61,7 +122,6 @@ def run_pipeline( if "haystack.component.name" in span.tags and "haystack.component.visits" in span.tags } results.append(_PipelineResult(outputs=outputs, component_calls=component_calls)) - spying_tracer.spans.clear() except Exception as e: return e diff --git a/test/core/pipeline/features/test_run.py b/test/core/pipeline/features/test_run.py index 30c415f68d..1aef887ba8 100644 --- a/test/core/pipeline/features/test_run.py +++ b/test/core/pipeline/features/test_run.py @@ -5,7 +5,7 @@ from pytest_bdd import scenarios, given import pytest -from haystack import Pipeline, Document, component +from haystack import Document, component from haystack.document_stores.types import DuplicatePolicy from haystack.dataclasses import ChatMessage, GeneratedAnswer, TextContent, ByteStream from haystack.components.routers import ConditionalRouter, FileTypeRouter @@ -36,22 +36,22 @@ from test.core.pipeline.features.conftest import PipelineRunData -pytestmark = pytest.mark.integration +pytestmark = [pytest.mark.usefixtures("pipeline_class"), pytest.mark.integration] scenarios("pipeline_run.feature") @given("a pipeline that has no components", target_fixture="pipeline_data") -def pipeline_that_has_no_components(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_no_components(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) inputs = {} expected_outputs = {} return pipeline, [PipelineRunData(inputs=inputs, expected_outputs=expected_outputs)] @given("a pipeline that is linear", target_fixture="pipeline_data") -def pipeline_that_is_linear(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_is_linear(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("first_addition", AddFixedValue(add=2)) pipeline.add_component("second_addition", AddFixedValue()) pipeline.add_component("double", Double()) @@ -75,7 +75,7 @@ def pipeline_that_is_linear(): @given("a pipeline that has an infinite loop", target_fixture="pipeline_data") -def pipeline_that_has_an_infinite_loop(): +def pipeline_that_has_an_infinite_loop(pipeline_class): routes = [ {"condition": "{{number > 2}}", "output": "{{number}}", "output_name": "big_number", "output_type": int}, {"condition": "{{number <= 2}}", "output": "{{number + 2}}", "output_name": "small_number", "output_type": int}, @@ -85,7 +85,7 @@ def pipeline_that_has_an_infinite_loop(): first_router = ConditionalRouter(routes=routes) second_router = ConditionalRouter(routes=routes) - pipe = Pipeline(max_runs_per_component=1) + pipe = pipeline_class(max_runs_per_component=1) pipe.add_component("main_input", main_input) pipe.add_component("first_router", first_router) pipe.add_component("second_router", second_router) @@ -98,8 +98,8 @@ def pipeline_that_has_an_infinite_loop(): @given("a pipeline that is really complex with lots of components, forks, and loops", target_fixture="pipeline_data") -def pipeline_complex(): - pipeline = Pipeline(max_runs_per_component=2) +def pipeline_complex(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=2) pipeline.add_component("greet_first", Greet(message="Hello, the value is {value}.")) pipeline.add_component("accumulate_1", Accumulate()) pipeline.add_component("add_two", AddFixedValue(add=2)) @@ -192,14 +192,14 @@ def pipeline_complex(): @given("a pipeline that has a single component with a default input", target_fixture="pipeline_data") -def pipeline_that_has_a_single_component_with_a_default_input(): +def pipeline_that_has_a_single_component_with_a_default_input(pipeline_class): @component class WithDefault: @component.output_types(b=int) def run(self, a: int, b: int = 2): return {"c": a + b} - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("with_defaults", WithDefault()) return ( @@ -220,8 +220,8 @@ def run(self, a: int, b: int = 2): @given("a pipeline that has two loops of identical lengths", target_fixture="pipeline_data") -def pipeline_that_has_two_loops_of_identical_lengths(): - pipeline = Pipeline(max_runs_per_component=10) +def pipeline_that_has_two_loops_of_identical_lengths(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=10) pipeline.add_component("branch_joiner", BranchJoiner(type_=int)) pipeline.add_component("remainder", Remainder(divisor=3)) pipeline.add_component("add_one", AddFixedValue(add=1)) @@ -277,8 +277,8 @@ def pipeline_that_has_two_loops_of_identical_lengths(): @given("a pipeline that has two loops of different lengths", target_fixture="pipeline_data") -def pipeline_that_has_two_loops_of_different_lengths(): - pipeline = Pipeline(max_runs_per_component=10) +def pipeline_that_has_two_loops_of_different_lengths(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=10) pipeline.add_component("branch_joiner", BranchJoiner(type_=int)) pipeline.add_component("remainder", Remainder(divisor=3)) pipeline.add_component("add_one", AddFixedValue(add=1)) @@ -338,9 +338,9 @@ def pipeline_that_has_two_loops_of_different_lengths(): @given("a pipeline that has a single loop with two conditional branches", target_fixture="pipeline_data") -def pipeline_that_has_a_single_loop_with_two_conditional_branches(): +def pipeline_that_has_a_single_loop_with_two_conditional_branches(pipeline_class): accumulator = Accumulate() - pipeline = Pipeline(max_runs_per_component=10) + pipeline = pipeline_class(max_runs_per_component=10) pipeline.add_component("add_one", AddFixedValue(add=1)) pipeline.add_component("branch_joiner", BranchJoiner(type_=int)) @@ -386,8 +386,8 @@ def pipeline_that_has_a_single_loop_with_two_conditional_branches(): @given("a pipeline that has a component with dynamic inputs defined in init", target_fixture="pipeline_data") -def pipeline_that_has_a_component_with_dynamic_inputs_defined_in_init(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_a_component_with_dynamic_inputs_defined_in_init(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("hello", Hello()) pipeline.add_component("fstring", FString(template="This is the greeting: {greeting}!", variables=["greeting"])) pipeline.add_component("splitter", TextSplitter()) @@ -420,8 +420,8 @@ def pipeline_that_has_a_component_with_dynamic_inputs_defined_in_init(): @given("a pipeline that has two branches that don't merge", target_fixture="pipeline_data") -def pipeline_that_has_two_branches_that_dont_merge(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_two_branches_that_dont_merge(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("add_one", AddFixedValue(add=1)) pipeline.add_component("parity", Parity()) pipeline.add_component("add_ten", AddFixedValue(add=10)) @@ -460,8 +460,8 @@ def pipeline_that_has_two_branches_that_dont_merge(): @given("a pipeline that has three branches that don't merge", target_fixture="pipeline_data") -def pipeline_that_has_three_branches_that_dont_merge(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_three_branches_that_dont_merge(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("add_one", AddFixedValue(add=1)) pipeline.add_component("repeat", Repeat(outputs=["first", "second"])) pipeline.add_component("add_ten", AddFixedValue(add=10)) @@ -495,8 +495,8 @@ def pipeline_that_has_three_branches_that_dont_merge(): @given("a pipeline that has two branches that merge", target_fixture="pipeline_data") -def pipeline_that_has_two_branches_that_merge(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_two_branches_that_merge(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("first_addition", AddFixedValue(add=2)) pipeline.add_component("second_addition", AddFixedValue(add=2)) pipeline.add_component("third_addition", AddFixedValue(add=2)) @@ -528,8 +528,8 @@ def pipeline_that_has_two_branches_that_merge(): @given( "a pipeline that has different combinations of branches that merge and do not merge", target_fixture="pipeline_data" ) -def pipeline_that_has_different_combinations_of_branches_that_merge_and_do_not_merge(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_different_combinations_of_branches_that_merge_and_do_not_merge(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("add_one", AddFixedValue()) pipeline.add_component("parity", Parity()) pipeline.add_component("add_ten", AddFixedValue(add=10)) @@ -578,8 +578,8 @@ def pipeline_that_has_different_combinations_of_branches_that_merge_and_do_not_m @given("a pipeline that has two branches, one of which loops back", target_fixture="pipeline_data") -def pipeline_that_has_two_branches_one_of_which_loops_back(): - pipeline = Pipeline(max_runs_per_component=10) +def pipeline_that_has_two_branches_one_of_which_loops_back(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=10) pipeline.add_component("add_zero", AddFixedValue(add=0)) pipeline.add_component("branch_joiner", BranchJoiner(type_=int)) pipeline.add_component("sum", Sum()) @@ -623,7 +623,7 @@ def pipeline_that_has_two_branches_one_of_which_loops_back(): @given("a pipeline that has a component with mutable input", target_fixture="pipeline_data") -def pipeline_that_has_a_component_with_mutable_input(): +def pipeline_that_has_a_component_with_mutable_input(pipeline_class): @component class InputMangler: @component.output_types(mangled_list=List[str]) @@ -631,7 +631,7 @@ def run(self, input_list: List[str]): input_list.append("extra_item") return {"mangled_list": input_list} - pipe = Pipeline(max_runs_per_component=1) + pipe = pipeline_class(max_runs_per_component=1) pipe.add_component("mangler1", InputMangler()) pipe.add_component("mangler2", InputMangler()) pipe.add_component("concat1", StringListJoiner()) @@ -662,7 +662,7 @@ def run(self, input_list: List[str]): @given("a pipeline that has a component with mutable output sent to multiple inputs", target_fixture="pipeline_data") -def pipeline_that_has_a_component_with_mutable_output_sent_to_multiple_inputs(): +def pipeline_that_has_a_component_with_mutable_output_sent_to_multiple_inputs(pipeline_class): @component class PassThroughPromptBuilder: # This is a pass-through component that returns the same input @@ -688,7 +688,7 @@ def run(self, messages: List[ChatMessage]): mm1 = MessageMerger() mm2 = MessageMerger() - pipe = Pipeline(max_runs_per_component=1) + pipe = pipeline_class(max_runs_per_component=1) pipe.add_component("prompt_builder", prompt_builder) pipe.add_component("llm", llm) pipe.add_component("mm1", mm1) @@ -792,7 +792,7 @@ def run(self, messages: List[ChatMessage]): "a pipeline that has a greedy and variadic component after a component with default input", target_fixture="pipeline_data", ) -def pipeline_that_has_a_greedy_and_variadic_component_after_a_component_with_default_input(): +def pipeline_that_has_a_greedy_and_variadic_component_after_a_component_with_default_input(pipeline_class): """ This test verifies that `Pipeline.run()` executes the components in the correct order when there's a greedy Component with variadic input right before a Component with at least one default input. @@ -805,7 +805,7 @@ def pipeline_that_has_a_greedy_and_variadic_component_after_a_component_with_def document_store = InMemoryDocumentStore() document_store.write_documents([Document(content="This is a simple document")]) - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) template = "Given this documents: {{ documents|join(', ', attribute='content') }} Answer this question: {{ query }}" pipeline.add_component("retriever", InMemoryBM25Retriever(document_store=document_store)) pipeline.add_component("prompt_builder", PromptBuilder(template=template)) @@ -868,7 +868,7 @@ def pipeline_that_has_a_greedy_and_variadic_component_after_a_component_with_def @given("a pipeline that has a component that doesn't return a dictionary", target_fixture="pipeline_data") -def pipeline_that_has_a_component_that_doesnt_return_a_dictionary(): +def pipeline_that_has_a_component_that_doesnt_return_a_dictionary(pipeline_class): BrokenComponent = component_class( "BrokenComponent", input_types={"a": int}, @@ -876,13 +876,13 @@ def pipeline_that_has_a_component_that_doesnt_return_a_dictionary(): output=1, # type:ignore ) - pipe = Pipeline(max_runs_per_component=10) + pipe = pipeline_class(max_runs_per_component=10) pipe.add_component("comp", BrokenComponent()) return pipe, [PipelineRunData({"comp": {"a": 1}})] @given("a pipeline that has a component with only default inputs", target_fixture="pipeline_data") -def pipeline_that_has_a_component_with_only_default_inputs(): +def pipeline_that_has_a_component_with_only_default_inputs(pipeline_class): FakeGenerator = component_class( "FakeGenerator", input_types={"prompt": str}, output_types={"replies": List[str]}, output={"replies": ["Paris"]} ) @@ -898,7 +898,7 @@ def pipeline_that_has_a_component_with_only_default_inputs(): "Question: {{ query }}" ) - pipe = Pipeline(max_runs_per_component=1) + pipe = pipeline_class(max_runs_per_component=1) pipe.add_component("retriever", InMemoryBM25Retriever(document_store=doc_store)) pipe.add_component("prompt_builder", PromptBuilder(template=template)) @@ -999,7 +999,9 @@ def pipeline_that_has_a_component_with_only_default_inputs(): "a pipeline that has a component with only default inputs as first to run and receives inputs from a loop", target_fixture="pipeline_data", ) -def pipeline_that_has_a_component_with_only_default_inputs_as_first_to_run_and_receives_inputs_from_a_loop(): +def pipeline_that_has_a_component_with_only_default_inputs_as_first_to_run_and_receives_inputs_from_a_loop( + pipeline_class, +): """ This tests verifies that a Pipeline doesn't get stuck running in a loop if it has all the following characterics: @@ -1050,7 +1052,7 @@ def fake_generator_run(self, generation_kwargs: Optional[Dict[str, Any]] = None, ] ) - pipe = Pipeline(max_runs_per_component=1) + pipe = pipeline_class(max_runs_per_component=1) pipe.add_component("prompt_builder", PromptBuilder(template=template)) pipe.add_component("generator", FakeGenerator()) @@ -1106,8 +1108,8 @@ def fake_generator_run(self, generation_kwargs: Optional[Dict[str, Any]] = None, "a pipeline that has multiple branches that merge into a component with a single variadic input", target_fixture="pipeline_data", ) -def pipeline_that_has_multiple_branches_that_merge_into_a_component_with_a_single_variadic_input(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_multiple_branches_that_merge_into_a_component_with_a_single_variadic_input(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("add_one", AddFixedValue()) pipeline.add_component("parity", Remainder(divisor=2)) pipeline.add_component("add_ten", AddFixedValue(add=10)) @@ -1159,8 +1161,10 @@ def pipeline_that_has_multiple_branches_that_merge_into_a_component_with_a_singl "a pipeline that has multiple branches of different lengths that merge into a component with a single variadic input", target_fixture="pipeline_data", ) -def pipeline_that_has_multiple_branches_of_different_lengths_that_merge_into_a_component_with_a_single_variadic_input(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_has_multiple_branches_of_different_lengths_that_merge_into_a_component_with_a_single_variadic_input( + pipeline_class, +): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("first_addition", AddFixedValue(add=2)) pipeline.add_component("second_addition", AddFixedValue(add=2)) pipeline.add_component("third_addition", AddFixedValue(add=2)) @@ -1192,8 +1196,8 @@ def pipeline_that_has_multiple_branches_of_different_lengths_that_merge_into_a_c @given("a pipeline that is linear and returns intermediate outputs", target_fixture="pipeline_data") -def pipeline_that_is_linear_and_returns_intermediate_outputs(): - pipeline = Pipeline(max_runs_per_component=1) +def pipeline_that_is_linear_and_returns_intermediate_outputs(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("first_addition", AddFixedValue(add=2)) pipeline.add_component("second_addition", AddFixedValue()) pipeline.add_component("double", Double()) @@ -1232,8 +1236,8 @@ def pipeline_that_is_linear_and_returns_intermediate_outputs(): @given("a pipeline that has a loop and returns intermediate outputs from it", target_fixture="pipeline_data") -def pipeline_that_has_a_loop_and_returns_intermediate_outputs_from_it(): - pipeline = Pipeline(max_runs_per_component=10) +def pipeline_that_has_a_loop_and_returns_intermediate_outputs_from_it(pipeline_class): + pipeline = pipeline_class(max_runs_per_component=10) pipeline.add_component("add_one", AddFixedValue(add=1)) pipeline.add_component("branch_joiner", BranchJoiner(type_=int)) pipeline.add_component("below_10", Threshold(threshold=10)) @@ -1297,7 +1301,7 @@ def pipeline_that_has_a_loop_and_returns_intermediate_outputs_from_it(): @given( "a pipeline that is linear and returns intermediate outputs from multiple sockets", target_fixture="pipeline_data" ) -def pipeline_that_is_linear_and_returns_intermediate_outputs_from_multiple_sockets(): +def pipeline_that_is_linear_and_returns_intermediate_outputs_from_multiple_sockets(pipeline_class): @component class DoubleWithOriginal: """ @@ -1308,7 +1312,7 @@ class DoubleWithOriginal: def run(self, value: int): return {"value": value * 2, "original": value} - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("first_addition", AddFixedValue(add=2)) pipeline.add_component("second_addition", AddFixedValue()) pipeline.add_component("double", DoubleWithOriginal()) @@ -1350,14 +1354,14 @@ def run(self, value: int): "a pipeline that has a component with default inputs that doesn't receive anything from its sender", target_fixture="pipeline_data", ) -def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anything_from_its_sender(): +def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anything_from_its_sender(pipeline_class): routes = [ {"condition": "{{'reisen' in sentence}}", "output": "German", "output_name": "language_1", "output_type": str}, {"condition": "{{'viajar' in sentence}}", "output": "Spanish", "output_name": "language_2", "output_type": str}, ] router = ConditionalRouter(routes) - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("router", router) pipeline.add_component("pb", PromptBuilder(template="Ok, I know, that's {{language}}")) pipeline.connect("router.language_2", "pb.language") @@ -1386,7 +1390,9 @@ def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anythi "a pipeline that has a component with default inputs that doesn't receive anything from its sender but receives input from user", target_fixture="pipeline_data", ) -def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anything_from_its_sender_but_receives_input_from_user(): +def pipeline_that_has_a_component_with_default_inputs_that_doesnt_receive_anything_from_its_sender_but_receives_input_from_user( + pipeline_class, +): prompt = PromptBuilder( template="""Please generate an SQL query. The query should answer the following Question: {{ question }}; If the question cannot be answered given the provided table and columns, return 'no_answer' @@ -1436,7 +1442,7 @@ def run(self, query: str): ) fallback_llm = FakeGenerator() - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("prompt", prompt) pipeline.add_component("llm", llm) pipeline.add_component("router", router) @@ -1615,7 +1621,9 @@ def run(self, query: str): "a pipeline that has a loop and a component with default inputs that doesn't receive anything from its sender but receives input from user", target_fixture="pipeline_data", ) -def pipeline_that_has_a_loop_and_a_component_with_default_inputs_that_doesnt_receive_anything_from_its_sender_but_receives_input_from_user(): +def pipeline_that_has_a_loop_and_a_component_with_default_inputs_that_doesnt_receive_anything_from_its_sender_but_receives_input_from_user( + pipeline_class, +): template = """ You are an experienced and accurate Turkish CX speacialist that classifies customer comments into pre-defined categories below:\n Negative experience labels: @@ -1664,7 +1672,7 @@ def run(self, prompt: str): llm = FakeGenerator() validator = FakeOutputValidator() - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("prompt_builder", prompt_builder) pipeline.add_component("llm", llm) @@ -1790,7 +1798,9 @@ def run(self, prompt: str): "a pipeline that has multiple components with only default inputs and are added in a different order from the order of execution", target_fixture="pipeline_data", ) -def pipeline_that_has_multiple_components_with_only_default_inputs_and_are_added_in_a_different_order_from_the_order_of_execution(): +def pipeline_that_has_multiple_components_with_only_default_inputs_and_are_added_in_a_different_order_from_the_order_of_execution( + pipeline_class, +): prompt_builder1 = PromptBuilder( template=""" You are a spellchecking system. Check the given query and fill in the corrected query. @@ -1851,7 +1861,7 @@ class FakeGenerator: def run(self, prompt: str, generation_kwargs: Optional[Dict[str, Any]] = None): return {"replies": ["This is a reply"], "meta": {"meta_key": "meta_value"}} - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component(name="retriever", instance=FakeRetriever()) pipeline.add_component(name="ranker", instance=FakeRanker()) pipeline.add_component(name="prompt_builder2", instance=prompt_builder2) @@ -1954,8 +1964,8 @@ def run(self, prompt: str, generation_kwargs: Optional[Dict[str, Any]] = None): @given("a pipeline that is linear with conditional branching and multiple joins", target_fixture="pipeline_data") -def that_is_linear_with_conditional_branching_and_multiple_joins(): - pipeline = Pipeline() +def that_is_linear_with_conditional_branching_and_multiple_joins(pipeline_class): + pipeline = pipeline_class() @component class FakeRouter: @@ -2065,7 +2075,7 @@ def run(self, query_embedding: List[float]): @given("a pipeline that is a simple agent", target_fixture="pipeline_data") -def that_is_a_simple_agent(): +def that_is_a_simple_agent(pipeline_class): search_message_template = """ Given these web search results: @@ -2170,7 +2180,7 @@ def run(self, query: str): } # main part - pipeline = Pipeline() + pipeline = pipeline_class() pipeline.add_component("main_input", BranchJoiner(List[ChatMessage])) pipeline.add_component("prompt_builder", ChatPromptBuilder(variables=["query"])) pipeline.add_component("llm", FakeThoughtActionOpenAIChatGenerator()) @@ -2507,7 +2517,7 @@ def run(self, replies: List[ChatMessage]): @given("a pipeline that has a variadic component that receives partial inputs", target_fixture="pipeline_data") -def that_has_a_variadic_component_that_receives_partial_inputs(): +def that_has_a_variadic_component_that_receives_partial_inputs(pipeline_class): @component class ConditionalDocumentCreator: def __init__(self, content: str): @@ -2519,7 +2529,7 @@ def run(self, create_document: bool = False): return {"documents": [Document(id=self._content, content=self._content)]} return {"noop": None} - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("first_creator", ConditionalDocumentCreator(content="First document")) pipeline.add_component("second_creator", ConditionalDocumentCreator(content="Second document")) pipeline.add_component("third_creator", ConditionalDocumentCreator(content="Third document")) @@ -2588,7 +2598,7 @@ def run(self, create_document: bool = False): "a pipeline that has a variadic component that receives partial inputs in a different order", target_fixture="pipeline_data", ) -def that_has_a_variadic_component_that_receives_partial_inputs_different_order(): +def that_has_a_variadic_component_that_receives_partial_inputs_different_order(pipeline_class): @component class ConditionalDocumentCreator: def __init__(self, content: str): @@ -2600,7 +2610,7 @@ def run(self, create_document: bool = False): return {"documents": [Document(id=self._content, content=self._content)]} return {"noop": None} - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("third_creator", ConditionalDocumentCreator(content="Third document")) pipeline.add_component("first_creator", ConditionalDocumentCreator(content="First document")) pipeline.add_component("second_creator", ConditionalDocumentCreator(content="Second document")) @@ -2666,10 +2676,10 @@ def run(self, create_document: bool = False): @given("a pipeline that has an answer joiner variadic component", target_fixture="pipeline_data") -def that_has_an_answer_joiner_variadic_component(): +def that_has_an_answer_joiner_variadic_component(pipeline_class): query = "What's Natural Language Processing?" - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("answer_builder_1", AnswerBuilder()) pipeline.add_component("answer_builder_2", AnswerBuilder()) pipeline.add_component("answer_joiner", AnswerJoiner()) @@ -2751,7 +2761,9 @@ def that_has_an_answer_joiner_variadic_component(): "a pipeline that is linear and a component in the middle receives optional input from other components and input from the user", target_fixture="pipeline_data", ) -def that_is_linear_and_a_component_in_the_middle_receives_optional_input_from_other_components_and_input_from_the_user(): +def that_is_linear_and_a_component_in_the_middle_receives_optional_input_from_other_components_and_input_from_the_user( + pipeline_class, +): @component class QueryMetadataExtractor: @component.output_types(filters=Dict[str, str]) @@ -2788,7 +2800,7 @@ def run(self, prompt: str): document_store = InMemoryDocumentStore(bm25_algorithm="BM25Plus") document_store.write_documents(documents=documents, policy=DuplicatePolicy.OVERWRITE) - pipeline = Pipeline() + pipeline = pipeline_class() pipeline.add_component(instance=PromptBuilder('{"disease": "Alzheimer", "year": 2023}'), name="builder") pipeline.add_component(instance=QueryMetadataExtractor(), name="metadata_extractor") pipeline.add_component(instance=InMemoryBM25Retriever(document_store=document_store), name="retriever") @@ -2852,7 +2864,7 @@ def run(self, prompt: str): @given("a pipeline that has a cycle that would get it stuck", target_fixture="pipeline_data") -def that_has_a_cycle_that_would_get_it_stuck(): +def that_has_a_cycle_that_would_get_it_stuck(pipeline_class): template = """ You are an experienced and accurate Turkish CX speacialist that classifies customer comments into pre-defined categories below:\n Negative experience labels: @@ -2903,7 +2915,7 @@ def run(self, prompt: str): llm = FakeGenerator() validator = FakeOutputValidator() - pipeline = Pipeline(max_runs_per_component=1) + pipeline = pipeline_class(max_runs_per_component=1) pipeline.add_component("prompt_builder", prompt_builder) pipeline.add_component("llm", llm) @@ -2920,7 +2932,7 @@ def run(self, prompt: str): @given("a pipeline that has a loop in the middle", target_fixture="pipeline_data") -def that_has_a_loop_in_the_middle(): +def that_has_a_loop_in_the_middle(pipeline_class): @component class FakeGenerator: @component.output_types(replies=List[str]) @@ -2954,7 +2966,7 @@ def run(self, prompt: str): }, ] - pipeline = Pipeline(max_runs_per_component=20) + pipeline = pipeline_class(max_runs_per_component=20) pipeline.add_component("prompt_cleaner", PromptCleaner()) pipeline.add_component("prompt_builder", PromptBuilder(template="", variables=["question", "invalid_replies"])) pipeline.add_component("llm", FakeGenerator()) @@ -3013,8 +3025,8 @@ def run(self, prompt: str): @given("a pipeline that has variadic component that receives a conditional input", target_fixture="pipeline_data") -def that_has_variadic_component_that_receives_a_conditional_input(): - pipe = Pipeline(max_runs_per_component=1) +def that_has_variadic_component_that_receives_a_conditional_input(pipeline_class): + pipe = pipeline_class(max_runs_per_component=1) routes = [ { "condition": "{{ documents|length > 1 }}", @@ -3279,11 +3291,11 @@ def run(self, documents: List[Document]): @given("a pipeline that has a string variadic component", target_fixture="pipeline_data") -def that_has_a_string_variadic_component(): +def that_has_a_string_variadic_component(pipeline_class): string_1 = "What's Natural Language Processing?" string_2 = "What's is life?" - pipeline = Pipeline() + pipeline = pipeline_class() pipeline.add_component("prompt_builder_1", PromptBuilder("Builder 1: {{query}}")) pipeline.add_component("prompt_builder_2", PromptBuilder("Builder 2: {{query}}")) pipeline.add_component("string_joiner", StringJoiner()) @@ -3318,7 +3330,7 @@ def that_has_a_string_variadic_component(): @given("a pipeline that is an agent that can use RAG", target_fixture="pipeline_data") -def an_agent_that_can_use_RAG(): +def an_agent_that_can_use_RAG(pipeline_class): @component class FixedGenerator: def __init__(self, replies): @@ -3397,7 +3409,7 @@ def run(self, query: str): answer_builder = AnswerBuilder() - pp = Pipeline(max_runs_per_component=2) + pp = pipeline_class(max_runs_per_component=2) pp.add_component("joiner", joiner) pp.add_component("rag_llm", rag_llm) @@ -3566,7 +3578,7 @@ def run(self, query: str): @given("a pipeline that has a feedback loop", target_fixture="pipeline_data") -def has_feedback_loop(): +def has_feedback_loop(pipeline_class): @component class FixedGenerator: def __init__(self, replies): @@ -3627,7 +3639,7 @@ def run(self, prompt: str): answer_builder = AnswerBuilder() - pp = Pipeline(max_runs_per_component=100) + pp = pipeline_class(max_runs_per_component=100) pp.add_component("code_llm", code_llm) pp.add_component("code_prompt", code_prompt) @@ -3730,7 +3742,7 @@ def run(self, prompt: str): @given("a pipeline created in a non-standard order that has a loop", target_fixture="pipeline_data") -def has_non_standard_order_loop(): +def has_non_standard_order_loop(pipeline_class): @component class FixedGenerator: def __init__(self, replies): @@ -3791,7 +3803,7 @@ def run(self, prompt: str): answer_builder = AnswerBuilder() - pp = Pipeline(max_runs_per_component=100) + pp = pipeline_class(max_runs_per_component=100) pp.add_component("concatenator", concatenator) pp.add_component("code_llm", code_llm) @@ -3895,7 +3907,7 @@ def run(self, prompt: str): @given("a pipeline that has an agent with a feedback cycle", target_fixture="pipeline_data") -def agent_with_feedback_cycle(): +def agent_with_feedback_cycle(pipeline_class): @component class FixedGenerator: def __init__(self, replies): @@ -3985,7 +3997,7 @@ def run(self, replies: List[str]): joiner = BranchJoiner(type_=str) agent_concatenator = OutputAdapter(template="{{current_prompt + '\n' + files}}", output_type=str) - pp = Pipeline(max_runs_per_component=100) + pp = pipeline_class(max_runs_per_component=100) pp.add_component("code_prompt", code_prompt) pp.add_component("joiner", joiner) @@ -4629,7 +4641,7 @@ def run(self, replies: List[str]): @given("a pipeline that passes outputs that are consumed in cycle to outside the cycle", target_fixture="pipeline_data") -def passes_outputs_outside_cycle(): +def passes_outputs_outside_cycle(pipeline_class): @component class FixedGenerator: def __init__(self, replies): @@ -4705,7 +4717,7 @@ def generate_santa_sleigh(): answer_builder = AnswerBuilderWithPrompt() - pp = Pipeline(max_runs_per_component=100) + pp = pipeline_class(max_runs_per_component=100) pp.add_component("concatenator", concatenator) pp.add_component("code_llm", code_llm) @@ -4899,7 +4911,7 @@ def generate_santa_sleigh(): @given("a pipeline with a component that has dynamic default inputs", target_fixture="pipeline_data") -def pipeline_with_dynamic_defaults(): +def pipeline_with_dynamic_defaults(pipeline_class): @component class ParrotWithDynamicDefaultInputs: def __init__(self, input_variable: str): @@ -4911,7 +4923,7 @@ def run(self, **kwargs): return {"response": kwargs[self.input_variable]} parrot = ParrotWithDynamicDefaultInputs("parrot") - pipeline = Pipeline() + pipeline = pipeline_class() pipeline.add_component("parrot", parrot) return ( pipeline, @@ -4931,7 +4943,7 @@ def run(self, **kwargs): @given("a pipeline with a component that has variadic dynamic default inputs", target_fixture="pipeline_data") -def pipeline_with_variadic_dynamic_defaults(): +def pipeline_with_variadic_dynamic_defaults(pipeline_class): @component class ParrotWithVariadicDynamicDefaultInputs: def __init__(self, input_variable: str): @@ -4943,7 +4955,7 @@ def run(self, **kwargs): return {"response": kwargs[self.input_variable]} parrot = ParrotWithVariadicDynamicDefaultInputs("parrot") - pipeline = Pipeline() + pipeline = pipeline_class() pipeline.add_component("parrot", parrot) return ( pipeline, @@ -4963,7 +4975,7 @@ def run(self, **kwargs): @given("a pipeline that is a file conversion pipeline with two joiners", target_fixture="pipeline_data") -def pipeline_that_converts_files(): +def pipeline_that_converts_files(pipeline_class): csv_data = """ some,header,row 0,1,0 @@ -4988,7 +5000,7 @@ def pipeline_that_converts_files(): b_joiner = DocumentJoiner() a_joiner = DocumentJoiner() - pp = Pipeline(max_runs_per_component=1) + pp = pipeline_class(max_runs_per_component=1) pp.add_component("router", router) pp.add_component("splitter", splitter) diff --git a/test/core/pipeline/test_async_pipeline.py b/test/core/pipeline/test_async_pipeline.py new file mode 100644 index 0000000000..65da849334 --- /dev/null +++ b/test/core/pipeline/test_async_pipeline.py @@ -0,0 +1,26 @@ +import asyncio + +from haystack import AsyncPipeline + + +def test_async_pipeline_reentrance(waiting_component, spying_tracer): + pp = AsyncPipeline() + pp.add_component("wait", waiting_component()) + + run_data = [{"wait_for": 1}, {"wait_for": 2}] + + async_loop = asyncio.new_event_loop() + asyncio.set_event_loop(async_loop) + + async def run_all(): + # Create concurrent tasks for each pipeline run + tasks = [pp.run_async(data) for data in run_data] + await asyncio.gather(*tasks) + + try: + async_loop.run_until_complete(run_all()) + component_spans = [sp for sp in spying_tracer.spans if sp.operation_name == "haystack.component.run_async"] + for span in component_spans: + assert span.tags["haystack.component.visits"] == 1 + finally: + async_loop.close() From fe1e1f1367f1306b5b4ade157e97514b585a321a Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 17:58:23 +0100 Subject: [PATCH 70/88] chore: Add release notes --- .../notes/feat-async-pipeline-338856a142e1318c.yaml | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml diff --git a/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml b/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml new file mode 100644 index 0000000000..f882e0a369 --- /dev/null +++ b/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml @@ -0,0 +1,8 @@ +--- +highlights: > + We are introducing the `AsyncPipeline`: Supports running pipelines asynchronously. Schedules components concurrently + whenever possible. Leads to major speed improvements for any pipelines that may run workloads in parallel. +features: + - | + Added a new `AsyncPipeline` implementation to call pipelines from async-code and support concurrent component + scheduling. \ No newline at end of file From 99e8da3ac013ab8008bdec6f0625b4f2c8380d80 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Tue, 4 Feb 2025 18:06:54 +0100 Subject: [PATCH 71/88] fix: format --- releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml b/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml index f882e0a369..4336a30623 100644 --- a/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml +++ b/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml @@ -5,4 +5,4 @@ highlights: > features: - | Added a new `AsyncPipeline` implementation to call pipelines from async-code and support concurrent component - scheduling. \ No newline at end of file + scheduling. From 267bbb9be374a1448ba7d30f190a3a2990b9258b Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 09:47:48 +0100 Subject: [PATCH 72/88] debug: switch run order to debug ubuntu and windows tests --- test/core/pipeline/features/pipeline_run.feature | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/core/pipeline/features/pipeline_run.feature b/test/core/pipeline/features/pipeline_run.feature index 8724dd3e1e..380459c947 100644 --- a/test/core/pipeline/features/pipeline_run.feature +++ b/test/core/pipeline/features/pipeline_run.feature @@ -3,8 +3,8 @@ Feature: Pipeline running Scenario Outline: Running a correct Pipeline Given a pipeline When I run the Pipeline - Then it should return the expected result - And components are called with the expected inputs + Then components are called with the expected inputs + And it should return the expected result Examples: | kind | From 2695163117e2cf02d49ede2038b0f3c8c8d89fb1 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 10:14:21 +0100 Subject: [PATCH 73/88] fix: consider priorities of other components while waiting for DEFER --- haystack/core/pipeline/async_pipeline.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/haystack/core/pipeline/async_pipeline.py b/haystack/core/pipeline/async_pipeline.py index 9b31d7ce1f..5f8eafe5ad 100644 --- a/haystack/core/pipeline/async_pipeline.py +++ b/haystack/core/pipeline/async_pipeline.py @@ -345,9 +345,15 @@ async def _schedule_defer_incrementally(component_name: str) -> AsyncIterator[Di async for partial_result in _wait_for_one_task_to_complete(): yield partial_result - elif priority in (ComponentPriority.DEFER, ComponentPriority.DEFER_LAST): + elif priority in (ComponentPriority.DEFER, ComponentPriority.DEFER_LAST) and running_tasks: # We do incremental waiting - async for partial_result in _schedule_defer_incrementally(component_name): + async for partial_result in _wait_for_one_task_to_complete(): + yield partial_result + + elif priority in (ComponentPriority.DEFER, ComponentPriority.DEFER_LAST) and not running_tasks: + await _schedule_ready_task(component_name) + + async for partial_result in _wait_for_one_task_to_complete(): yield partial_result # End main loop From 1ab79cbbf99faacf6a69dcd29209d497f4a0788a Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 10:54:48 +0100 Subject: [PATCH 74/88] refactor: simplify code --- haystack/core/pipeline/async_pipeline.py | 67 ++++-------------------- 1 file changed, 9 insertions(+), 58 deletions(-) diff --git a/haystack/core/pipeline/async_pipeline.py b/haystack/core/pipeline/async_pipeline.py index 5f8eafe5ad..da2c21414a 100644 --- a/haystack/core/pipeline/async_pipeline.py +++ b/haystack/core/pipeline/async_pipeline.py @@ -189,9 +189,9 @@ async def _run_highest_in_isolation(component_name: str) -> AsyncIterator[Dict[s if result: yield {component_name: deepcopy(result)} - async def _schedule_ready_task(component_name: str) -> None: + async def _schedule_task(component_name: str) -> None: """ - Schedule a component that is considered READY (or just turned READY). + Schedule a component to run. We do NOT wait for it to finish here. This allows us to run other components concurrently. @@ -247,48 +247,6 @@ async def _wait_for_all_tasks_to_complete() -> AsyncIterator[Dict[str, Any]]: if partial_result: yield {finished_component_name: deepcopy(partial_result)} - async def _schedule_defer_incrementally(component_name: str) -> AsyncIterator[Dict[str, Any]]: - """ - Schedule a component that has priority DEFER or DEFER_LAST. - - Waits for tasks to complete one-by-one. Schedules the component as soon as it turns READY. - If the component does not turn READY, it drains the queue completely before scheduling the component. - - :param component_name: The name of the component. - :returns: An async iterator of partial outputs. - """ - comp_dict = self._get_component_with_graph_metadata_and_visits( - component_name, component_visits[component_name] - ) - while True: - # Already scheduled => stop - if component_name in scheduled_components: - return - # Priority is recalculated after each completed task - - new_prio = self._calculate_priority(comp_dict, inputs_state.get(component_name, {})) - if new_prio == ComponentPriority.READY: - # It's now ready => schedule it - await _schedule_ready_task(component_name) - return - - elif new_prio == ComponentPriority.HIGHEST: - # Edge case: somehow became HIGHEST => run in isolation - async for partial_out in _run_highest_in_isolation(component_name): - yield partial_out - return - - # else it remains DEFER or DEFER_LAST, keep waiting - if running_tasks: - # Wait for just one task to finish - async for part in _wait_for_one_task_to_complete(): - yield part - else: - # No tasks left => schedule anyway (end of pipeline) - # This ensures we don't deadlock forever. - await _schedule_ready_task(component_name) - return - # ------------------------------------------------- # MAIN SCHEDULING LOOP # ------------------------------------------------- @@ -323,7 +281,7 @@ async def _schedule_defer_incrementally(component_name: str) -> AsyncIterator[Di if priority == ComponentPriority.READY: # 1) schedule this one - await _schedule_ready_task(component_name) + await _schedule_task(component_name) # 2) Possibly schedule more READY tasks if concurrency not fully used while len(priority_queue) > 0 and not ready_sem.locked(): @@ -333,7 +291,7 @@ async def _schedule_defer_incrementally(component_name: str) -> AsyncIterator[Di break if peek_prio == ComponentPriority.READY: priority_queue.pop() - await _schedule_ready_task(peek_name) + await _schedule_task(peek_name) # keep adding while concurrency is not locked continue @@ -341,20 +299,13 @@ async def _schedule_defer_incrementally(component_name: str) -> AsyncIterator[Di # We'll handle it in the next iteration or with incremental waiting break - # 3) Wait for at least 1 task to finish => yield partial - async for partial_result in _wait_for_one_task_to_complete(): - yield partial_result - - elif priority in (ComponentPriority.DEFER, ComponentPriority.DEFER_LAST) and running_tasks: - # We do incremental waiting - async for partial_result in _wait_for_one_task_to_complete(): - yield partial_result - + # We only schedule components with priority DEFER or DEFER_LAST when no other tasks are running elif priority in (ComponentPriority.DEFER, ComponentPriority.DEFER_LAST) and not running_tasks: - await _schedule_ready_task(component_name) + await _schedule_task(component_name) - async for partial_result in _wait_for_one_task_to_complete(): - yield partial_result + # To make progress, we wait for one task to complete before re-starting the loop + async for partial_result in _wait_for_one_task_to_complete(): + yield partial_result # End main loop From c3f4c726f455f3dc3e6fd549beeedaff5f2a04db Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 15:03:44 +0100 Subject: [PATCH 75/88] fix: resolve merge conflict with mermaid changes --- haystack/core/pipeline/base.py | 74 ++++++++++++++++++++++++++++------ 1 file changed, 62 insertions(+), 12 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index c4686f8aac..95f0635354 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -8,9 +8,9 @@ from datetime import datetime from enum import IntEnum from pathlib import Path -from typing import Any, Dict, Iterator, List, Optional, TextIO, Tuple, Type, TypeVar, Union +from typing import Any, Dict, Iterator, List, Optional, Set, TextIO, Tuple, Type, TypeVar, Union -from networkx import MultiDiGraph # type:ignore +import networkx # type:ignore from haystack import logging from haystack.core.component import Component, InputSocket, OutputSocket, component @@ -42,9 +42,14 @@ from haystack.marshal import Marshaller, YamlMarshaller from haystack.utils import is_in_jupyter, type_serialization +from .descriptions import find_pipeline_inputs, find_pipeline_outputs +from .draw import _to_mermaid_image +from .template import PipelineTemplate, PredefinedPipeline +from .utils import parse_connect_string + DEFAULT_MARSHALLER = YamlMarshaller() -# We use a generic type to annotate the return value of classmethods, +# We use a generic type to annotate the return value of class methods, # so that static analyzers won't be confused when derived classes # use those methods. T = TypeVar("T", bound="PipelineBase") @@ -82,7 +87,7 @@ def __init__(self, metadata: Optional[Dict[str, Any]] = None, max_runs_per_compo self._telemetry_runs = 0 self._last_telemetry_sent: Optional[datetime] = None self.metadata = metadata or {} - self.graph = MultiDiGraph() + self.graph = networkx.MultiDiGraph() self._max_runs_per_component = max_runs_per_component def __eq__(self, other) -> bool: @@ -637,31 +642,76 @@ def outputs(self, include_components_with_connected_outputs: bool = False) -> Di } return outputs - def show(self) -> None: + def show(self, server_url: str = "https://mermaid.ink", params: Optional[dict] = None) -> None: """ - If running in a Jupyter notebook, display an image representing this `Pipeline`. + Display an image representing this `Pipeline` in a Jupyter notebook. + + This function generates a diagram of the `Pipeline` using a Mermaid server and displays it directly in + the notebook. + + :param server_url: + The base URL of the Mermaid server used for rendering (default: 'https://mermaid.ink'). + See https://github.com/jihchi/mermaid.ink and https://github.com/mermaid-js/mermaid-live-editor for more + info on how to set up your own Mermaid server. + :param params: + Dictionary of customization parameters to modify the output. Refer to Mermaid documentation for more details + Supported keys: + - format: Output format ('img', 'svg', or 'pdf'). Default: 'img'. + - type: Image type for /img endpoint ('jpeg', 'png', 'webp'). Default: 'png'. + - theme: Mermaid theme ('default', 'neutral', 'dark', 'forest'). Default: 'neutral'. + - bgColor: Background color in hexadecimal (e.g., 'FFFFFF') or named format (e.g., '!white'). + - width: Width of the output image (integer). + - height: Height of the output image (integer). + - scale: Scaling factor (1–3). Only applicable if 'width' or 'height' is specified. + - fit: Whether to fit the diagram size to the page (PDF only, boolean). + - paper: Paper size for PDFs (e.g., 'a4', 'a3'). Ignored if 'fit' is true. + - landscape: Landscape orientation for PDFs (boolean). Ignored if 'fit' is true. + + :raises PipelineDrawingError: + If the function is called outside of a Jupyter notebook or if there is an issue with rendering. """ if is_in_jupyter(): from IPython.display import Image, display # type: ignore - image_data = _to_mermaid_image(self.graph) - + image_data = _to_mermaid_image(self.graph, server_url=server_url, params=params) display(Image(image_data)) else: msg = "This method is only supported in Jupyter notebooks. Use Pipeline.draw() to save an image locally." raise PipelineDrawingError(msg) - def draw(self, path: Path) -> None: + def draw(self, path: Path, server_url: str = "https://mermaid.ink", params: Optional[dict] = None) -> None: """ - Save an image representing this `Pipeline` to `path`. + Save an image representing this `Pipeline` to the specified file path. + + This function generates a diagram of the `Pipeline` using the Mermaid server and saves it to the provided path. :param path: - The path to save the image to. + The file path where the generated image will be saved. + :param server_url: + The base URL of the Mermaid server used for rendering (default: 'https://mermaid.ink'). + See https://github.com/jihchi/mermaid.ink and https://github.com/mermaid-js/mermaid-live-editor for more + info on how to set up your own Mermaid server. + :param params: + Dictionary of customization parameters to modify the output. Refer to Mermaid documentation for more details + Supported keys: + - format: Output format ('img', 'svg', or 'pdf'). Default: 'img'. + - type: Image type for /img endpoint ('jpeg', 'png', 'webp'). Default: 'png'. + - theme: Mermaid theme ('default', 'neutral', 'dark', 'forest'). Default: 'neutral'. + - bgColor: Background color in hexadecimal (e.g., 'FFFFFF') or named format (e.g., '!white'). + - width: Width of the output image (integer). + - height: Height of the output image (integer). + - scale: Scaling factor (1–3). Only applicable if 'width' or 'height' is specified. + - fit: Whether to fit the diagram size to the page (PDF only, boolean). + - paper: Paper size for PDFs (e.g., 'a4', 'a3'). Ignored if 'fit' is true. + - landscape: Landscape orientation for PDFs (boolean). Ignored if 'fit' is true. + + :raises PipelineDrawingError: + If there is an issue with rendering or saving the image. """ # Before drawing we edit a bit the graph, to avoid modifying the original that is # used for running the pipeline we copy it. - image_data = _to_mermaid_image(self.graph) + image_data = _to_mermaid_image(self.graph, server_url=server_url, params=params) Path(path).write_bytes(image_data) def walk(self) -> Iterator[Tuple[str, Component]]: From e0d27121fef87d913cd75e9402472e449d86eed6 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 15:16:48 +0100 Subject: [PATCH 76/88] fix: format --- haystack/core/pipeline/base.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 95f0635354..51cdf9fdd5 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -33,9 +33,6 @@ is_any_greedy_socket_ready, is_socket_lazy_variadic, ) -from haystack.core.pipeline.descriptions import find_pipeline_inputs, find_pipeline_outputs -from haystack.core.pipeline.draw import _to_mermaid_image -from haystack.core.pipeline.template import PipelineTemplate, PredefinedPipeline from haystack.core.pipeline.utils import FIFOPriorityQueue, parse_connect_string from haystack.core.serialization import DeserializationCallbacks, component_from_dict, component_to_dict from haystack.core.type_utils import _type_name, _types_are_compatible @@ -45,7 +42,6 @@ from .descriptions import find_pipeline_inputs, find_pipeline_outputs from .draw import _to_mermaid_image from .template import PipelineTemplate, PredefinedPipeline -from .utils import parse_connect_string DEFAULT_MARSHALLER = YamlMarshaller() From 3ded49f00ea905aa71e95bac1ef0fbf9ab3c90cb Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 15:32:19 +0100 Subject: [PATCH 77/88] fix: remove unused import --- haystack/core/pipeline/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/haystack/core/pipeline/base.py b/haystack/core/pipeline/base.py index 51cdf9fdd5..d1eb983957 100644 --- a/haystack/core/pipeline/base.py +++ b/haystack/core/pipeline/base.py @@ -8,7 +8,7 @@ from datetime import datetime from enum import IntEnum from pathlib import Path -from typing import Any, Dict, Iterator, List, Optional, Set, TextIO, Tuple, Type, TypeVar, Union +from typing import Any, Dict, Iterator, List, Optional, TextIO, Tuple, Type, TypeVar, Union import networkx # type:ignore From 4564c3abee2addefba4dcf78b2b055c579575ce8 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Wed, 5 Feb 2025 15:34:33 +0100 Subject: [PATCH 78/88] refactor: rename to avoid accidental conflicts --- haystack/core/pipeline/utils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/haystack/core/pipeline/utils.py b/haystack/core/pipeline/utils.py index a9006cf110..0746dc6746 100644 --- a/haystack/core/pipeline/utils.py +++ b/haystack/core/pipeline/utils.py @@ -52,8 +52,8 @@ def push(self, item: Any, priority: int) -> None: :param priority: Priority level for the item. Lower numbers indicate higher priority. """ - count = next(self._counter) - entry = (priority, count, item) + next_count = next(self._counter) + entry = (priority, next_count, item) heapq.heappush(self._queue, entry) def pop(self) -> Tuple[int, Any]: From e9f43531a6e91bc73eb3dc7928cc265a246e9faf Mon Sep 17 00:00:00 2001 From: mathislucka Date: Thu, 6 Feb 2025 15:41:22 +0100 Subject: [PATCH 79/88] fix: track pipeline type --- haystack/telemetry/_telemetry.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/haystack/telemetry/_telemetry.py b/haystack/telemetry/_telemetry.py index f0cfbf269b..4ba3a0c765 100644 --- a/haystack/telemetry/_telemetry.py +++ b/haystack/telemetry/_telemetry.py @@ -8,7 +8,7 @@ import uuid from collections import defaultdict from pathlib import Path -from typing import TYPE_CHECKING, Any, Dict, List, Optional, Tuple +from typing import TYPE_CHECKING, Any, Dict, List, Optional, Tuple, Union import posthog import yaml @@ -18,7 +18,7 @@ from haystack.telemetry._environment import collect_system_specs if TYPE_CHECKING: - from haystack.core.pipeline import Pipeline + from haystack.core.pipeline import AsyncPipeline, Pipeline HAYSTACK_TELEMETRY_ENABLED = "HAYSTACK_TELEMETRY_ENABLED" @@ -135,7 +135,7 @@ def send_telemetry_wrapper(*args, **kwargs): @send_telemetry -def pipeline_running(pipeline: "Pipeline") -> Optional[Tuple[str, Dict[str, Any]]]: +def pipeline_running(pipeline: Union["Pipeline", "AsyncPipeline"]) -> Optional[Tuple[str, Dict[str, Any]]]: """ Collects telemetry data for a pipeline run and sends it to Posthog. @@ -170,6 +170,7 @@ def pipeline_running(pipeline: "Pipeline") -> Optional[Tuple[str, Dict[str, Any] # Data sent to Posthog return "Pipeline run (2.x)", { "pipeline_id": str(id(pipeline)), + "pipeline_type": generate_qualified_class_name(type(pipeline)), "runs": pipeline._telemetry_runs, "components": components, } From e9fbf7b450ba76c5fe501df6599e3b9b0685e8d9 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Thu, 6 Feb 2025 15:52:38 +0100 Subject: [PATCH 80/88] fix: and extend test --- test/test_telemetry.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/test/test_telemetry.py b/test/test_telemetry.py index 2b6f9e7e71..3cb7338333 100644 --- a/test/test_telemetry.py +++ b/test/test_telemetry.py @@ -7,13 +7,17 @@ import pytest -from haystack import Pipeline, component +from haystack import AsyncPipeline, Pipeline, component +from haystack.core.serialization import generate_qualified_class_name from haystack.telemetry._telemetry import pipeline_running from haystack.utils.auth import Secret, TokenSecret - +@pytest.mark.parametrize( + "pipeline_class", + [Pipeline, AsyncPipeline] +) @patch("haystack.telemetry._telemetry.telemetry") -def test_pipeline_running(telemetry): +def test_pipeline_running(telemetry, pipeline_class): telemetry.send_event = Mock() @component @@ -25,15 +29,17 @@ def _get_telemetry_data(self): def run(self): pass - pipe = Pipeline() + pipe = pipeline_class() pipe.add_component("component", Component()) pipeline_running(pipe) + expected_type = generate_qualified_class_name(type(pipe)) # First run is always sent telemetry.send_event.assert_called_once_with( "Pipeline run (2.x)", { "pipeline_id": str(id(pipe)), + "pipeline_type": expected_type, "runs": 1, "components": {"test.test_telemetry.Component": [{"name": "component", "key": "values"}]}, }, @@ -53,6 +59,7 @@ def run(self): "Pipeline run (2.x)", { "pipeline_id": str(id(pipe)), + "pipeline_type": expected_type, "runs": 3, "components": {"test.test_telemetry.Component": [{"name": "component", "key": "values"}]}, }, @@ -82,6 +89,7 @@ def run(self): "Pipeline run (2.x)", { "pipeline_id": str(id(pipe)), + "pipeline_type": "haystack.core.pipeline.pipeline.Pipeline", "runs": 1, "components": {"test.test_telemetry.Component": [{"name": "component", "key": "values"}]}, }, From 56d28532e18d76bc74e92ff8e2d661b5e27f2608 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Thu, 6 Feb 2025 15:54:17 +0100 Subject: [PATCH 81/88] fix: format --- test/test_telemetry.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/test/test_telemetry.py b/test/test_telemetry.py index 3cb7338333..18b88c20fe 100644 --- a/test/test_telemetry.py +++ b/test/test_telemetry.py @@ -12,10 +12,8 @@ from haystack.telemetry._telemetry import pipeline_running from haystack.utils.auth import Secret, TokenSecret -@pytest.mark.parametrize( - "pipeline_class", - [Pipeline, AsyncPipeline] -) + +@pytest.mark.parametrize("pipeline_class", [Pipeline, AsyncPipeline]) @patch("haystack.telemetry._telemetry.telemetry") def test_pipeline_running(telemetry, pipeline_class): telemetry.send_event = Mock() From f2bde4ff92740882e93aee8acbe9d42c3597ecf4 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Fri, 7 Feb 2025 09:51:52 +0100 Subject: [PATCH 82/88] style: sort alphabetically --- haystack/__init__.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/haystack/__init__.py b/haystack/__init__.py index 7d521b433e..7d375c9a4a 100644 --- a/haystack/__init__.py +++ b/haystack/__init__.py @@ -18,16 +18,16 @@ haystack.tracing.auto_enable_tracing() __all__ = [ - "component", - "default_from_dict", - "default_to_dict", - "DeserializationError", - "ComponentError", + "Answer", "AsyncPipeline", - "Pipeline", - "PredefinedPipeline", + "ComponentError", + "DeserializationError", "Document", - "Answer", - "GeneratedAnswer", "ExtractedAnswer", + "GeneratedAnswer", + "Pipeline", + "PredefinedPipeline", + "component", + "default_from_dict", + "default_to_dict", ] From 06cfad978524c908cd47f5ccf568ea317a18bcb0 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Fri, 7 Feb 2025 14:30:56 +0100 Subject: [PATCH 83/88] Update test/core/pipeline/features/conftest.py Co-authored-by: Amna Mubashar --- test/core/pipeline/features/conftest.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index 73b19bc205..d2e5705469 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -74,9 +74,8 @@ async def run_inner(data, include_outputs_from): for data in pipeline_run_data: try: - async_loop = asyncio.new_event_loop() - asyncio.set_event_loop(async_loop) - outputs = async_loop.run_until_complete(run_inner(data, data.include_outputs_from)) + outputs = asyncio.run(run_inner(data, data.include_outputs_from)) + component_calls = { (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ From 54f33ebcdc24372257864f40db380de1a82b1add Mon Sep 17 00:00:00 2001 From: mathislucka Date: Fri, 7 Feb 2025 14:31:10 +0100 Subject: [PATCH 84/88] Update test/core/pipeline/features/conftest.py Co-authored-by: Amna Mubashar --- test/core/pipeline/features/conftest.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index d2e5705469..9d956d8bf8 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -70,6 +70,8 @@ def run_async_pipeline( results: List[_PipelineResult] = [] async def run_inner(data, include_outputs_from): + """Wrapper function to call pipeline.run_async method with required params.""" + return await pipeline.run_async(data=data.inputs, include_outputs_from=include_outputs_from) for data in pipeline_run_data: From 2d7c33d123b30cf7227a553471c2adaa92f30ef9 Mon Sep 17 00:00:00 2001 From: Amna Mubashar Date: Fri, 7 Feb 2025 14:33:57 +0100 Subject: [PATCH 85/88] Update releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml --- releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml b/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml index 4336a30623..323c78c266 100644 --- a/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml +++ b/releasenotes/notes/feat-async-pipeline-338856a142e1318c.yaml @@ -4,5 +4,5 @@ highlights: > whenever possible. Leads to major speed improvements for any pipelines that may run workloads in parallel. features: - | - Added a new `AsyncPipeline` implementation to call pipelines from async-code and support concurrent component - scheduling. + Added a new `AsyncPipeline` implementation that allows pipelines to be executed from async code, + supporting concurrent scheduling of pipeline components for faster processing. From f56186c8b934838ef42b0b23d2cf4f0d200a90f5 Mon Sep 17 00:00:00 2001 From: mathislucka Date: Fri, 7 Feb 2025 14:34:48 +0100 Subject: [PATCH 86/88] fix: indentation, do not close loop --- test/core/pipeline/features/conftest.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/test/core/pipeline/features/conftest.py b/test/core/pipeline/features/conftest.py index 9d956d8bf8..d4139968aa 100644 --- a/test/core/pipeline/features/conftest.py +++ b/test/core/pipeline/features/conftest.py @@ -70,15 +70,13 @@ def run_async_pipeline( results: List[_PipelineResult] = [] async def run_inner(data, include_outputs_from): - """Wrapper function to call pipeline.run_async method with required params.""" - + """Wrapper function to call pipeline.run_async method with required params.""" return await pipeline.run_async(data=data.inputs, include_outputs_from=include_outputs_from) for data in pipeline_run_data: try: outputs = asyncio.run(run_inner(data, data.include_outputs_from)) - component_calls = { (span.tags["haystack.component.name"], span.tags["haystack.component.visits"]): span.tags[ "haystack.component.input" @@ -90,8 +88,7 @@ async def run_inner(data, include_outputs_from): spying_tracer.spans.clear() except Exception as e: return e - finally: - async_loop.close() + return [e for e in zip(results, pipeline_run_data)] From 57b1cac6b3d1c3fe20d5738ba82ed1197ef7be5d Mon Sep 17 00:00:00 2001 From: mathislucka Date: Fri, 7 Feb 2025 16:08:06 +0100 Subject: [PATCH 87/88] fix: use asyncio.run --- test/core/pipeline/test_async_pipeline.py | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/test/core/pipeline/test_async_pipeline.py b/test/core/pipeline/test_async_pipeline.py index 65da849334..dbf46fcc42 100644 --- a/test/core/pipeline/test_async_pipeline.py +++ b/test/core/pipeline/test_async_pipeline.py @@ -9,18 +9,13 @@ def test_async_pipeline_reentrance(waiting_component, spying_tracer): run_data = [{"wait_for": 1}, {"wait_for": 2}] - async_loop = asyncio.new_event_loop() - asyncio.set_event_loop(async_loop) - async def run_all(): # Create concurrent tasks for each pipeline run tasks = [pp.run_async(data) for data in run_data] await asyncio.gather(*tasks) - try: - async_loop.run_until_complete(run_all()) - component_spans = [sp for sp in spying_tracer.spans if sp.operation_name == "haystack.component.run_async"] - for span in component_spans: - assert span.tags["haystack.component.visits"] == 1 - finally: - async_loop.close() + + asyncio.run(run_all()) + component_spans = [sp for sp in spying_tracer.spans if sp.operation_name == "haystack.component.run_async"] + for span in component_spans: + assert span.tags["haystack.component.visits"] == 1 From 47f419c29c3089e496b9466b0b989d8e3273d6fc Mon Sep 17 00:00:00 2001 From: mathislucka Date: Fri, 7 Feb 2025 16:08:36 +0100 Subject: [PATCH 88/88] fix: format --- test/core/pipeline/test_async_pipeline.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/core/pipeline/test_async_pipeline.py b/test/core/pipeline/test_async_pipeline.py index dbf46fcc42..474449f782 100644 --- a/test/core/pipeline/test_async_pipeline.py +++ b/test/core/pipeline/test_async_pipeline.py @@ -14,7 +14,6 @@ async def run_all(): tasks = [pp.run_async(data) for data in run_data] await asyncio.gather(*tasks) - asyncio.run(run_all()) component_spans = [sp for sp in spying_tracer.spans if sp.operation_name == "haystack.component.run_async"] for span in component_spans: