def execute_step_out_of_process(step_context, step, errors, term_events): command = InProcessExecutorChildProcessCommand( step_context.run_config, step_context.pipeline_run, step_context.executor_config, step.key, step_context.instance.get_ref(), term_events[step.key], ) yield DagsterEvent.engine_event( step_context, 'Launching subprocess for {}'.format(step.key), EngineEventData(marker_start=DELEGATE_MARKER), step_key=step.key, ) for ret in execute_child_process_command(command): if ret is None or isinstance(ret, DagsterEvent): yield ret elif isinstance(ret, ChildProcessEvent): if isinstance(ret, ChildProcessSystemErrorEvent): errors[ret.pid] = ret.error_info elif isinstance(ret, KeyboardInterrupt): yield DagsterEvent.engine_event( step_context, 'Multiprocess engine: received KeyboardInterrupt - forwarding to active child processes', EngineEventData.interrupted(list(term_events.keys())), ) for term_event in term_events.values(): term_event.set() else: check.failed( 'Unexpected return value from child process {}'.format( type(ret)))
def bounded_parallel_executor(pipeline_context, step_contexts, limit): pending_execution = list(step_contexts) active_iters = {} errors = {} term_events = {} stopping = False while (not stopping and pending_execution) or active_iters: try: while len(active_iters ) < limit and pending_execution and not stopping: step_context = pending_execution.pop(0) step = step_context.step term_events[step.key] = get_multiprocessing_context().Event() active_iters[step.key] = execute_step_out_of_process( step_context, step, errors, term_events) empty_iters = [] for key, step_iter in active_iters.items(): try: event_or_none = next(step_iter) if event_or_none is None: continue else: yield event_or_none except StopIteration: empty_iters.append(key) for key in empty_iters: del active_iters[key] if term_events[key].is_set(): stopping = True del term_events[key] # In the very small chance that we get interrupted in this coordination section and not # polling the subprocesses for events - try to clean up greacefully except KeyboardInterrupt: yield DagsterEvent.engine_event( pipeline_context, 'Multiprocess engine: received KeyboardInterrupt - forwarding to active child processes', EngineEventData.interrupted(list(term_events.keys())), ) for event in term_events.values(): event.set() errs = {pid: err for pid, err in errors.items() if err} if errs: raise DagsterSubprocessError( 'During multiprocess execution errors occured in child processes:\n{error_list}' .format(error_list='\n'.join([ 'In process {pid}: {err}'.format(pid=pid, err=err.to_string()) for pid, err in errs.items() ])), subprocess_error_infos=list(errs.values()), )
def execute(self, pipeline_context, execution_plan): check.inst_param(pipeline_context, "pipeline_context", SystemPipelineExecutionContext) check.inst_param(execution_plan, "execution_plan", ExecutionPlan) limit = self.max_concurrent yield DagsterEvent.engine_event( pipeline_context, "Executing steps using multiprocess engine: parent process (pid: {pid})" .format(pid=os.getpid()), event_specific_data=EngineEventData.multiprocess( os.getpid(), step_keys_to_execute=execution_plan.step_keys_to_execute), ) # It would be good to implement a reference tracking algorithm here so we could # garbage collection results that are no longer needed by any steps # https://github.com/dagster-io/dagster/issues/811 with time_execution_scope() as timer_result: with execution_plan.start( retries=self.retries) as active_execution: active_iters = {} errors = {} term_events = {} stopping = False while (not stopping and not active_execution.is_complete) or active_iters: try: # start iterators while len(active_iters) < limit and not stopping: steps = active_execution.get_steps_to_execute( limit=(limit - len(active_iters))) if not steps: break for step in steps: step_context = pipeline_context.for_step(step) term_events[step.key] = multiprocessing.Event() active_iters[ step. key] = self.execute_step_out_of_process( step_context, step, errors, term_events) # process active iterators empty_iters = [] for key, step_iter in active_iters.items(): try: event_or_none = next(step_iter) if event_or_none is None: continue else: yield event_or_none active_execution.handle_event( event_or_none) except ChildProcessCrashException as crash: serializable_error = serializable_error_info_from_exc_info( sys.exc_info()) yield DagsterEvent.engine_event( pipeline_context, ("Multiprocess executor: child process for step {step_key} " "unexpectedly exited with code {exit_code}" ).format(step_key=key, exit_code=crash.exit_code), EngineEventData.engine_error( serializable_error), step_key=key, ) step_failure_event = DagsterEvent.step_failure_event( step_context=pipeline_context.for_step( active_execution.get_step_by_key(key)), step_failure_data=StepFailureData( error=serializable_error, user_failure_data=None), ) active_execution.handle_event( step_failure_event) yield step_failure_event empty_iters.append(key) except StopIteration: empty_iters.append(key) # clear and mark complete finished iterators for key in empty_iters: del active_iters[key] if term_events[key].is_set(): stopping = True del term_events[key] active_execution.verify_complete( pipeline_context, key) # process skips from failures or uncovered inputs for event in active_execution.skipped_step_events_iterator( pipeline_context): yield event # In the very small chance that we get interrupted in this coordination section and not # polling the subprocesses for events - try to clean up gracefully except KeyboardInterrupt: yield DagsterEvent.engine_event( pipeline_context, "Multiprocess engine: received KeyboardInterrupt - forwarding to active child processes", EngineEventData.interrupted( list(term_events.keys())), ) stopping = True for event in term_events.values(): event.set() errs = {pid: err for pid, err in errors.items() if err} if errs: raise DagsterSubprocessError( "During multiprocess execution errors occurred in child processes:\n{error_list}" .format(error_list="\n".join([ "In process {pid}: {err}".format( pid=pid, err=err.to_string()) for pid, err in errs.items() ])), subprocess_error_infos=list(errs.values()), ) yield DagsterEvent.engine_event( pipeline_context, "Multiprocess engine: parent process exiting after {duration} (pid: {pid})" .format(duration=format_duration(timer_result.millis), pid=os.getpid()), event_specific_data=EngineEventData.multiprocess(os.getpid()), )
def execute(pipeline_context, execution_plan): check.inst_param(pipeline_context, 'pipeline_context', SystemPipelineExecutionContext) check.inst_param(execution_plan, 'execution_plan', ExecutionPlan) intermediates_manager = pipeline_context.intermediates_manager limit = pipeline_context.executor_config.max_concurrent yield DagsterEvent.engine_event( pipeline_context, 'Executing steps using multiprocess engine: parent process (pid: {pid})' .format(pid=os.getpid()), event_specific_data=EngineEventData.multiprocess( os.getpid(), step_keys_to_execute=execution_plan.step_keys_to_execute), ) # It would be good to implement a reference tracking algorithm here so we could # garbage collection results that are no longer needed by any steps # https://github.com/dagster-io/dagster/issues/811 with time_execution_scope() as timer_result: active_execution = execution_plan.start( retries=pipeline_context.executor_config.retries) active_iters = {} errors = {} term_events = {} stopping = False while (not stopping and not active_execution.is_complete) or active_iters: try: # start iterators while len(active_iters) < limit and not stopping: steps = active_execution.get_steps_to_execute( limit=(limit - len(active_iters))) if not steps: break for step in steps: step_context = pipeline_context.for_step(step) term_events[ step.key] = get_multiprocessing_context( ).Event() active_iters[ step.key] = execute_step_out_of_process( step_context, step, errors, term_events) # process active iterators empty_iters = [] for key, step_iter in active_iters.items(): try: event_or_none = next(step_iter) if event_or_none is None: continue else: yield event_or_none active_execution.handle_event(event_or_none) except StopIteration: empty_iters.append(key) # clear and mark complete finished iterators for key in empty_iters: del active_iters[key] if term_events[key].is_set(): stopping = True del term_events[key] active_execution.verify_complete(pipeline_context, key) # process skips from failures or uncovered inputs for event in active_execution.skipped_step_events_iterator( pipeline_context): yield event # In the very small chance that we get interrupted in this coordination section and not # polling the subprocesses for events - try to clean up gracefully except KeyboardInterrupt: yield DagsterEvent.engine_event( pipeline_context, 'Multiprocess engine: received KeyboardInterrupt - forwarding to active child processes', EngineEventData.interrupted(list(term_events.keys())), ) stopping = True for event in term_events.values(): event.set() errs = {pid: err for pid, err in errors.items() if err} if errs: raise DagsterSubprocessError( 'During multiprocess execution errors occurred in child processes:\n{error_list}' .format(error_list='\n'.join([ 'In process {pid}: {err}'.format(pid=pid, err=err.to_string()) for pid, err in errs.items() ])), subprocess_error_infos=list(errs.values()), ) yield DagsterEvent.engine_event( pipeline_context, 'Multiprocess engine: parent process exiting after {duration} (pid: {pid})' .format(duration=format_duration(timer_result.millis), pid=os.getpid()), event_specific_data=EngineEventData.multiprocess(os.getpid()), )
def execute(self, pipeline_context, execution_plan): check.inst_param(pipeline_context, "pipeline_context", SystemPipelineExecutionContext) check.inst_param(execution_plan, "execution_plan", ExecutionPlan) limit = self.max_concurrent yield DagsterEvent.engine_event( pipeline_context, "Executing steps using multiprocess executor: parent process (pid: {pid})" .format(pid=os.getpid()), event_specific_data=EngineEventData.multiprocess( os.getpid(), step_keys_to_execute=execution_plan.step_keys_to_execute), ) # It would be good to implement a reference tracking algorithm here so we could # garbage collect results that are no longer needed by any steps # https://github.com/dagster-io/dagster/issues/811 with time_execution_scope() as timer_result: with execution_plan.start( retry_mode=self.retries) as active_execution: active_iters = {} errors = {} term_events = {} stopping = False while (not stopping and not active_execution.is_complete) or active_iters: if active_execution.check_for_interrupts(): yield DagsterEvent.engine_event( pipeline_context, "Multiprocess executor: received termination signal - " "forwarding to active child processes", EngineEventData.interrupted( list(term_events.keys())), ) stopping = True active_execution.mark_interrupted() for key, event in term_events.items(): event.set() # start iterators while len(active_iters) < limit and not stopping: steps = active_execution.get_steps_to_execute( limit=(limit - len(active_iters))) if not steps: break for step in steps: step_context = pipeline_context.for_step(step) term_events[step.key] = multiprocessing.Event() active_iters[ step.key] = self.execute_step_out_of_process( step_context, step, errors, term_events, active_execution.get_known_state(), ) # process active iterators empty_iters = [] for key, step_iter in active_iters.items(): try: event_or_none = next(step_iter) if event_or_none is None: continue else: yield event_or_none active_execution.handle_event(event_or_none) except ChildProcessCrashException as crash: serializable_error = serializable_error_info_from_exc_info( sys.exc_info()) yield DagsterEvent.engine_event( pipeline_context, ("Multiprocess executor: child process for step {step_key} " "unexpectedly exited with code {exit_code}" ).format(step_key=key, exit_code=crash.exit_code), EngineEventData.engine_error( serializable_error), step_handle=active_execution.get_step_by_key( key).handle, ) step_failure_event = DagsterEvent.step_failure_event( step_context=pipeline_context.for_step( active_execution.get_step_by_key(key)), step_failure_data=StepFailureData( error=serializable_error, user_failure_data=None), ) active_execution.handle_event(step_failure_event) yield step_failure_event empty_iters.append(key) except StopIteration: empty_iters.append(key) # clear and mark complete finished iterators for key in empty_iters: del active_iters[key] del term_events[key] active_execution.verify_complete(pipeline_context, key) # process skipped and abandoned steps yield from active_execution.plan_events_iterator( pipeline_context) errs = {pid: err for pid, err in errors.items() if err} # After termination starts, raise an interrupted exception once all subprocesses # have finished cleaning up (and the only errors were from being interrupted) if (stopping and (not active_iters) and all([ err_info.cls_name == "DagsterExecutionInterruptedError" for err_info in errs.values() ])): yield DagsterEvent.engine_event( pipeline_context, "Multiprocess executor: interrupted all active child processes", event_specific_data=EngineEventData(), ) raise DagsterExecutionInterruptedError() elif errs: raise DagsterSubprocessError( "During multiprocess execution errors occurred in child processes:\n{error_list}" .format(error_list="\n".join([ "In process {pid}: {err}".format( pid=pid, err=err.to_string()) for pid, err in errs.items() ])), subprocess_error_infos=list(errs.values()), ) yield DagsterEvent.engine_event( pipeline_context, "Multiprocess executor: parent process exiting after {duration} (pid: {pid})" .format(duration=format_duration(timer_result.millis), pid=os.getpid()), event_specific_data=EngineEventData.multiprocess(os.getpid()), )
def core_celery_execution_loop(pipeline_context, execution_plan, step_execution_fn): check.inst_param(pipeline_context, "pipeline_context", SystemPipelineExecutionContext) check.inst_param(execution_plan, "execution_plan", ExecutionPlan) check.callable_param(step_execution_fn, "step_execution_fn") executor = pipeline_context.executor # https://github.com/dagster-io/dagster/issues/2440 check.invariant( execution_plan.artifacts_persisted, "Cannot use in-memory storage with Celery, use filesystem (on top of NFS or " "similar system that allows files to be available to all nodes), S3, or GCS", ) app = make_app(executor.app_args()) priority_for_step = lambda step: (-1 * int( step.tags.get(DAGSTER_CELERY_STEP_PRIORITY_TAG, task_default_priority) ) + -1 * _get_run_priority(pipeline_context)) priority_for_key = lambda step_key: (priority_for_step( execution_plan.get_step_by_key(step_key))) _warn_on_priority_misuse(pipeline_context, execution_plan) step_results = {} # Dict[ExecutionStep, celery.AsyncResult] step_errors = {} with execution_plan.start( retries=pipeline_context.executor.retries, sort_key_fn=priority_for_step, ) as active_execution: stopping = False while (not active_execution.is_complete and not stopping) or step_results: if active_execution.check_for_interrupts(): yield DagsterEvent.engine_event( pipeline_context, "Celery executor: received termination signal - revoking active tasks from workers", EngineEventData.interrupted(list(step_results.keys())), ) stopping = True active_execution.mark_interrupted() for result in step_results.values(): result.revoke() results_to_pop = [] for step_key, result in sorted( step_results.items(), key=lambda x: priority_for_key(x[0])): if result.ready(): try: step_events = result.get() except TaskRevokedError: step_events = [] yield DagsterEvent.engine_event( pipeline_context, 'celery task for running step "{step_key}" was revoked.' .format(step_key=step_key, ), EngineEventData(marker_end=DELEGATE_MARKER), step_handle=active_execution.get_step_by_key( step_key).handle, ) except Exception: # pylint: disable=broad-except # We will want to do more to handle the exception here.. maybe subclass Task # Certainly yield an engine or pipeline event step_events = [] step_errors[ step_key] = serializable_error_info_from_exc_info( sys.exc_info()) for step_event in step_events: event = deserialize_json_to_dagster_namedtuple( step_event) yield event active_execution.handle_event(event) results_to_pop.append(step_key) for step_key in results_to_pop: if step_key in step_results: del step_results[step_key] active_execution.verify_complete(pipeline_context, step_key) # process skips from failures or uncovered inputs for event in active_execution.plan_events_iterator( pipeline_context): yield event # don't add any new steps if we are stopping if stopping or step_errors: continue # This is a slight refinement. If we have n workers idle and schedule m > n steps for # execution, the first n steps will be picked up by the idle workers in the order in # which they are scheduled (and the following m-n steps will be executed in priority # order, provided that it takes longer to execute a step than to schedule it). The test # case has m >> n to exhibit this behavior in the absence of this sort step. for step in active_execution.get_steps_to_execute(): try: queue = step.tags.get(DAGSTER_CELERY_QUEUE_TAG, task_default_queue) yield DagsterEvent.engine_event( pipeline_context, 'Submitting celery task for step "{step_key}" to queue "{queue}".' .format(step_key=step.key, queue=queue), EngineEventData(marker_start=DELEGATE_MARKER), step_handle=step.handle, ) # Get the Celery priority for this step priority = _get_step_priority(pipeline_context, step) # Submit the Celery tasks step_results[step.key] = step_execution_fn( app, pipeline_context, step, queue, priority) except Exception: yield DagsterEvent.engine_event( pipeline_context, "Encountered error during celery task submission.". format(), event_specific_data=EngineEventData.engine_error( serializable_error_info_from_exc_info( sys.exc_info()), ), ) raise time.sleep(TICK_SECONDS) if step_errors: raise DagsterSubprocessError( "During celery execution errors occurred in workers:\n{error_list}" .format(error_list="\n".join([ "[{step}]: {err}".format(step=key, err=err.to_string()) for key, err in step_errors.items() ])), subprocess_error_infos=list(step_errors.values()), )
def execute(self, plan_context: PlanOrchestrationContext, execution_plan: ExecutionPlan): check.inst_param(plan_context, "plan_context", PlanOrchestrationContext) check.inst_param(execution_plan, "execution_plan", ExecutionPlan) self._event_cursor = -1 # pylint: disable=attribute-defined-outside-init yield DagsterEvent.engine_event( plan_context, f"Starting execution with step handler {self._step_handler.name}", EngineEventData(), ) with execution_plan.start(retry_mode=self.retries) as active_execution: running_steps: Dict[str, ExecutionStep] = {} if plan_context.resume_from_failure: yield DagsterEvent.engine_event( plan_context, "Resuming execution from failure", EngineEventData(), ) prior_events = self._pop_events( plan_context.instance, plan_context.run_id, ) for dagster_event in prior_events: yield dagster_event possibly_in_flight_steps = active_execution.rebuild_from_events( prior_events) for step in possibly_in_flight_steps: yield DagsterEvent.engine_event( plan_context, "Checking on status of possibly launched steps", EngineEventData(), step.handle, ) # TODO: check if failure event included. For now, hacky assumption that # we don't log anything on successful check if self._step_handler.check_step_health( self._get_step_handler_context( plan_context, [step], active_execution)): # health check failed, launch the step self._log_new_events( self._step_handler.launch_step( self._get_step_handler_context( plan_context, [step], active_execution)), plan_context, { step.key: step for step in possibly_in_flight_steps }, ) running_steps[step.key] = step last_check_step_health_time = pendulum.now("UTC") # Order of events is important here. During an interation, we call handle_event, then get_steps_to_execute, # then is_complete. get_steps_to_execute updates the state of ActiveExecution, and without it # is_complete can return true when we're just between steps. while not active_execution.is_complete: if active_execution.check_for_interrupts(): if not plan_context.instance.run_will_resume( plan_context.run_id): yield DagsterEvent.engine_event( plan_context, "Executor received termination signal, forwarding to steps", EngineEventData.interrupted( list(running_steps.keys())), ) active_execution.mark_interrupted() for _, step in running_steps.items(): self._log_new_events( self._step_handler.terminate_step( self._get_step_handler_context( plan_context, [step], active_execution)), plan_context, running_steps, ) else: yield DagsterEvent.engine_event( plan_context, "Executor received termination signal, not forwarding to steps because " "run will be resumed", EngineEventData(metadata_entries=[ EventMetadataEntry.text( str(running_steps.keys()), "steps_in_flight") ]), ) active_execution.mark_interrupted() return for dagster_event in self._pop_events( plan_context.instance, plan_context.run_id, ): # type: ignore # STEP_SKIPPED events are only emitted by ActiveExecution, which already handles # and yields them. if dagster_event.is_step_skipped: assert isinstance(dagster_event.step_key, str) active_execution.verify_complete( plan_context, dagster_event.step_key) else: yield dagster_event active_execution.handle_event(dagster_event) if dagster_event.is_step_success or dagster_event.is_step_failure: assert isinstance(dagster_event.step_key, str) del running_steps[dagster_event.step_key] active_execution.verify_complete( plan_context, dagster_event.step_key) # process skips from failures or uncovered inputs for event in active_execution.plan_events_iterator( plan_context): yield event curr_time = pendulum.now("UTC") if (curr_time - last_check_step_health_time).total_seconds( ) >= self._check_step_health_interval_seconds: last_check_step_health_time = curr_time for _, step in running_steps.items(): self._log_new_events( self._step_handler.check_step_health( self._get_step_handler_context( plan_context, [step], active_execution)), plan_context, running_steps, ) for step in active_execution.get_steps_to_execute(): running_steps[step.key] = step self._log_new_events( self._step_handler.launch_step( self._get_step_handler_context( plan_context, [step], active_execution)), plan_context, running_steps, ) time.sleep(self._sleep_seconds)
def execute(self, pipeline_context: PlanOrchestrationContext, execution_plan: ExecutionPlan): check.inst_param(pipeline_context, "pipeline_context", PlanOrchestrationContext) check.inst_param(execution_plan, "execution_plan", ExecutionPlan) self._event_cursor = -1 # pylint: disable=attribute-defined-outside-init yield DagsterEvent.engine_event( pipeline_context, f"Starting execution with step handler {self._step_handler.name}", EngineEventData(), ) with execution_plan.start(retry_mode=self.retries) as active_execution: stopping = False running_steps: Dict[str, ExecutionStep] = {} last_check_step_health_time = pendulum.now("UTC") while (not active_execution.is_complete and not stopping) or running_steps: events = [] if active_execution.check_for_interrupts(): yield DagsterEvent.engine_event( pipeline_context, "Executor received termination signal, forwarding to steps", EngineEventData.interrupted(list( running_steps.keys())), ) stopping = True active_execution.mark_interrupted() for _, step in running_steps.items(): events.extend( self._log_new_events( self._step_handler.terminate_step( self._get_step_handler_context( pipeline_context, [step], active_execution)), pipeline_context, running_steps, )) running_steps.clear() events.extend( self._pop_events( pipeline_context.plan_data.instance, pipeline_context.plan_data.pipeline_run.run_id, )) if not stopping: curr_time = pendulum.now("UTC") if (curr_time - last_check_step_health_time).total_seconds( ) >= self._check_step_health_interval_seconds: last_check_step_health_time = curr_time for _, step in running_steps.items(): events.extend( self._log_new_events( self._step_handler.check_step_health( self._get_step_handler_context( pipeline_context, [step], active_execution)), pipeline_context, running_steps, )) for step in active_execution.get_steps_to_execute(): running_steps[step.key] = step events.extend( self._log_new_events( self._step_handler.launch_step( self._get_step_handler_context( pipeline_context, [step], active_execution)), pipeline_context, running_steps, )) for dagster_event in events: yield dagster_event active_execution.handle_event(dagster_event) if (dagster_event.is_step_success or dagster_event.is_step_failure or dagster_event.is_step_skipped): assert isinstance(dagster_event.step_key, str) del running_steps[dagster_event.step_key] active_execution.verify_complete( pipeline_context, dagster_event.step_key) # process skips from failures or uncovered inputs for event in active_execution.plan_events_iterator( pipeline_context): yield event time.sleep(self._sleep_seconds)