def _event_record(run_id, solid_name, timestamp, event_type, event_specific_data=None): pipeline_name = "pipeline_name" solid_handle = SolidHandle(solid_name, None) step_handle = StepHandle(solid_handle) return EventRecord( None, "", "debug", "", run_id, timestamp, step_key=step_handle.to_key(), pipeline_name=pipeline_name, dagster_event=DagsterEvent( event_type.value, pipeline_name, solid_handle=solid_handle, step_handle=step_handle, event_specific_data=event_specific_data, ), )
def test_step_handles(): plain = StepHandle.parse_from_key("foo") assert isinstance(plain, StepHandle) unresolved = StepHandle.parse_from_key("foo[?]") assert isinstance(unresolved, UnresolvedStepHandle) resolved = StepHandle.parse_from_key("foo[bar]") assert isinstance(resolved, ResolvedFromDynamicStepHandle) assert unresolved.resolve("bar") == resolved assert resolved.unresolved_form == unresolved
def build_subset_plan(self, step_keys_to_execute: List[str]) -> "ExecutionPlan": check.list_param(step_keys_to_execute, "step_keys_to_execute", of_type=str) step_handles_to_execute = [ StepHandle.parse_from_key(key) for key in step_keys_to_execute ] bad_keys = [] for handle in step_handles_to_execute: if handle not in self.step_dict: bad_keys.append(handle.to_key()) if bad_keys: raise DagsterExecutionStepNotFoundError( f"Can not build subset plan from unknown step{'s' if len(bad_keys)> 1 else ''}: {', '.join(bad_keys)}", step_keys=bad_keys, ) return ExecutionPlan( self.pipeline, self.step_dict, step_handles_to_execute, self.environment_config, self.known_state, )
def create_compute_step(pipeline_name, environment_config, solid, step_inputs, handle): check.str_param(pipeline_name, "pipeline_name") check.inst_param(solid, "solid", Solid) check.list_param(step_inputs, "step_inputs", of_type=StepInput) check.opt_inst_param(handle, "handle", SolidHandle) # the environment config has the solid output name configured config_output_names = set() current_handle = handle while current_handle: solid_config = environment_config.solids.get( current_handle.to_string()) current_handle = current_handle.parent config_output_names = config_output_names.union( solid_config.outputs.output_names) return ExecutionStep( handle=StepHandle(solid_handle=handle), pipeline_name=pipeline_name, step_inputs=step_inputs, step_outputs=[ StepOutput(output_def=output_def, should_materialize=name in config_output_names) for name, output_def in solid.definition.output_dict.items() ], compute_fn=lambda step_context, inputs: _execute_core_compute( step_context.for_compute(), inputs, solid.definition.compute_fn), solid=solid, )
def build(pipeline, environment_config, mode=None, step_keys_to_execute=None): """Here we build a new ExecutionPlan from a pipeline definition and the environment config. To do this, we iterate through the pipeline's solids in topological order, and hand off the execution steps for each solid to a companion _PlanBuilder object. Once we've processed the entire pipeline, we invoke _PlanBuilder.build() to construct the ExecutionPlan object. """ check.inst_param(pipeline, "pipeline", IPipeline) check.inst_param(environment_config, "environment_config", EnvironmentConfig) check.opt_str_param(mode, "mode") check.opt_list_param(step_keys_to_execute, "step_keys_to_execute", of_type=str) step_handles_to_execute = ( [StepHandle.from_key(key) for key in step_keys_to_execute] if step_keys_to_execute else None ) plan_builder = _PlanBuilder( pipeline, environment_config, mode=mode, step_handles_to_execute=step_handles_to_execute, ) # Finally, we build and return the execution plan return plan_builder.build()
def build_subset_plan(self, step_keys_to_execute): check.list_param(step_keys_to_execute, "step_keys_to_execute", of_type=str) step_handles_to_execute = [StepHandle.from_key(key) for key in step_keys_to_execute] return ExecutionPlan( self.pipeline, self.step_dict, step_handles_to_execute, self.artifacts_persisted, self.environment_config, )
def __new__( cls, event_type_value, pipeline_name, step_handle=None, solid_handle=None, step_kind_value=None, logging_tags=None, event_specific_data=None, message=None, pid=None, # legacy step_key=None, ): event_type_value, event_specific_data = _handle_back_compat( event_type_value, event_specific_data ) # old events may contain solid_handle but not step_handle if solid_handle is not None and step_handle is None: step_handle = StepHandle(solid_handle) # Legacy events may have step_key set directly, preserve those to stay in sync # with legacy execution plan snapshots. if step_handle is not None and step_key is None: step_key = step_handle.to_key() return super(DagsterEvent, cls).__new__( cls, check.str_param(event_type_value, "event_type_value"), check.str_param(pipeline_name, "pipeline_name"), check.opt_inst_param( step_handle, "step_handle", (StepHandle, ResolvedFromDynamicStepHandle) ), check.opt_inst_param(solid_handle, "solid_handle", SolidHandle), check.opt_str_param(step_kind_value, "step_kind_value"), check.opt_dict_param(logging_tags, "logging_tags"), _validate_event_specific_data(DagsterEventType(event_type_value), event_specific_data), check.opt_str_param(message, "message"), check.opt_int_param(pid, "pid"), check.opt_str_param(step_key, "step_key"), )
def create_compute_step(solid, solid_handle, step_inputs, pipeline_name, environment_config): check.inst_param(solid, "solid", Solid) check.inst_param(solid_handle, "solid_handle", SolidHandle) check.list_param(step_inputs, "step_inputs", of_type=StepInput) check.str_param(pipeline_name, "pipeline_name") return ExecutionStep( handle=StepHandle(solid_handle=solid_handle), pipeline_name=pipeline_name, step_inputs=step_inputs, step_outputs=_create_step_outputs(solid, solid_handle, environment_config), compute_fn=lambda step_context, inputs: _execute_core_compute( step_context.for_compute(), inputs, solid.definition.compute_fn), solid=solid, )
def build_subset_plan( self, step_keys_to_execute: List[str], pipeline_def: PipelineDefinition, environment_config: EnvironmentConfig, ) -> "ExecutionPlan": check.list_param(step_keys_to_execute, "step_keys_to_execute", of_type=str) step_handles_to_execute = [ StepHandle.parse_from_key(key) for key in step_keys_to_execute ] bad_keys = [] for handle in step_handles_to_execute: if handle not in self.step_dict: bad_keys.append(handle.to_key()) if bad_keys: raise DagsterExecutionStepNotFoundError( f"Can not build subset plan from unknown step{'s' if len(bad_keys)> 1 else ''}: {', '.join(bad_keys)}", step_keys=bad_keys, ) executable_map, resolvable_map = _compute_step_maps( self.step_dict, step_handles_to_execute, self.known_state, ) return ExecutionPlan( self.step_dict, executable_map, resolvable_map, step_handles_to_execute, self.known_state, _compute_artifacts_persisted( self.step_dict, step_handles_to_execute, pipeline_def, environment_config, executable_map, ), )
def build_subset_plan(self, step_keys_to_execute: List[str]) -> "ExecutionPlan": check.list_param(step_keys_to_execute, "step_keys_to_execute", of_type=str) step_handles_to_execute = [ StepHandle.parse_from_key(key) for key in step_keys_to_execute ] bad_keys = [] for handle in step_handles_to_execute: if handle in self.step_dict: pass # no further processing required elif (isinstance(handle, ResolvedFromDynamicStepHandle) and handle.unresolved_form in self.step_dict): unresolved_step = cast(UnresolvedExecutionStep, self.step_dict[handle.unresolved_form]) # self.step_dict updated as side effect self.resolve( unresolved_step.resolved_by_step_key, { unresolved_step.resolved_by_output_name: [handle.mapping_key] }, ) check.invariant( handle in self.step_dict, f"Handle did not resolve as expected, not found in step dict {handle}", ) else: bad_keys.append(handle.to_key()) if bad_keys: raise DagsterExecutionStepNotFoundError( f"Can not build subset plan from unknown step{'s' if len(bad_keys)> 1 else ''}: {', '.join(bad_keys)}", step_keys=bad_keys, ) return ExecutionPlan( self.pipeline, self.step_dict, step_handles_to_execute, self.environment_config, )
def rebuild_from_snapshot(pipeline_name, execution_plan_snapshot): if not execution_plan_snapshot.can_reconstruct_plan: raise DagsterInvariantViolationError( "Tried to reconstruct an old ExecutionPlanSnapshot that was created before snapshots " "had enough information to fully reconstruct the ExecutionPlan" ) step_dict = {} for step_snap in execution_plan_snapshot.steps: input_snaps = step_snap.inputs output_snaps = step_snap.outputs step_inputs = [ ExecutionPlan.rebuild_step_input(step_input_snap) for step_input_snap in input_snaps ] step_outputs = [ StepOutput( step_output_snap.solid_handle, step_output_snap.name, step_output_snap.dagster_type_key, step_output_snap.properties, ) for step_output_snap in output_snaps ] if step_snap.kind == StepKind.COMPUTE: step = ExecutionStep( step_snap.step_handle, pipeline_name, step_inputs, step_outputs, step_snap.tags, ) elif step_snap.kind == StepKind.UNRESOLVED_MAPPED: step = UnresolvedMappedExecutionStep( step_snap.step_handle, pipeline_name, step_inputs, step_outputs, step_snap.tags, ) elif step_snap.kind == StepKind.UNRESOLVED_COLLECT: step = UnresolvedCollectExecutionStep( step_snap.step_handle, pipeline_name, step_inputs, step_outputs, step_snap.tags, ) else: raise Exception(f"Unexpected step kind {str(step_snap.kind)}") step_dict[step.handle] = step step_handles_to_execute = [ StepHandle.parse_from_key(key) for key in execution_plan_snapshot.step_keys_to_execute ] executable_map, resolvable_map = _compute_step_maps( step_dict, step_handles_to_execute, execution_plan_snapshot.initial_known_state, ) return ExecutionPlan( step_dict, executable_map, resolvable_map, step_handles_to_execute, execution_plan_snapshot.initial_known_state, execution_plan_snapshot.artifacts_persisted, )
def _build_from_sorted_solids( self, solids: List[Solid], dependency_structure: DependencyStructure, parent_handle: Optional[SolidHandle] = None, parent_step_inputs: Optional[List[ Union[StepInput, UnresolvedMappedStepInput, UnresolvedCollectStepInput]]] = None, ): for solid in solids: handle = SolidHandle(solid.name, parent_handle) ### 1. INPUTS # Create and add execution plan steps for solid inputs has_unresolved_input = False has_pending_input = False step_inputs: List[Union[StepInput, UnresolvedMappedStepInput, UnresolvedCollectStepInput]] = [] for input_name, input_def in solid.definition.input_dict.items(): step_input_source = get_step_input_source( self, solid, input_name, input_def, dependency_structure, handle, parent_step_inputs, ) # If an input with dagster_type "Nothing" doesn't have a value # we don't create a StepInput if step_input_source is None: continue if isinstance( step_input_source, (FromPendingDynamicStepOutput, FromUnresolvedStepOutput), ): has_unresolved_input = True step_inputs.append( UnresolvedMappedStepInput( name=input_name, dagster_type_key=input_def.dagster_type.key, source=step_input_source, )) elif isinstance(step_input_source, FromDynamicCollect): has_pending_input = True step_inputs.append( UnresolvedCollectStepInput( name=input_name, dagster_type_key=input_def.dagster_type.key, source=step_input_source, )) else: check.inst_param(step_input_source, "step_input_source", StepInputSource) step_inputs.append( StepInput( name=input_name, dagster_type_key=input_def.dagster_type.key, source=step_input_source, )) ### 2a. COMPUTE FUNCTION # Create and add execution plan step for the solid compute function if isinstance(solid.definition, SolidDefinition): step_outputs = create_step_outputs(solid, handle, self.environment_config) if has_pending_input and has_unresolved_input: check.failed( "Can not have pending and unresolved step inputs") elif has_unresolved_input: new_step: IExecutionStep = UnresolvedMappedExecutionStep( handle=UnresolvedStepHandle(solid_handle=handle), pipeline_name=self.pipeline_name, step_inputs=cast( List[Union[StepInput, UnresolvedMappedStepInput]], step_inputs), step_outputs=step_outputs, tags=solid.tags, ) elif has_pending_input: new_step = UnresolvedCollectExecutionStep( handle=StepHandle(solid_handle=handle), pipeline_name=self.pipeline_name, step_inputs=cast( List[Union[StepInput, UnresolvedCollectStepInput]], step_inputs), step_outputs=step_outputs, tags=solid.tags, ) else: new_step = ExecutionStep( handle=StepHandle(solid_handle=handle), pipeline_name=self.pipeline_name, step_inputs=cast(List[StepInput], step_inputs), step_outputs=step_outputs, tags=solid.tags, ) self.add_step(new_step) ### 2b. RECURSE # Recurse over the solids contained in an instance of GraphDefinition elif isinstance(solid.definition, GraphDefinition): self._build_from_sorted_solids( solid.definition.solids_in_topological_order, solid.definition.dependency_structure, parent_handle=handle, parent_step_inputs=step_inputs, ) else: check.invariant( False, "Unexpected solid type {type} encountered during execution planning" .format(type=type(solid.definition)), ) ### 3. OUTPUTS # Create output handles for solid outputs for name, output_def in solid.definition.output_dict.items(): output_handle = solid.output_handle(name) # Punch through layers of composition scope to map to the output of the # actual compute step resolved_output_def, resolved_handle = solid.definition.resolve_output_to_origin( output_def.name, handle) step = self.get_step_by_solid_handle(resolved_handle) if isinstance(step, (ExecutionStep, UnresolvedCollectExecutionStep)): step_output_handle: Union[ StepOutputHandle, UnresolvedStepOutputHandle] = StepOutputHandle( step.key, resolved_output_def.name) elif isinstance(step, UnresolvedMappedExecutionStep): step_output_handle = UnresolvedStepOutputHandle( step.handle, resolved_output_def.name, step.resolved_by_step_key, step.resolved_by_output_name, ) else: check.failed(f"Unexpected step type {step}") self.set_output_handle(output_handle, step_output_handle)
def has_step(self, key): check.str_param(key, "key") handle = StepHandle.parse_from_key(key) if isinstance(handle, ResolvedFromDynamicStepHandle): return handle.unresolved_form.to_key() in self._step_index return key in self._step_index
def get_retry_steps_from_execution_plan(instance, execution_plan, parent_run_id): check.inst_param(instance, "instance", DagsterInstance) check.inst_param(execution_plan, "execution_plan", ExternalExecutionPlan) check.opt_str_param(parent_run_id, "parent_run_id") if not parent_run_id: return execution_plan.step_keys_in_plan parent_run = instance.get_run_by_id(parent_run_id) parent_run_logs = instance.all_logs(parent_run_id) # keep track of steps with dicts that point: # * step_key -> set(step_handle) in the normal case # * unresolved_step_key -> set(resolved_step_handle, ...) for dynamic outputs all_steps_in_parent_run_logs = defaultdict(set) failed_steps_in_parent_run_logs = defaultdict(set) successful_steps_in_parent_run_logs = defaultdict(set) interrupted_steps_in_parent_run_logs = defaultdict(set) skipped_steps_in_parent_run_logs = defaultdict(set) for record in parent_run_logs: if record.dagster_event and record.dagster_event.step_handle: step_handle = record.dagster_event.step_handle _update_tracking_dict(all_steps_in_parent_run_logs, step_handle) if record.dagster_event_type == DagsterEventType.STEP_FAILURE: _update_tracking_dict(failed_steps_in_parent_run_logs, step_handle) if record.dagster_event_type == DagsterEventType.STEP_SUCCESS: _update_tracking_dict(successful_steps_in_parent_run_logs, step_handle) if record.dagster_event_type == DagsterEventType.STEP_SKIPPED: _update_tracking_dict(skipped_steps_in_parent_run_logs, step_handle) for step_set in all_steps_in_parent_run_logs.values(): for step_handle in step_set: if ( not _in_tracking_dict(step_handle, failed_steps_in_parent_run_logs) and not _in_tracking_dict(step_handle, successful_steps_in_parent_run_logs) and not _in_tracking_dict(step_handle, skipped_steps_in_parent_run_logs) ): _update_tracking_dict(interrupted_steps_in_parent_run_logs, step_handle) to_retry = defaultdict(set) execution_deps = execution_plan.execution_deps() for step_snap in execution_plan.topological_steps(): step_key = step_snap.key step_handle = StepHandle.parse_from_key(step_snap.key) if parent_run.step_keys_to_execute and step_snap.key not in parent_run.step_keys_to_execute: continue if step_snap.key in failed_steps_in_parent_run_logs: to_retry[step_key].update(failed_steps_in_parent_run_logs[step_key]) # Interrupted steps can occur when graceful cleanup from a step failure fails to run, # and a step failure event is not generated if step_key in interrupted_steps_in_parent_run_logs: to_retry[step_key].update(interrupted_steps_in_parent_run_logs[step_key]) # Missing steps did not execute, e.g. when a run was terminated if step_key not in all_steps_in_parent_run_logs: to_retry[step_key].add(step_handle) step_dep_keys = execution_deps[step_key] retrying_dep_keys = step_dep_keys.intersection(to_retry.keys()) # this step is downstream of a step we are about to retry if retrying_dep_keys: for retrying_key in retrying_dep_keys: # If this step and its ancestor are both downstream of a dynamic output, # add resolved instances of this step for the retrying mapping keys if isinstance(step_handle, UnresolvedStepHandle) and all( map( lambda handle: isinstance(handle, ResolvedFromDynamicStepHandle), to_retry[retrying_key], ) ): for resolved_handle in to_retry[retrying_key]: to_retry[step_key].add(step_handle.resolve(resolved_handle.mapping_key)) else: to_retry[step_key].add(step_handle) return [step_handle.to_key() for step_set in to_retry.values() for step_handle in step_set]
def get_retry_steps_from_parent_run( instance, parent_run_id: str = None, parent_run: PipelineRun = None ) -> Tuple[List[str], Optional[KnownExecutionState]]: check.inst_param(instance, "instance", DagsterInstance) check.invariant( bool(parent_run_id) != bool(parent_run), "Must provide one of parent_run_id or parent_run") check.opt_str_param(parent_run_id, "parent_run_id") check.opt_inst_param(parent_run, "parent_run", PipelineRun) parent_run = parent_run or instance.get_run_by_id(parent_run_id) parent_run_id = parent_run.run_id parent_run_logs = instance.all_logs(parent_run_id) execution_plan_snapshot = instance.get_execution_plan_snapshot( parent_run.execution_plan_snapshot_id) if not execution_plan_snapshot: raise DagsterExecutionPlanSnapshotNotFoundError( f"Could not load execution plan snapshot for run {parent_run_id}") execution_plan = ExternalExecutionPlan( execution_plan_snapshot=execution_plan_snapshot) # keep track of steps with dicts that point: # * step_key -> set(step_handle) in the normal case # * unresolved_step_key -> set(resolved_step_handle, ...) for dynamic outputs all_steps_in_parent_run_logs: Dict[str, set] = defaultdict(set) failed_steps_in_parent_run_logs: Dict[str, set] = defaultdict(set) successful_steps_in_parent_run_logs: Dict[str, set] = defaultdict(set) interrupted_steps_in_parent_run_logs: Dict[str, set] = defaultdict(set) skipped_steps_in_parent_run_logs: Dict[str, set] = defaultdict(set) for record in parent_run_logs: if record.dagster_event and record.dagster_event.step_handle: step_handle = record.dagster_event.step_handle _update_tracking_dict(all_steps_in_parent_run_logs, step_handle) if record.dagster_event_type == DagsterEventType.STEP_FAILURE: _update_tracking_dict(failed_steps_in_parent_run_logs, step_handle) if record.dagster_event_type == DagsterEventType.STEP_SUCCESS: _update_tracking_dict(successful_steps_in_parent_run_logs, step_handle) if record.dagster_event_type == DagsterEventType.STEP_SKIPPED: _update_tracking_dict(skipped_steps_in_parent_run_logs, step_handle) for step_set in all_steps_in_parent_run_logs.values(): for step_handle in step_set: if (not _in_tracking_dict(step_handle, failed_steps_in_parent_run_logs) and not _in_tracking_dict( step_handle, successful_steps_in_parent_run_logs) and not _in_tracking_dict( step_handle, skipped_steps_in_parent_run_logs)): _update_tracking_dict(interrupted_steps_in_parent_run_logs, step_handle) to_retry = defaultdict(set) execution_deps = execution_plan.execution_deps() for step_snap in execution_plan.topological_steps(): step_key = step_snap.key step_handle = StepHandle.parse_from_key(step_snap.key) if parent_run.step_keys_to_execute and step_snap.key not in parent_run.step_keys_to_execute: continue if step_snap.key in failed_steps_in_parent_run_logs: to_retry[step_key].update( failed_steps_in_parent_run_logs[step_key]) # Interrupted steps can occur when graceful cleanup from a step failure fails to run, # and a step failure event is not generated if step_key in interrupted_steps_in_parent_run_logs: to_retry[step_key].update( interrupted_steps_in_parent_run_logs[step_key]) # Missing steps did not execute, e.g. when a run was terminated if step_key not in all_steps_in_parent_run_logs: to_retry[step_key].add(step_handle) step_dep_keys = execution_deps[step_key] retrying_dep_keys = step_dep_keys.intersection(to_retry.keys()) # this step is downstream of a step we are about to retry if retrying_dep_keys: for retrying_key in retrying_dep_keys: # If this step and its ancestor are both downstream of a dynamic output, # add resolved instances of this step for the retrying mapping keys if isinstance(step_handle, UnresolvedStepHandle) and all( map( lambda handle: isinstance( handle, ResolvedFromDynamicStepHandle), to_retry[retrying_key], )): for resolved_handle in to_retry[retrying_key]: to_retry[step_key].add( step_handle.resolve(resolved_handle.mapping_key)) else: to_retry[step_key].add(step_handle) steps_to_retry = [ step_handle.to_key() for step_set in to_retry.values() for step_handle in step_set ] return steps_to_retry, KnownExecutionState.for_reexecution( parent_run_logs, steps_to_retry)