def start( self, retry_mode: RetryMode, sort_key_fn: Optional[Callable[[ExecutionStep], float]] = None, ) -> "ActiveExecution": from .active import ActiveExecution return ActiveExecution( self, retry_mode, self.known_state.get_retry_state() if self.known_state else RetryState(), sort_key_fn, )
def get_retry_state(self): return RetryState(self.previous_retry_attempts)
def test_execute_step_verify_step(): with get_foo_pipeline_handle() as pipeline_handle: runner = CliRunner() with instance_for_test( overrides={ "compute_logs": { "module": "dagster.core.storage.noop_compute_log_manager", "class": "NoOpComputeLogManager", } }) as instance: run = create_run_for_test( instance, pipeline_name="foo", run_id="new_run", ) input_json = serialize_dagster_namedtuple( ExecuteStepArgs( pipeline_origin=pipeline_handle.get_python_origin(), pipeline_run_id=run.run_id, step_keys_to_execute=None, instance_ref=instance.get_ref(), )) # Check that verify succeeds for step that has hasn't been fun (case 3) retries = RetryState() assert verify_step(instance, run, retries, step_keys_to_execute=["do_something"]) # Check that verify fails when trying to retry with no original attempt (case 3) retries = RetryState() retries.mark_attempt("do_something") assert not verify_step( instance, run, retries, step_keys_to_execute=["do_something"]) # Test trying to re-run a retry fails verify_step (case 2) with mock.patch("dagster.cli.api.get_step_stats_by_key" ) as _step_stats_by_key: _step_stats_by_key.return_value = { "do_something": RunStepKeyStatsSnapshot(run_id=run.run_id, step_key="do_something", attempts=2) } retries = RetryState() retries.mark_attempt("do_something") assert not verify_step(instance, run, retries, step_keys_to_execute=["do_something"]) runner_execute_step( runner, [input_json], ) # # Check that verify fails for step that has already run (case 1) retries = RetryState() assert not verify_step( instance, run, retries, step_keys_to_execute=["do_something"])
def _dagster_event_sequence_for_step( step_context: StepExecutionContext, retry_state: RetryState ) -> Iterator[DagsterEvent]: """ Yield a sequence of dagster events for the given step with the step context. This function also processes errors. It handles a few error cases: (1) User code requests to be retried: A RetryRequested has been raised. We will either put the step in to up_for_retry state or a failure state depending on the number of previous attempts and the max_retries on the received RetryRequested. (2) User code fails successfully: The user-space code has raised a Failure which may have explicit metadata attached. (3) User code fails unexpectedly: The user-space code has raised an Exception. It has been wrapped in an exception derived from DagsterUserCodeException. In that case the original user exc_info is stashed on the exception as the original_exc_info property. (4) Execution interrupted: The run was interrupted in the middle of execution (typically by a termination request). (5) User error: The framework raised a DagsterError that indicates a usage error or some other error not communicated by a user-thrown exception. For example, if the user yields an object out of a compute function that is not a proper event (not an Output, ExpectationResult, etc). (6) Framework failure: An unexpected error occurred. This is a framework error. Either there has been an internal error in the framework OR we have forgotten to put a user code error boundary around invoked user-space code. These terminate the computation immediately (by re-raising). The "raised_dagster_errors" context manager can be used to force these errors to be re-raised and surfaced to the user. This is mostly to get sensible errors in test and ad-hoc contexts, rather than forcing the user to wade through the PipelineExecutionResult API in order to find the step that failed. For tools, however, this option should be false, and a sensible error message signaled to the user within that tool. """ check.inst_param(step_context, "step_context", StepExecutionContext) check.inst_param(retry_state, "retry_state", RetryState) try: prior_attempt_count = retry_state.get_attempt_count(step_context.step.key) if step_context.step_launcher: step_events = step_context.step_launcher.launch_step(step_context, prior_attempt_count) else: step_events = core_dagster_event_sequence_for_step(step_context, prior_attempt_count) for step_event in check.generator(step_events): yield step_event # case (1) in top comment except RetryRequested as retry_request: retry_err_info = serializable_error_info_from_exc_info(sys.exc_info()) if step_context.retry_mode.disabled: fail_err = SerializableErrorInfo( message="RetryRequested but retries are disabled", stack=retry_err_info.stack, cls_name=retry_err_info.cls_name, cause=retry_err_info.cause, ) step_context.capture_step_exception(retry_request) yield DagsterEvent.step_failure_event( step_context=step_context, step_failure_data=StepFailureData(error=fail_err, user_failure_data=None), ) else: # retries.enabled or retries.deferred prev_attempts = retry_state.get_attempt_count(step_context.step.key) if prev_attempts >= retry_request.max_retries: fail_err = SerializableErrorInfo( message="Exceeded max_retries of {}".format(retry_request.max_retries), stack=retry_err_info.stack, cls_name=retry_err_info.cls_name, cause=retry_err_info.cause, ) step_context.capture_step_exception(retry_request) yield DagsterEvent.step_failure_event( step_context=step_context, step_failure_data=StepFailureData(error=fail_err, user_failure_data=None), ) else: yield DagsterEvent.step_retry_event( step_context, StepRetryData( error=retry_err_info, seconds_to_wait=retry_request.seconds_to_wait, ), ) # case (2) in top comment except Failure as failure: step_context.capture_step_exception(failure) yield step_failure_event_from_exc_info( step_context, sys.exc_info(), UserFailureData( label="intentional-failure", description=failure.description, metadata_entries=failure.metadata_entries, ), ) if step_context.raise_on_error: raise failure # case (3) in top comment except DagsterUserCodeExecutionError as dagster_user_error: step_context.capture_step_exception(dagster_user_error.user_exception) yield step_failure_event_from_exc_info( step_context, sys.exc_info(), error_source=ErrorSource.USER_CODE_ERROR, ) if step_context.raise_on_error: raise dagster_user_error.user_exception # case (4) in top comment except (KeyboardInterrupt, DagsterExecutionInterruptedError) as interrupt_error: step_context.capture_step_exception(interrupt_error) yield step_failure_event_from_exc_info( step_context, sys.exc_info(), error_source=ErrorSource.INTERRUPT, ) raise interrupt_error # case (5) in top comment except DagsterError as dagster_error: step_context.capture_step_exception(dagster_error) yield step_failure_event_from_exc_info( step_context, sys.exc_info(), error_source=ErrorSource.FRAMEWORK_ERROR, ) if step_context.raise_on_error: raise dagster_error # case (6) in top comment except Exception as unexpected_exception: # pylint: disable=broad-except step_context.capture_step_exception(unexpected_exception) yield step_failure_event_from_exc_info( step_context, sys.exc_info(), error_source=ErrorSource.UNEXPECTED_ERROR, ) raise unexpected_exception