kaxil opened a new pull request, #68254: URL: https://github.com/apache/airflow/pull/68254
A custom `retry_policy` was honored on the deferrable `TriggerDagRunOperator` wait path but silently ignored on the non-deferrable one. With `wait_for_completion=True` and `deferrable=False`, the operator polls the triggered DagRun. When it reached a failed state, the branch called `_handle_current_task_failed` directly, which only checks the standard `retries` count and never consults `task.retry_policy`. The deferrable path instead raises `AirflowException` from [`execute_complete`](https://github.com/apache/airflow/blob/2a92467755718f29d6ef3464323bab41309eab1b/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py#L486), which flows through `run()`'s exception handler into `_apply_retry_policy_or_default`, so the policy got a vote there. This routes the non-deferrable failed-state branch through `_apply_retry_policy_or_default` with a synthesized `AirflowException` that mirrors the deferrable path, so a configured policy is evaluated on both. ## Design notes - The synthesized exception reuses the deferrable path's message shape (`"<dag_id> failed with failed state <state>"`). The exception type is what `RetryRule` matches on, and `AirflowException` is the same class on both paths. - With no `retry_policy` set, behavior is unchanged: `_apply_retry_policy_or_default` falls back to `_handle_current_task_failed`, so the standard retry-count check still applies. - The conflict branch (triggered DagRun already exists) stays a hard failure, since a retry would just re-fail. ## Example ```python from airflow.sdk.exceptions import AirflowException from airflow.sdk.definitions.retry_policy import ExceptionRetryPolicy, RetryRule, RetryAction TriggerDagRunOperator( task_id="trigger", trigger_dag_id="downstream", wait_for_completion=True, deferrable=False, # Fail immediately on a failed downstream run, even if retries remain. retry_policy=ExceptionRetryPolicy( rules=[RetryRule(exception=AirflowException, action=RetryAction.FAIL)], ), ) ``` Follow-up to #65390, which fixed the count-based retry on this same branch. -- 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]
