def _resolve_reexecute_step_selection( instance: DagsterInstance, pipeline: IPipeline, mode: Optional[str], run_config: Optional[dict], parent_pipeline_run: PipelineRun, step_selection: List[str], ) -> ExecutionPlan: if parent_pipeline_run.solid_selection: pipeline = pipeline.subset_for_execution( parent_pipeline_run.solid_selection) parent_logs = instance.all_logs(parent_pipeline_run.run_id) parent_plan = create_execution_plan( pipeline, parent_pipeline_run.run_config, mode, known_state=KnownExecutionState.derive_from_logs(parent_logs), ) step_keys_to_execute = parse_step_selection( parent_plan.get_all_step_deps(), step_selection) execution_plan = create_execution_plan( pipeline, run_config, mode, step_keys_to_execute=list(step_keys_to_execute), known_state=KnownExecutionState.for_reexecution( parent_logs, step_keys_to_execute), ) return execution_plan
def reexecute_pipeline_iterator( pipeline: Union[IPipeline, PipelineDefinition], parent_run_id: str, run_config: Optional[dict] = None, step_selection: Optional[List[str]] = None, mode: Optional[str] = None, preset: Optional[str] = None, tags: Optional[Dict[str, Any]] = None, instance: DagsterInstance = None, ) -> Iterator[DagsterEvent]: """Reexecute a pipeline iteratively. Rather than package up the result of running a pipeline into a single object, like :py:func:`reexecute_pipeline`, this function yields the stream of events resulting from pipeline reexecution. This is intended to allow the caller to handle these events on a streaming basis in whatever way is appropriate. Parameters: pipeline (Union[IPipeline, PipelineDefinition]): The pipeline to execute. parent_run_id (str): The id of the previous run to reexecute. The run must exist in the instance. run_config (Optional[dict]): The environment configuration that parametrizes this run, as a dict. step_selection (Optional[List[str]]): A list of step selection queries (including single step keys) to execute. For example: - ['some_solid']: select the execution step "some_solid" itself. - ['*some_solid']: select the step "some_solid" and all its ancestors (upstream dependencies). - ['*some_solid+++']: select the step "some_solid", all its ancestors, and its descendants (downstream dependencies) within 3 levels down. - ['*some_solid', 'other_solid_a', 'other_solid_b+']: select "some_solid" and all its ancestors, "other_solid_a" itself, and "other_solid_b" and its direct child execution steps. mode (Optional[str]): The name of the pipeline mode to use. You may not set both ``mode`` and ``preset``. preset (Optional[str]): The name of the pipeline preset to use. You may not set both ``mode`` and ``preset``. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. instance (Optional[DagsterInstance]): The instance to execute against. If this is ``None``, an ephemeral instance will be used, and no artifacts will be persisted from the run. Returns: Iterator[DagsterEvent]: The stream of events resulting from pipeline reexecution. """ check.opt_list_param(step_selection, "step_selection", of_type=str) check.str_param(parent_run_id, "parent_run_id") with _ephemeral_instance_if_missing(instance) as execute_instance: (pipeline, run_config, mode, tags, _, _) = _check_execute_pipeline_args( pipeline=pipeline, run_config=run_config, mode=mode, preset=preset, tags=tags, solid_selection=None, ) parent_pipeline_run = execute_instance.get_run_by_id(parent_run_id) check.invariant( parent_pipeline_run, "No parent run with id {parent_run_id} found in instance.".format( parent_run_id=parent_run_id), ) # resolve step selection DSL queries using parent execution plan snapshot if step_selection: parent_execution_plan_snapshot = execute_instance.get_execution_plan_snapshot( parent_pipeline_run.execution_plan_snapshot_id) step_keys_to_execute = parse_step_selection( parent_execution_plan_snapshot.step_deps, step_selection) else: step_keys_to_execute = None pipeline_run = execute_instance.create_run_for_pipeline( pipeline_def=pipeline.get_definition(), run_config=run_config, mode=mode, tags=tags, solid_selection=parent_pipeline_run.solid_selection, solids_to_execute=parent_pipeline_run.solids_to_execute, # convert to frozenset https://github.com/dagster-io/dagster/issues/2914 step_keys_to_execute=list(step_keys_to_execute) if step_keys_to_execute else None, root_run_id=parent_pipeline_run.root_run_id or parent_pipeline_run.run_id, parent_run_id=parent_pipeline_run.run_id, ) return execute_run_iterator(pipeline, pipeline_run, execute_instance)
def reexecute_pipeline( pipeline: Union[IPipeline, PipelineDefinition], parent_run_id: str, run_config: Optional[dict] = None, step_selection: Optional[List[str]] = None, mode: Optional[str] = None, preset: Optional[str] = None, tags: Optional[Dict[str, Any]] = None, instance: DagsterInstance = None, raise_on_error: bool = True, ) -> PipelineExecutionResult: """Reexecute an existing pipeline run. Users will typically call this API when testing pipeline reexecution, or running standalone scripts. Parameters: pipeline (Union[IPipeline, PipelineDefinition]): The pipeline to execute. parent_run_id (str): The id of the previous run to reexecute. The run must exist in the instance. run_config (Optional[dict]): The environment configuration that parametrizes this run, as a dict. step_selection (Optional[List[str]]): A list of step selection queries (including single step keys) to execute. For example: - ['some_solid']: select the execution step "some_solid" itself. - ['*some_solid']: select the step "some_solid" and all its ancestors (upstream dependencies). - ['*some_solid+++']: select the step "some_solid", all its ancestors, and its descendants (downstream dependencies) within 3 levels down. - ['*some_solid', 'other_solid_a', 'other_solid_b+']: select "some_solid" and all its ancestors, "other_solid_a" itself, and "other_solid_b" and its direct child execution steps. mode (Optional[str]): The name of the pipeline mode to use. You may not set both ``mode`` and ``preset``. preset (Optional[str]): The name of the pipeline preset to use. You may not set both ``mode`` and ``preset``. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. instance (Optional[DagsterInstance]): The instance to execute against. If this is ``None``, an ephemeral instance will be used, and no artifacts will be persisted from the run. raise_on_error (Optional[bool]): Whether or not to raise exceptions when they occur. Defaults to ``True``, since this is the most useful behavior in test. Returns: :py:class:`PipelineExecutionResult`: The result of pipeline execution. For the asynchronous version, see :py:func:`reexecute_pipeline_iterator`. """ check.opt_list_param(step_selection, "step_selection", of_type=str) check.str_param(parent_run_id, "parent_run_id") with _ephemeral_instance_if_missing(instance) as execute_instance: (pipeline, run_config, mode, tags, _, _) = _check_execute_pipeline_args( pipeline=pipeline, run_config=run_config, mode=mode, preset=preset, tags=tags, ) parent_pipeline_run = execute_instance.get_run_by_id(parent_run_id) check.invariant( parent_pipeline_run, "No parent run with id {parent_run_id} found in instance.".format( parent_run_id=parent_run_id), ) # resolve step selection DSL queries using parent execution plan snapshot if step_selection: full_plan = create_execution_plan(pipeline, parent_pipeline_run.run_config, mode) step_keys = parse_items_from_selection(step_selection) # resolve execution plan with any resolved dynamic step keys resolved_plan = full_plan.build_subset_plan(step_keys) # parse selection using all step deps step_keys_to_execute = parse_step_selection( resolved_plan.get_all_step_deps(), step_selection) else: step_keys_to_execute = None pipeline_run = execute_instance.create_run_for_pipeline( pipeline_def=pipeline.get_definition(), run_config=run_config, mode=mode, tags=tags, solid_selection=parent_pipeline_run.solid_selection, solids_to_execute=parent_pipeline_run.solids_to_execute, # convert to frozenset https://github.com/dagster-io/dagster/issues/2914 step_keys_to_execute=list(step_keys_to_execute) if step_keys_to_execute else None, root_run_id=parent_pipeline_run.root_run_id or parent_pipeline_run.run_id, parent_run_id=parent_pipeline_run.run_id, ) return execute_run(pipeline, pipeline_run, execute_instance, raise_on_error=raise_on_error)