Apache Airflow version: 1.10.10
Kubernetes version (if you are using kubernetes) (use kubectl version):
Environment:
- Cloud provider or hardware configuration: any
- OS (e.g. from /etc/os-release): any
- Kernel (e.g.
uname -a): any
- Install tools: any
- Others: any
What happened:
- When a sensor is set to
soft_fail=True, it becomes skipped when it fails. However, it also calls BaseSensorOperator._do_skip_downstream_tasks() and skips all of its downstream tasks unconditionally, including those with trigger_rule none_failed, one_success etc.
ShortCircuitOperator is similar. When it is skipped, it skips all its downstream tasks unconditionally.
What you expected to happen:
- When a soft_fail sensor fails, it should skip itself. Downstream tasks with
trigger_rule all_success (i.e. the default) should be skipped because of TriggerRuleDep. Tasks that are none_failed or one_success etc should not be skipped unconditionally by the soft_fail sensor.
- Same applies for
ShortCircuitOperator
How to reproduce it:
Any DAG with soft_fail or ShortCircuitOperator and downstream tasks having trigger_rule none_failed, one_success will have this problem.
Anything else we need to know:
An old issue addressed this in general and made it possible for operators to skip themselves and not its downstream. The same principle should be applied to soft_fail and ShortCircuitOperator:
#1292
The fix should to this issue is rather simple, just raise AirflowSkipException and not skip downstream tasks. Leave it to the TriggerRuleDep to do the skip based on the trigger_rule of tasks.
Apache Airflow version: 1.10.10
Kubernetes version (if you are using kubernetes) (use
kubectl version):Environment:
uname -a): anyWhat happened:
soft_fail=True, it becomes skipped when it fails. However, it also callsBaseSensorOperator._do_skip_downstream_tasks()and skips all of its downstream tasks unconditionally, including those withtrigger_rulenone_failed,one_successetc.ShortCircuitOperatoris similar. When it is skipped, it skips all its downstream tasks unconditionally.What you expected to happen:
trigger_ruleall_success(i.e. the default) should be skipped because ofTriggerRuleDep. Tasks that arenone_failedorone_successetc should not be skipped unconditionally by the soft_fail sensor.ShortCircuitOperatorHow to reproduce it:
Any DAG with
soft_failorShortCircuitOperatorand downstream tasks havingtrigger_rulenone_failed,one_successwill have this problem.Anything else we need to know:
An old issue addressed this in general and made it possible for operators to skip themselves and not its downstream. The same principle should be applied to
soft_failandShortCircuitOperator:#1292
The fix should to this issue is rather simple, just raise
AirflowSkipExceptionand not skip downstream tasks. Leave it to the TriggerRuleDep to do the skip based on the trigger_rule of tasks.