nailo2c opened a new pull request, #67873: URL: https://github.com/apache/airflow/pull/67873
## Summary [`none_failed_min_one_success`](https://airflow.apache.org/docs/apache-airflow/stable/core-concepts/dags.html) requires all upstream tasks to avoid `failed` / `upstream_failed` and at least one upstream task to succeed. The existing `TriggerRuleDep` dep-check path shared the same [failure-count logic](https://github.com/apache/airflow/blob/33363d541851f3017ee96f94ceb328d684ed8360/airflow-core/src/airflow/ti_deps/deps/trigger_rule_dep.py#L539-L540) as `none_failed`. As a result, zero-success upstream states could be reported as dependencies met, even though the trigger rule's min-one-success contract could not be satisfied. This showed up in two counterexamples: - all upstream tasks skipped while `flag_upstream_failed=False`, where direct dependency checks such as `airflow tasks failed-deps` missed the trigger-rule failure reason; - dynamically mapped upstreams where all upstream task instances were `removed`, where the scheduler rewrite path also did not assign a terminal failure state. Before the fix, `airflow tasks failed-deps` did not report the trigger-rule violation for an all-skipped upstream case: <details> <summary>Reproduction Dag</summary> ```python import pendulum from airflow.providers.standard.operators.empty import EmptyOperator from airflow.providers.standard.operators.python import BranchPythonOperator from airflow.sdk import DAG def choose_branch(): return "branch_b" with DAG( dag_id="pattern5_repro", schedule=None, start_date=pendulum.datetime(2026, 1, 1, tz="UTC"), catchup=False, tags=["z3-counterexample"], ) as dag: pick = BranchPythonOperator( task_id="pick", python_callable=choose_branch, ) branch_a = EmptyOperator(task_id="branch_a") branch_b = EmptyOperator(task_id="branch_b") downstream = EmptyOperator( task_id="downstream", trigger_rule="none_failed_min_one_success", ) pick >> [branch_a, branch_b] branch_a >> downstream ``` </details> ```console [Breeze:3.10.20] root@57ce6ad76e64:/opt/airflow$ airflow tasks failed-deps pattern5_repro downstream manual__2026-05-31T23:57:32.573541+00:00 Task instance dependencies not met: Dagrun Running: Task instance's dagrun was not in the 'running' state but in the state 'success'. Task Instance State: Task is in the 'skipped' state. ``` After the fix, the missing trigger-rule dependency is reported: ```console [Breeze:3.10.20] root@57ce6ad76e64:/opt/airflow$ airflow tasks failed-deps pattern5_repro downstream manual__2026-05-31T23:57:32.573541+00:00 Task instance dependencies not met: Trigger Rule: Task's trigger rule 'none_failed_min_one_success' requires at least one upstream task success, but none were found. Dagrun Running: Task instance's dagrun was not in the 'running' state but in the state 'success'. Task Instance State: Task is in the 'skipped' state. ``` ## Fix Fail the `none_failed_min_one_success` dep check whenever no upstream task has succeeded. Also extend the `flag_upstream_failed=True` rewrite path so that, once all upstreams are terminal and none succeeded, the downstream task instance is marked `UPSTREAM_FAILED`. This covers mapped tasks whose relevant upstream task instances were all `removed`. The existing all-skipped behavior is preserved: that case is still rewritten to `SKIPPED` before the new all-terminal zero-success branch can run. ## Test Updated regression coverage for `none_failed_min_one_success`: - all-skipped upstreams now report the missing upstream success for both `flag_upstream_failed=True` and `False`; - all-removed mapped upstreams now report the missing upstream success, and with `flag_upstream_failed=True` the task instance is rewritten to `UPSTREAM_FAILED`. Verified with: ```bash uv run ruff format airflow-core/tests/unit/ti_deps/deps/test_trigger_rule_dep.py uv run ruff check --fix airflow-core/tests/unit/ti_deps/deps/test_trigger_rule_dep.py breeze run --backend postgres --python 3.10 --skip-image-upgrade-check --answer no pytest airflow-core/tests/unit/ti_deps/deps/test_trigger_rule_dep.py::TestTriggerRuleDep::test_none_failed_min_one_success_tr_skipped airflow-core/tests/unit/ti_deps/deps/test_trigger_rule_dep.py::TestTriggerRuleDep::test_mapped_task_upstream_all_removed_with_none_failed_min_one_success_trigger_rule -xvs ``` Result: `4 passed, 1 warning`. ## Reported by SMT solver sweep of Airflow trigger-rule semantics using a local Z3 model. --- ##### Was generative AI tooling used to co-author this PR? - [X] Yes - Codex (GPT-5) Generated-by: Codex (GPT-5) following [the guidelines](https://github.com/apache/airflow/blob/main/contributing-docs/05_pull_requests.rst#gen-ai-assisted-contributions) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
