def launch_step( self, step_context: StepExecutionContext, prior_attempts_count: int, ) -> Iterator[DagsterEvent]: step_run_ref = step_context_to_step_run_ref(step_context, prior_attempts_count) run_id = step_context.pipeline_run.run_id step_run_dir = os.path.join(self.scratch_dir, run_id, step_run_ref.step_key) os.makedirs(step_run_dir) step_run_ref_file_path = os.path.join(step_run_dir, PICKLED_STEP_RUN_REF_FILE_NAME) with open(step_run_ref_file_path, "wb") as step_pickle_file: pickle.dump(step_run_ref, step_pickle_file) command_tokens = [ sys.executable, "-m", "dagster.core.execution.plan.local_external_step_main", step_run_ref_file_path, ] # If this is being called within a `capture_interrupts` context, allow interrupts # while waiting for the subprocess to complete, so that we can terminate slow or # hanging steps with raise_execution_interrupts(): subprocess.call(command_tokens, stdout=sys.stdout, stderr=sys.stderr) events_file_path = os.path.join(step_run_dir, PICKLED_EVENTS_FILE_NAME) file_manager = LocalFileManager(".") events_file_handle = LocalFileHandle(events_file_path) events_data = file_manager.read_data(events_file_handle) events = pickle.loads(events_data) yield from events
def launch_step(self, step_context, prior_attempts_count): step_run_ref = step_context_to_step_run_ref( step_context, prior_attempts_count, self.local_dagster_job_package_path) run_id = step_context.pipeline_run.run_id log = step_context.log step_key = step_run_ref.step_key self._upload_artifacts(log, step_run_ref, run_id, step_key) task = self._get_databricks_task(run_id, step_key) databricks_run_id = self.databricks_runner.submit_run( self.run_config, task) try: # If this is being called within a `capture_interrupts` context, allow interrupts while # waiting for the execution to complete, so that we can terminate slow or hanging steps with raise_execution_interrupts(): yield from self.step_events_iterator(step_context, step_key, databricks_run_id) finally: self.log_compute_logs(log, run_id, step_key) # this is somewhat obsolete if self.wait_for_logs: self._log_logs_from_cluster(log, databricks_run_id)
def wait_for_completion(self, log, s3, run_id, step_key, emr_step_id, check_interval=15): """We want to wait for the EMR steps to complete, and while that's happening, we want to yield any events that have been written to S3 for us by the remote process. After the the EMR steps complete, we want a final chance to fetch events before finishing the step. """ done = False all_events = [] # If this is being called within a `capture_interrupts` context, allow interrupts # while waiting for the pyspark execution to complete, so that we can terminate slow or # hanging steps while not done: with raise_execution_interrupts(): time.sleep(check_interval ) # AWS rate-limits us if we poll it too often done = self.emr_job_runner.is_emr_step_complete( log, self.cluster_id, emr_step_id) all_events_new = self.read_events(s3, run_id, step_key) if len(all_events_new) > len(all_events): for i in range(len(all_events), len(all_events_new)): yield all_events_new[i] all_events = all_events_new
def test_raise_execution_interrupts(): with raise_execution_interrupts(): try: _send_interrupt_to_self() except DagsterExecutionInterruptedError: standard_interrupt = True assert standard_interrupt
def test_calling_raise_execution_interrupts_also_raises_any_captured_interrupts(): interrupt_from_raise_execution_interrupts = False interrupt_after_delay = False try: with capture_interrupts(): _send_interrupt_to_self() try: with raise_execution_interrupts(): pass except DagsterExecutionInterruptedError: interrupt_from_raise_execution_interrupts = True except: interrupt_after_delay = True assert interrupt_from_raise_execution_interrupts assert not interrupt_after_delay
def test_interrupt_inside_nested_delay_and_raise(): interrupt_inside_nested_raise = False interrupt_after_delay = False try: with capture_interrupts(): with raise_execution_interrupts(): try: _send_interrupt_to_self() except DagsterExecutionInterruptedError: interrupt_inside_nested_raise = True except: # pylint: disable=bare-except interrupt_after_delay = True assert interrupt_inside_nested_raise assert not interrupt_after_delay
def step_events_iterator(self, step_context, step_key: str, databricks_run_id: int): """The launched Databricks job writes all event records to a specific dbfs file. This iterator regularly reads the contents of the file, adds any events that have not yet been seen to the instance, and yields any DagsterEvents. By doing this, we simulate having the remote Databricks process able to directly write to the local DagsterInstance. Importantly, this means that timestamps (and all other record properties) will be sourced from the Databricks process, rather than recording when this process happens to log them. """ check.int_param(databricks_run_id, "databricks_run_id") processed_events = 0 start = time.time() done = False step_context.log.info("Waiting for Databricks run %s to complete..." % databricks_run_id) while not done: with raise_execution_interrupts(): step_context.log.debug( "Waiting %.1f seconds...", self.databricks_runner.poll_interval_sec) time.sleep(self.databricks_runner.poll_interval_sec) try: done = poll_run_state( self.databricks_runner.client, step_context.log, start, databricks_run_id, self.databricks_runner.max_wait_time_sec, ) finally: all_events = self.get_step_events(step_context.run_id, step_key) # we get all available records on each poll, but we only want to process the # ones we haven't seen before for event in all_events[processed_events:]: # write each event from the DataBricks instance to the local instance step_context.instance.handle_new_event(event) if event.is_dagster_event: yield event.dagster_event processed_events = len(all_events) step_context.log.info(f"Databricks run {databricks_run_id} completed.")
def test_no_interrupt_after_nested_delay_and_raise(): interrupt_inside_nested_raise = False interrupt_after_delay = False try: with capture_interrupts(): with raise_execution_interrupts(): try: time.sleep(5) except: # pylint: disable=bare-except interrupt_inside_nested_raise = True _send_interrupt_to_self() except: # pylint: disable=bare-except interrupt_after_delay = True assert not interrupt_inside_nested_raise assert not interrupt_after_delay
def solid_execution_error_boundary(error_cls, msg_fn, step_context, **kwargs): """ A specialization of user_code_error_boundary for the steps involved in executing a solid. This variant supports the control flow exceptions RetryRequested and Failure as well as respecting the RetryPolicy if present. """ from dagster.core.execution.context.system import StepExecutionContext check.callable_param(msg_fn, "msg_fn") check.subclass_param(error_cls, "error_cls", DagsterUserCodeExecutionError) check.inst_param(step_context, "step_context", StepExecutionContext) with raise_execution_interrupts(): try: yield except (RetryRequested, Failure) as cf: # A control flow exception has occurred and should be propagated raise cf except DagsterError as de: # The system has thrown an error that is part of the user-framework contract raise de except Exception as e: # pylint: disable=W0703 # An exception has been thrown by user code and computation should cease # with the error reported further up the stack policy = step_context.solid_retry_policy if policy: # could check exc against a whitelist of exceptions raise RetryRequested( max_retries=policy.max_retries, # could support an enum of "delay curves" which use delay and # step_context.previous_attempt_count to calculate wait time seconds_to_wait=policy.delay, ) from e raise error_cls( msg_fn(), user_exception=e, original_exc_info=sys.exc_info(), **kwargs, ) from e
def solid_execution_error_boundary(error_cls, msg_fn, step_context, **kwargs): """ A specialization of user_code_error_boundary for the steps involved in executing a solid. This variant supports the control flow exceptions RetryRequested and Failure as well as respecting the RetryPolicy if present. """ from dagster.core.execution.context.system import StepExecutionContext check.callable_param(msg_fn, "msg_fn") check.class_param(error_cls, "error_cls", superclass=DagsterUserCodeExecutionError) check.inst_param(step_context, "step_context", StepExecutionContext) with raise_execution_interrupts(): step_context.log.begin_python_log_capture() retry_policy = step_context.solid_retry_policy try: yield except DagsterError as de: # The system has thrown an error that is part of the user-framework contract raise de except Exception as e: # pylint: disable=W0703 # An exception has been thrown by user code and computation should cease # with the error reported further up the stack # Directly thrown RetryRequested escalate before evaluating the retry policy. if isinstance(e, RetryRequested): raise e if retry_policy: raise RetryRequested( max_retries=retry_policy.max_retries, seconds_to_wait=retry_policy.calculate_delay( step_context.previous_attempt_count + 1), ) from e # Failure exceptions get re-throw without wrapping if isinstance(e, Failure): raise e # Otherwise wrap the user exception with context raise error_cls( msg_fn(), user_exception=e, original_exc_info=sys.exc_info(), **kwargs, ) from e except (DagsterExecutionInterruptedError, KeyboardInterrupt) as ie: # respect retry policy when interrupts occur if retry_policy: raise RetryRequested( max_retries=retry_policy.max_retries, seconds_to_wait=retry_policy.calculate_delay( step_context.previous_attempt_count + 1), ) from ie else: raise ie finally: step_context.log.end_python_log_capture()