Apache Airflow® trigger rules
Apache Airflow® trigger rules
Trigger rules are used to determine when a task should run in relation to the previous task. By default, Apache Airflow® runs a task when all directly upstream tasks are successful. However, you can change this behavior using the trigger_rule parameter in the task definition.
Trigger rules define whether a task runs based on its direct upstream dependencies. To learn how to set task dependencies, see the Manage task and task group dependencies in Airflow guide.
Define a trigger rule
You can override the default trigger rule by setting the trigger_rule parameter in the task definition.
Trigger rules in Airflow
The following trigger rules are available:
all_success: (default) The task runs only when all upstream tasks have succeeded.all_done: The task runs once all upstream tasks are done with their execution.all_done_min_one_success: The task runs once all upstream tasks are done with their execution and at least one upstream task has succeeded. Note thatskippeddoes not count as done for this rule and will cause the downstream task to be skipped as well.all_failed: The task runs only when all upstream tasks are in afailedorupstream_failedstate.all_skipped: The task runs only when all upstream tasks have been skipped.always: The task runs as soon as the Dag run starts, independently of the status of any upstream tasks.none_failed: The task runs only when all upstream tasks have succeeded or been skipped.none_failed_min_one_success: The task runs only when all upstream tasks are not in the statefailedorupstream_failed, and at least one upstream task has succeeded.none_skipped: The task runs only when no upstream task is in askippedstate.one_done: The task runs as soon as at least one upstream task has either succeeded or failed.one_failed: The task runs as soon as at least one upstream task is infailedorupstream_failedstate.one_success: The task runs as soon as at least one upstream task has succeeded.all_done_setup_success: Special trigger rule used in teardown tasks. The task runs when all upstream tasks have finished and at least one directly connected setup task has been successful. See the Use setup and teardown tasks in Airflow guide for more information.
You can define a Dag in which any task failure stops the Dag execution by setting the Dag parameter fail_fast to True. This will set all tasks that are still running to failed and mark any tasks that have not run yet as skipped, as soon as any task in the Dag fails. Note that you cannot have any trigger rule other than all_success (and all_done_setup_success) in a Dag with fail_fast set to True and that teardown tasks are exempt and will still run.
Branching and trigger rules
One common scenario where you might need to implement trigger rules is if your Dag contains conditional logic such as branching. In these cases, for the task directly after the branches, none_failed_min_one_success or none_failed are likely more helpful than all_success, because unless all branches are run, at least one upstream task will always be in a skipped state.
In the following example Dag there is a simple branch with a downstream task that needs to run if either of the branches are followed. With the all_success rule, the end task never runs because all but one of the branch tasks is always ignored and therefore doesn’t have a success state. If you change the trigger rule to none_failed_min_one_success, then the end task can run so long as one at least one of the branches has succeeded and none of the branches have failed.
This image shows the resulting Dag:

Traditional Syntax
Airflow trigger rules in detail
all_success
This is the default trigger rule. A task with the trigger rule all_success only runs when all upstream tasks have succeeded.

As soon as any upstream tasks are in the state of failed, upstream_failed, the downstream task is set to the state upstream_failed and does not run.

Similarly, as soon as any upstream task is in the state skipped, the downstream task is set to the state skipped and does not run.

If a task with the trigger rule all_success has one upstream task that is skipped and one that is failed / upstream_failed, whether the downstream task is set to skipped or upstream_failed depends on which of the upstream tasks finishes first. If the first upstream task that is not successful ends with the state skipped the downstream task is skipped, if it ends in failed or upstream_failed the downstream task is set to upstream_failed. If both upstream tasks, skipped and failed/upstream_failed finish in the same scheduler evaluation period, the downstream task will be set to upstream_failed.
all_done
The all_done trigger rule will make a task wait until all upstream tasks are done with their execution.

As soon as all tasks finish, no matter what their state is, the downstream task will run.

all_failed
The all_failed trigger rule will make a task wait until all upstream tasks are in a failed or upstream_failed state.


As soon as any upstream task is in the state success or skipped, the downstream task is set to the state skipped and does not run.


all_skipped
A task with the trigger rule all_skipped waits for all its upstream tasks to be skipped.


As soon as any upstream task is in the state success, failed, or upstream_failed, the downstream task with the trigger rule all_skipped is set to the state skipped and does not run.

all_done_min_one_success
Tasks using the all_done_min_one_success trigger rule run only when three conditions are met:
- All upstream tasks are in either
success,failedorupstream_failedstate. - At least one upstream task is in the success state.
- No upstream task is in the
skippedstate.


If all upstream tasks finish in either failed or upstream_failed state, the task using the all_done_min_one_success trigger rule is set to upstream_failed.

As soon as any upstream task is in a skipped state the task using the all_done_min_one_success trigger rule is skipped as well.

always
A task with the trigger rule always runs as soon as the Dag run is started, regardless of the state of its upstream tasks.

none_failed
The none_failed trigger rule makes a task run only when all upstream tasks have either succeeded or been skipped.

As soon as any upstream task is in the state failed or upstream_failed, the downstream task is set to the state upstream_failed and does not run.

none_failed_min_one_success
Tasks using the none_failed_min_one_success trigger rule run only when three conditions are met:
- All upstream tasks are finished.
- No upstream tasks are in the
failedorupstream_failedstate. - At least one upstream task is in the
successstate.


If any upstream task is in the failed or upstream_failed state, the downstream task is set to the state upstream_failed and does not run.

If all upstream tasks are in the skipped state, the downstream task is set to the state skipped and does not run.

none_skipped
Tasks using the none_skipped trigger rule run only when no upstream task is in the skipped state. Upstream tasks can be in any other state: success, failed, or upstream_failed.


If any upstream task is in the skipped state, the downstream task is set to the state skipped and does not run.

one_done
The one_done trigger rule makes a task run as soon as at least one of its upstream tasks is in either the success or failed state. Upstream tasks with skipped or upstream_failed states are not considered “done”.

Once one upstream task finishes (either in the success or failed state), the downstream task runs.


If all upstream tasks are either in skipped or upstream_failed states, the downstream task with the one_done trigger rule is set to the state skipped.

one_failed
The one_failed trigger rule will make a task run as soon as at least one of its upstream tasks is in either the failed or upstream_failed state.


If all upstream tasks have completed and none of them are in the failed or upstream_failed state, the downstream task will be set to the state skipped.

one_success
The one_success trigger rule will make a task run as soon as at least one of its upstream tasks is in the success state.

If all upstream tasks have been skipped, the downstream task with the one_success trigger rule is set to the state skipped as well.

If all upstream tasks have completed and at least one of them is in the failed or upstream_failed state, the downstream task will be set to the state upstream_failed.
