def execute_run_host_mode( pipeline: ReconstructablePipeline, pipeline_run: PipelineRun, instance: DagsterInstance, executor_defs: Optional[List[ExecutorDefinition]] = None, raise_on_error: bool = False, ): check.inst_param(pipeline, "pipeline", ReconstructablePipeline) check.inst_param(pipeline_run, "pipeline_run", PipelineRun) check.inst_param(instance, "instance", DagsterInstance) check.opt_list_param(executor_defs, "executor_defs", of_type=ExecutorDefinition) executor_defs = executor_defs if executor_defs != None else default_executors if pipeline_run.status == PipelineRunStatus.CANCELED: message = "Not starting execution since the run was canceled before execution could start" instance.report_engine_event( message, pipeline_run, ) raise DagsterInvariantViolationError(message) check.invariant( pipeline_run.status == PipelineRunStatus.NOT_STARTED or pipeline_run.status == PipelineRunStatus.STARTING, desc="Pipeline run {} ({}) in state {}, expected NOT_STARTED or STARTING" .format(pipeline_run.pipeline_name, pipeline_run.run_id, pipeline_run.status), ) if pipeline_run.solids_to_execute: pipeline = pipeline.subset_for_execution_from_existing_pipeline( frozenset(pipeline_run.solids_to_execute)) execution_plan_snapshot = instance.get_execution_plan_snapshot( pipeline_run.execution_plan_snapshot_id) execution_plan = ExecutionPlan.rebuild_from_snapshot( pipeline_run.pipeline_name, execution_plan_snapshot, ) _execute_run_iterable = ExecuteRunWithPlanIterable( execution_plan=execution_plan, iterator=pipeline_execution_iterator, execution_context_manager=PlanOrchestrationContextManager( context_event_generator=host_mode_execution_context_event_generator, pipeline=pipeline, execution_plan=execution_plan, run_config=pipeline_run.run_config, pipeline_run=pipeline_run, instance=instance, raise_on_error=raise_on_error, executor_defs=executor_defs, output_capture=None, ), ) event_list = list(_execute_run_iterable) return event_list
def execute_run_host_mode( pipeline: ReconstructablePipeline, pipeline_run: PipelineRun, instance: DagsterInstance, get_executor_def_fn: Callable[[Optional[str]], ExecutorDefinition] = None, raise_on_error: bool = False, ): check.inst_param(pipeline, "pipeline", ReconstructablePipeline) check.inst_param(pipeline_run, "pipeline_run", PipelineRun) check.inst_param(instance, "instance", DagsterInstance) check.opt_callable_param(get_executor_def_fn, "get_executor_def_fn") if pipeline_run.status == PipelineRunStatus.CANCELED: message = "Not starting execution since the run was canceled before execution could start" instance.report_engine_event( message, pipeline_run, ) raise DagsterInvariantViolationError(message) check.invariant( pipeline_run.status == PipelineRunStatus.NOT_STARTED or pipeline_run.status == PipelineRunStatus.STARTING, desc="Pipeline run {} ({}) in state {}, expected NOT_STARTED or STARTING" .format(pipeline_run.pipeline_name, pipeline_run.run_id, pipeline_run.status), ) if pipeline_run.solids_to_execute: pipeline = pipeline.subset_for_execution_from_existing_pipeline( pipeline_run.solids_to_execute) execution_plan_snapshot = instance.get_execution_plan_snapshot( pipeline_run.execution_plan_snapshot_id) execution_plan = ExecutionPlan.rebuild_from_snapshot( pipeline_run.pipeline_name, execution_plan_snapshot, ) _execute_run_iterable = ExecuteRunWithPlanIterable( execution_plan=execution_plan, iterator=pipeline_execution_iterator, execution_context_manager=HostModeRunWorkerExecutionContextManager( execution_plan=execution_plan, recon_pipeline=pipeline, pipeline_run=pipeline_run, instance=instance, run_config=pipeline_run.run_config, get_executor_def_fn=get_executor_def_fn, raise_on_error=raise_on_error, ), ) event_list = list(_execute_run_iterable) return event_list
def _get_execution_plan_from_run(pipeline: IPipeline, pipeline_run: PipelineRun, instance: DagsterInstance) -> ExecutionPlan: if pipeline_run.execution_plan_snapshot_id: execution_plan_snapshot = instance.get_execution_plan_snapshot( pipeline_run.execution_plan_snapshot_id) if execution_plan_snapshot.can_reconstruct_plan: return ExecutionPlan.rebuild_from_snapshot( pipeline_run.pipeline_name, execution_plan_snapshot, ) return create_execution_plan( pipeline, run_config=pipeline_run.run_config, mode=pipeline_run.mode, step_keys_to_execute=pipeline_run.step_keys_to_execute, )
def _get_execution_plan_from_run(pipeline: IPipeline, pipeline_run: PipelineRun, instance: DagsterInstance) -> ExecutionPlan: if ( # need to rebuild execution plan so it matches the subsetted graph pipeline.solids_to_execute is None and pipeline_run.execution_plan_snapshot_id): execution_plan_snapshot = instance.get_execution_plan_snapshot( pipeline_run.execution_plan_snapshot_id) if execution_plan_snapshot.can_reconstruct_plan: return ExecutionPlan.rebuild_from_snapshot( pipeline_run.pipeline_name, execution_plan_snapshot, ) return create_execution_plan( pipeline, run_config=pipeline_run.run_config, mode=pipeline_run.mode, step_keys_to_execute=pipeline_run.step_keys_to_execute, instance_ref=instance.get_ref() if instance.is_persistent else None, )
def execute_run_iterator(pipeline: IPipeline, pipeline_run: PipelineRun, instance: DagsterInstance) -> Iterator[DagsterEvent]: check.inst_param(pipeline, "pipeline", IPipeline) check.inst_param(pipeline_run, "pipeline_run", PipelineRun) check.inst_param(instance, "instance", DagsterInstance) if pipeline_run.status == PipelineRunStatus.CANCELED: # This can happen if the run was force-terminated while it was starting def gen_execute_on_cancel(): yield instance.report_engine_event( "Not starting execution since the run was canceled before execution could start", pipeline_run, ) return gen_execute_on_cancel() check.invariant( pipeline_run.status == PipelineRunStatus.NOT_STARTED or pipeline_run.status == PipelineRunStatus.STARTING, desc="Pipeline run {} ({}) in state {}, expected NOT_STARTED or STARTING" .format(pipeline_run.pipeline_name, pipeline_run.run_id, pipeline_run.status), ) if pipeline_run.solids_to_execute: pipeline_def = pipeline.get_definition() if isinstance(pipeline_def, PipelineSubsetDefinition): check.invariant( pipeline_run.solids_to_execute == pipeline.solids_to_execute, "Cannot execute PipelineRun with solids_to_execute {solids_to_execute} that conflicts " "with pipeline subset {pipeline_solids_to_execute}.".format( pipeline_solids_to_execute=str_format_set( pipeline.solids_to_execute), solids_to_execute=str_format_set( pipeline_run.solids_to_execute), ), ) else: # when `execute_run_iterator` is directly called, the sub pipeline hasn't been created # note that when we receive the solids to execute via PipelineRun, it won't support # solid selection query syntax pipeline = pipeline.subset_for_execution_from_existing_pipeline( pipeline_run.solids_to_execute) known_state = None if pipeline_run.parent_run_id and pipeline_run.step_keys_to_execute: plan_snap = instance.get_execution_plan_snapshot( pipeline_run.execution_plan_snapshot_id) known_state = plan_snap.initial_known_state execution_plan = create_execution_plan( pipeline, run_config=pipeline_run.run_config, mode=pipeline_run.mode, step_keys_to_execute=pipeline_run.step_keys_to_execute, known_state=known_state, ) return iter( ExecuteRunWithPlanIterable( execution_plan=execution_plan, iterator=pipeline_execution_iterator, execution_context_manager=PipelineExecutionContextManager( execution_plan=execution_plan, pipeline_run=pipeline_run, instance=instance, run_config=pipeline_run.run_config, raise_on_error=False, ), ))
def execute_run( pipeline: IPipeline, pipeline_run: PipelineRun, instance: DagsterInstance, raise_on_error: bool = False, ) -> PipelineExecutionResult: """Executes an existing pipeline run synchronously. Synchronous version of execute_run_iterator. Args: pipeline (IPipeline): The pipeline to execute. pipeline_run (PipelineRun): The run to execute instance (DagsterInstance): The instance in which the run has been created. raise_on_error (Optional[bool]): Whether or not to raise exceptions when they occur. Defaults to ``False``. Returns: PipelineExecutionResult: The result of the execution. """ if isinstance(pipeline, PipelineDefinition): raise DagsterInvariantViolationError( "execute_run requires an IPipeline but received a PipelineDefinition " "directly instead. To support hand-off to other processes provide a " "ReconstructablePipeline which can be done using reconstructable(). For in " "process only execution you can use InMemoryPipeline.") check.inst_param(pipeline, "pipeline", IPipeline) check.inst_param(pipeline_run, "pipeline_run", PipelineRun) check.inst_param(instance, "instance", DagsterInstance) if pipeline_run.status == PipelineRunStatus.CANCELED: message = "Not starting execution since the run was canceled before execution could start" instance.report_engine_event( message, pipeline_run, ) raise DagsterInvariantViolationError(message) check.invariant( pipeline_run.status == PipelineRunStatus.NOT_STARTED or pipeline_run.status == PipelineRunStatus.STARTING, desc="Pipeline run {} ({}) in state {}, expected NOT_STARTED or STARTING" .format(pipeline_run.pipeline_name, pipeline_run.run_id, pipeline_run.status), ) pipeline_def = pipeline.get_definition() if pipeline_run.solids_to_execute: if isinstance(pipeline_def, PipelineSubsetDefinition): check.invariant( pipeline_run.solids_to_execute == pipeline.solids_to_execute, "Cannot execute PipelineRun with solids_to_execute {solids_to_execute} that " "conflicts with pipeline subset {pipeline_solids_to_execute}.". format( pipeline_solids_to_execute=str_format_set( pipeline.solids_to_execute), solids_to_execute=str_format_set( pipeline_run.solids_to_execute), ), ) else: # when `execute_run` is directly called, the sub pipeline hasn't been created # note that when we receive the solids to execute via PipelineRun, it won't support # solid selection query syntax pipeline = pipeline.subset_for_execution_from_existing_pipeline( pipeline_run.solids_to_execute) known_state = None if pipeline_run.parent_run_id and pipeline_run.step_keys_to_execute: plan_snap = instance.get_execution_plan_snapshot( pipeline_run.execution_plan_snapshot_id) known_state = plan_snap.initial_known_state execution_plan = create_execution_plan( pipeline, run_config=pipeline_run.run_config, mode=pipeline_run.mode, step_keys_to_execute=pipeline_run.step_keys_to_execute, known_state=known_state, ) if is_memoized_run(pipeline_run.tags): execution_plan = resolve_memoized_execution_plan( execution_plan, pipeline_run.run_config, instance) _execute_run_iterable = ExecuteRunWithPlanIterable( execution_plan=execution_plan, iterator=pipeline_execution_iterator, execution_context_manager=PipelineExecutionContextManager( execution_plan=execution_plan, pipeline_run=pipeline_run, instance=instance, run_config=pipeline_run.run_config, raise_on_error=raise_on_error, ), ) event_list = list(_execute_run_iterable) pipeline_context = _execute_run_iterable.pipeline_context # workaround for mem_io_manager to work in reconstruct_context, e.g. result.result_for_solid # in-memory values dict will get lost when the resource is re-initiated in reconstruct_context # so instead of re-initiating every single resource, we pass the resource instances to # reconstruct_context directly to avoid re-building from resource def. resource_instances_to_override = {} if pipeline_context: # None if we have a pipeline failure for ( key, resource_instance, ) in pipeline_context.scoped_resources_builder.resource_instance_dict.items( ): if isinstance(resource_instance, InMemoryIOManager): resource_instances_to_override[key] = resource_instance return PipelineExecutionResult( pipeline.get_definition(), pipeline_run.run_id, event_list, lambda hardcoded_resources_arg: scoped_pipeline_context( execution_plan, pipeline_run.run_config, pipeline_run, instance, intermediate_storage=pipeline_context.intermediate_storage, resource_instances_to_override=hardcoded_resources_arg, ), resource_instances_to_override=resource_instances_to_override, )