If you configure retries at the Job level, a new run will be kicked off when a run for that job fails. Compared to Op retries, the max retry limit for run retries applies to the whole run instead of each individual Op. Run retries also handle the cases where a run worker crashed.
To enable run retries, add the following to your dagster.yaml. This will start a new daemon which polls to the event log for run failure events.
run_retries:enabled:truemax_retries:3# Sets a default for all jobs. 0 if not set
You can also configure retries using tags either on Job definitions or in the Dagit Launchpad.
from dagster import job
@job(tags={"dagster/max_retries":3})defsample_job():pass@job(tags={"dagster/max_retries":3,"dagster/retry_strategy":"ALL_STEPS"})defother_sample_sample_job():pass
The dagster/retry_strategy tag controls which Ops the retry will run.
By default, retries will re-execute from failure (tag value FROM_FAILURE). This means that any successful Ops will be skipped, but their output will be used for downstream Ops. If the dagster/retry_strategy tag is set to ALL_STEPS, all the Ops will run again.
NOTE: FROM_FAILURE requires an IOManager that can access outputs from other runs. For example, on Kubernetes the s3_pickle_io_manager would work but the fs_io_manager would not, since the new run is in a new Kubernetes Job with a separate filesystem.