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 should_skip_step(execution_plan: ExecutionPlan, instance: DagsterInstance, run_id: str) -> bool: """[INTERNAL] Check if it should skip executing the plan. Primarily used by execution without run-level plan process, e.g. Airflow step execution. Note: this only checks one step at a time. For each Dagster execution step - if none of its inputs come from optional outputs, do not skip - if there is at least one input, where none of the upstream steps have yielded an output, we should skip the step. """ check.inst_param(execution_plan, "execution_plan", ExecutionPlan) check.inst_param(instance, "instance", DagsterInstance) check.str_param(run_id, "run_id") # only checks one step at a time if len(execution_plan.step_keys_to_execute) != 1: return False step_key = execution_plan.step_keys_to_execute[0] optional_source_handles = set() step = execution_plan.get_executable_step_by_key(step_key) for step_input in step.step_inputs: for source_handle in step_input.get_step_output_handle_dependencies(): if not execution_plan.get_step_output(source_handle).is_required: optional_source_handles.add(source_handle) # early terminate to avoid unnecessary instance/db calls if len(optional_source_handles) == 0: # do not skip when all the inputs come from non-optional outputs return False # find all yielded step outputs all_logs = instance.all_logs(run_id) yielded_step_output_handles = set() for event_record in all_logs: if event_record.dagster_event and event_record.dagster_event.is_successful_output: yielded_step_output_handles.add( event_record.dagster_event.event_specific_data. step_output_handle) # If there is at least one of the step's inputs, none of whose upstream steps has # yielded an output, we should skip that step. for step_input in step.step_inputs: missing_source_handles = [ source_handle for source_handle in step_input.get_step_output_handle_dependencies() if source_handle in optional_source_handles and source_handle not in yielded_step_output_handles ] if len(missing_source_handles) > 0 and len( missing_source_handles) == len( step_input.get_step_output_handle_dependencies()): return True return False