def test_connected_pipeline(self, disjoint_pipeline): """Connect two separate pipelines.""" nodes = disjoint_pipeline["nodes"] subpipeline = Pipeline(nodes, tags=["subpipeline"]) assert len(subpipeline.inputs()) == 2 assert len(subpipeline.outputs()) == 2 pipeline = Pipeline( [node(identity, "C", "D", name="connecting_node"), subpipeline], tags="main" ) assert len(pipeline.nodes) == 1 + len(nodes) assert len(pipeline.inputs()) == 1 assert len(pipeline.outputs()) == 1
def test_free_input(self, input_data): nodes = input_data["nodes"] inputs = input_data["free_inputs"] pipeline = Pipeline(nodes) assert pipeline.inputs() == set(inputs)
def run(self, pipeline: Pipeline, catalog: DataCatalog) -> Dict[str, Any]: """Run the ``Pipeline`` using the ``DataSet``s provided by ``catalog`` and save results back to the same objects. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. Raises: ValueError: Raised when ``Pipeline`` inputs cannot be satisfied. Returns: Any node outputs that cannot be processed by the ``DataCatalog``. These are returned in a dictionary, where the keys are defined by the node outputs. """ catalog = catalog.shallow_copy() unsatisfied = pipeline.inputs() - set(catalog.list()) if unsatisfied: raise ValueError("Pipeline input(s) {} not found in the " "DataCatalog".format(unsatisfied)) free_outputs = pipeline.outputs() - set(catalog.list()) unregistered_ds = pipeline.data_sets() - set(catalog.list()) for ds_name in unregistered_ds: catalog.add(ds_name, self.create_default_data_set(ds_name)) self._run(pipeline, catalog) self._logger.info("Pipeline execution completed successfully.") return {ds_name: catalog.load(ds_name) for ds_name in free_outputs}
def _run(self, pipeline: Pipeline, catalog: DataCatalog) -> None: """The method implementing sequential pipeline running. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. """ nodes = pipeline.nodes load_counts = Counter(chain.from_iterable(n.inputs for n in nodes)) for exec_index, node in enumerate(nodes): run_node(node, catalog) # decrement load counts and release any data sets we've finished with for data_set in node.inputs: load_counts[data_set] -= 1 if load_counts[data_set] < 1 and data_set not in pipeline.inputs(): catalog.release(data_set) for data_set in node.outputs: if load_counts[data_set] < 1 and data_set not in pipeline.outputs(): catalog.release(data_set) self._logger.info( "Completed %d out of %d tasks", exec_index + 1, len(nodes) )
def _run(self, pipeline: Pipeline, catalog: DataCatalog) -> None: """The abstract interface for running pipelines. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. Raises: AttributeError: when the provided pipeline is not suitable for parallel execution. """ self._validate_catalog(catalog, pipeline) self._validate_nodes(pipeline.nodes) done_inputs = pipeline.inputs() todo_nodes = set(pipeline.nodes) futures = set() with ProcessPoolExecutor() as pool: while True: ready = {n for n in todo_nodes if set(n.inputs) <= done_inputs} todo_nodes -= ready for node in ready: futures.add(pool.submit(run_node, node, catalog)) if not futures: assert not todo_nodes break done, futures = wait(futures, return_when=FIRST_COMPLETED) for future in done: done_inputs.update(future.result().outputs)
def _run( # pylint: disable=too-many-locals,useless-suppression self, pipeline: Pipeline, catalog: DataCatalog) -> None: """The abstract interface for running pipelines. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. Raises: AttributeError: when the provided pipeline is not suitable for parallel execution. Exception: in case of any downstream node failure. """ nodes = pipeline.nodes self._validate_catalog(catalog, pipeline) self._validate_nodes(nodes) load_counts = Counter(chain.from_iterable(n.inputs for n in nodes)) node_dependencies = pipeline.node_dependencies todo_nodes = set(node_dependencies.keys()) done_nodes = set() # type: Set[Node] futures = set() done = None max_workers = self._get_required_workers_count(pipeline) with ProcessPoolExecutor(max_workers=max_workers) as pool: while True: ready = { n for n in todo_nodes if node_dependencies[n] <= done_nodes } todo_nodes -= ready for node in ready: futures.add(pool.submit(run_node, node, catalog)) if not futures: assert not todo_nodes, (todo_nodes, done_nodes, ready, done) break done, futures = wait(futures, return_when=FIRST_COMPLETED) for future in done: try: node = future.result() except Exception: self._suggest_resume_scenario(pipeline, done_nodes) raise done_nodes.add(node) # decrement load counts and release any data sets we've finished with # this is particularly important for the shared datasets we create above for data_set in node.inputs: load_counts[data_set] -= 1 if (load_counts[data_set] < 1 and data_set not in pipeline.inputs()): catalog.release(data_set) for data_set in node.outputs: if (load_counts[data_set] < 1 and data_set not in pipeline.outputs()): catalog.release(data_set)
def test_remove_from_empty_pipeline(self): """Remove node from an empty pipeline""" pipeline1 = Pipeline([node(biconcat, ["input", "input1"], "output1", name="a")]) pipeline2 = Pipeline([]) new_pipeline = pipeline2 - pipeline1 assert new_pipeline.inputs() == pipeline2.inputs() assert new_pipeline.outputs() == pipeline2.outputs() assert not new_pipeline.nodes
def test_remove_empty_from_pipeline(self): """Remove an empty pipeline""" pipeline1 = Pipeline([node(biconcat, ["input", "input1"], "output1", name="a")]) pipeline2 = Pipeline([]) new_pipeline = pipeline1 - pipeline2 assert new_pipeline.inputs() == pipeline1.inputs() assert new_pipeline.outputs() == pipeline1.outputs() assert {n.name for n in new_pipeline.nodes} == {"a"}
def _run(self, pipeline: Pipeline, catalog: DataCatalog, run_id: str = None) -> None: """The method implementing sequential pipeline running. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. run_id: The id of the run. Raises: Exception: in case of any downstream node failure. """ nodes = pipeline.nodes done_nodes = set() load_counts = Counter(chain.from_iterable(n.inputs for n in nodes)) for exec_index, node in enumerate(nodes): try: run_node(node, catalog, self._is_async, run_id) done_nodes.add(node) except Exception: self._suggest_resume_scenario(pipeline, done_nodes) raise # decrement load counts and release any data sets we've finished with for data_set in node.inputs: load_counts[data_set] -= 1 if load_counts[ data_set] < 1 and data_set not in pipeline.inputs(): catalog.release(data_set) for data_set in node.outputs: if load_counts[ data_set] < 1 and data_set not in pipeline.outputs(): catalog.release(data_set) self._logger.info("Completed %d out of %d tasks", exec_index + 1, len(nodes))
def _run( # pylint: disable=too-many-locals,useless-suppression self, pipeline: Pipeline, catalog: DataCatalog, run_id: str = None) -> None: """The abstract interface for running pipelines. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. run_id: The id of the run. Raises: AttributeError: When the provided pipeline is not suitable for parallel execution. RuntimeError: If the runner is unable to schedule the execution of all pipeline nodes. Exception: In case of any downstream node failure. """ # pylint: disable=import-outside-toplevel,cyclic-import from kedro.framework.session.session import get_current_session nodes = pipeline.nodes self._validate_catalog(catalog, pipeline) self._validate_nodes(nodes) load_counts = Counter(chain.from_iterable(n.inputs for n in nodes)) node_dependencies = pipeline.node_dependencies todo_nodes = set(node_dependencies.keys()) done_nodes = set() # type: Set[Node] futures = set() done = None max_workers = self._get_required_workers_count(pipeline) from kedro.framework.project import PACKAGE_NAME session = get_current_session(silent=True) # pylint: disable=protected-access conf_logging = session._get_logging_config() if session else None with ProcessPoolExecutor(max_workers=max_workers) as pool: while True: ready = { n for n in todo_nodes if node_dependencies[n] <= done_nodes } todo_nodes -= ready for node in ready: futures.add( pool.submit( _run_node_synchronization, node, catalog, self._is_async, run_id, package_name=PACKAGE_NAME, conf_logging=conf_logging, )) if not futures: if todo_nodes: debug_data = { "todo_nodes": todo_nodes, "done_nodes": done_nodes, "ready_nodes": ready, "done_futures": done, } debug_data_str = "\n".join( f"{k} = {v}" for k, v in debug_data.items()) raise RuntimeError( f"Unable to schedule new tasks although some nodes " f"have not been run:\n{debug_data_str}") break # pragma: no cover done, futures = wait(futures, return_when=FIRST_COMPLETED) for future in done: try: node = future.result() except Exception: self._suggest_resume_scenario(pipeline, done_nodes) raise done_nodes.add(node) # decrement load counts and release any data sets we've finished with # this is particularly important for the shared datasets we create above for data_set in node.inputs: load_counts[data_set] -= 1 if (load_counts[data_set] < 1 and data_set not in pipeline.inputs()): catalog.release(data_set) for data_set in node.outputs: if (load_counts[data_set] < 1 and data_set not in pipeline.outputs()): catalog.release(data_set)
def _run( # pylint: disable=too-many-locals,useless-suppression self, pipeline: Pipeline, catalog: DataCatalog, run_id: str = None) -> None: """The abstract interface for running pipelines. Args: pipeline: The ``Pipeline`` to run. catalog: The ``DataCatalog`` from which to fetch data. run_id: The id of the run. Raises: Exception: in case of any downstream node failure. """ nodes = pipeline.nodes load_counts = Counter(chain.from_iterable(n.inputs for n in nodes)) node_dependencies = pipeline.node_dependencies todo_nodes = set(node_dependencies.keys()) done_nodes = set() # type: Set[Node] futures = set() done = None max_workers = self._get_required_workers_count(pipeline) with ThreadPoolExecutor(max_workers=max_workers) as pool: while True: ready = { n for n in todo_nodes if node_dependencies[n] <= done_nodes } todo_nodes -= ready for node in ready: futures.add( pool.submit(run_node, node, catalog, self._is_async, run_id)) if not futures: assert not todo_nodes, (todo_nodes, done_nodes, ready, done) break done, futures = wait(futures, return_when=FIRST_COMPLETED) for future in done: try: node = future.result() except Exception: self._suggest_resume_scenario(pipeline, done_nodes) raise done_nodes.add(node) self._logger.info("Completed node: %s", node.name) self._logger.info("Completed %d out of %d tasks", len(done_nodes), len(nodes)) # Decrement load counts, and release any datasets we # have finished with. for data_set in node.inputs: load_counts[data_set] -= 1 if (load_counts[data_set] < 1 and data_set not in pipeline.inputs()): catalog.release(data_set) for data_set in node.outputs: if (load_counts[data_set] < 1 and data_set not in pipeline.outputs()): catalog.release(data_set)