Skip to content

Add on_kill() hook to BaseTrigger to handle user actions on triggers#65590

Merged
amoghrajesh merged 4 commits intoapache:mainfrom
astronomer:on-cancel-basetrigger
Apr 22, 2026
Merged

Add on_kill() hook to BaseTrigger to handle user actions on triggers#65590
amoghrajesh merged 4 commits intoapache:mainfrom
astronomer:on-cancel-basetrigger

Conversation

@amoghrajesh
Copy link
Copy Markdown
Contributor

@amoghrajesh amoghrajesh commented Apr 21, 2026


Was generative AI tooling used to co-author this PR?
  • Yes: Claude code

closes: #36090

Why + What

Deferred tasks interacting with external systems (BQ, DBX, etc.) have no standard way to cancel the external job when a user marks the task failed, clears it, or marks it succeeded. This leaves orphaned jobs running in the background.

I intend to fix that by adding on_kill() to BaseTrigger — a no-op by default that trigger authors override to cancel their external job when the task is acted on by a user.

The triggerer already passes trigger IDs through a to_cancel queue when a trigger is removed. Everything in that queue is a user action —> redistribution only happens when a triggerer is considered dead, at which point it is no longer processing its queue. We use asyncio.Task.cancel("user-action") to pass the reason through the existing cancellation path, so run_trigger() can check e.args[0] and invoke on_kill() without any extra API round-trip.

Impact

  1. Existing trigger subclasses
    Purely additive. on_kill() defaults to a no-op so nothing breaks if you do not implement it.

  2. Triggerer shutdown / redistribution
    No performance impact. on_kill() is only invoked when CancelledError carries the "user-action" message. Shutdown fires task.cancel("EOF - shutting down") and redistribution goes through the to_create path — neither triggers on_kill().

  3. Provider authors
    Override on_kill() to cancel external jobs. The framework guarantees it is only called for user initiated cancellations and you can implement it as needed.

Testing

Trigger

SENTINEL_FILE = "/tmp/on_cancel_was_called"


class VerifyOnCancelTrigger(BaseTrigger):
    """
    Trigger that sleeps forever.

    on_cancel() writes a sentinel file so we can verify it was called
    without needing a real external service like Databricks or BigQuery.
    """

    def serialize(self) -> tuple[str, dict]:
        return ("verify_on_cancel_trigger.VerifyOnCancelTrigger", {})

    async def run(self) -> AsyncIterator[TriggerEvent]:
        log.info("VerifyOnCancelTrigger running — mark this task as failed to test on_cancel()")
        await asyncio.sleep(99999)
        yield TriggerEvent("done")  # never reached

    async def on_kill(self) -> None:
        log.info("VerifyOnCancelTrigger.on_cancel() CALLED — simulated external job cancelled")
        with open(SENTINEL_FILE+"-"+str(random.randint(1, 10000)), "w") as f:
            f.write("cancelled")

DAG:
```python

class DeferForeverOperator(BaseOperator):
    def execute(self, context):
        self.defer(trigger=VerifyOnCancelTrigger(), method_name="resume")

    def resume(self, context, event=None):
        log.info("Resumed after trigger fired (should not happen in cancel test)")


with DAG(
    dag_id="test_on_cancel",
    start_date=datetime(2024, 1, 1),
    schedule=None,
    tags=["manual-test"],
) as dag:
    DeferForeverOperator(task_id="deferred_task")

The trigger is such that if on_cancel is called, once I mark it failed from UI, it will create a sentinel file.

  1. Cancelled trigger
image

The first one is marked failed, second is mark success and third is cancelled.

  1. Before run
 root@0c61b077a443:/opt/airflow$ ls /tmp/on_cancel_was_called
ls: cannot access '/tmp/on_cancel_was_called': No such file or directory
[Breeze:3.10.19] root@0c61b077a443:/opt/airflow$
[Breeze:3.10.19] root@0c61b077a443:/opt/airflow$
  1. After run
image

The image shows 3 files created for each action mentioned in 1.

Whats next?

  • Databricks triggers: add on_kill() to DatabricksExecutionTrigger and
    DatabricksSQLStatementExecutionTrigger (follow-up PR)
  • BigQuery trigger: migrate from duplicated safe_to_cancel/get_task_state
    boilerplate to on_kill()

  • Read the Pull Request Guidelines for more information. Note: commit author/co-author name and email in commits become permanently public when merged.
  • For fundamental code changes, an Airflow Improvement Proposal (AIP) is needed.
  • When adding dependency, check compliance with the ASF 3rd Party License Policy.
  • For significant user-facing changes create newsfragment: {pr_number}.significant.rst, in airflow-core/newsfragments. You can add this file in a follow-up commit after the PR is created so you know the PR number.

@amoghrajesh amoghrajesh added this to the Airflow 3.2.2 milestone Apr 21, 2026
@amoghrajesh amoghrajesh self-assigned this Apr 21, 2026
@amoghrajesh amoghrajesh added the full tests needed We need to run full set of tests for this PR to merge label Apr 21, 2026
@amoghrajesh amoghrajesh reopened this Apr 21, 2026
Comment thread airflow-core/src/airflow/triggers/base.py Outdated
Comment thread airflow-core/src/airflow/triggers/base.py Outdated
@amoghrajesh amoghrajesh force-pushed the on-cancel-basetrigger branch from 2ecbb75 to 393fdab Compare April 21, 2026 11:25
@amoghrajesh amoghrajesh requested a review from ashb April 21, 2026 11:25
Comment thread airflow-core/src/airflow/jobs/triggerer_job_runner.py Outdated
Comment thread airflow-core/src/airflow/jobs/triggerer_job_runner.py Outdated
Comment thread airflow-core/src/airflow/jobs/triggerer_job_runner.py Outdated
Comment thread airflow-core/src/airflow/jobs/triggerer_job_runner.py Outdated
Comment thread airflow-core/src/airflow/triggers/base.py Outdated
Comment thread airflow-core/tests/unit/jobs/test_triggerer_job.py Outdated
Comment thread airflow-core/tests/unit/jobs/test_triggerer_job.py Outdated
@amoghrajesh amoghrajesh requested a review from kaxil April 21, 2026 12:14
@kaxil
Copy link
Copy Markdown
Member

kaxil commented Apr 21, 2026

As discussed on Slack with you and @ashb, two suggestions:

1. Rename on_cancelon_kill

cancel already means asyncio.Task.cancel() / CancelledError in async land (fires on every exit: shutdown, redeploy, timeout, user action). on_kill is Airflow's established vocabulary for "task is being killed, stop the external work" -- see BaseOperator.on_kill(). DBX/BQ/Spark operators already implement it. Same name on the trigger is symmetric with the operator and avoids the CancelledError overlap.

2. Distinguish from cleanup() in the docstring

BaseTrigger.cleanup() already runs on every exit via the finally at triggerer_job_runner.py:1338-1353, including triggerer redeploys where the trigger hops to another process and keeps watching the same external job. If "cancel external work" went in cleanup, every rolling triggerer deploy would kill in-flight work.

So cleanup = "this trigger instance is leaving this process" (release local resources) and the new hook = "the task itself is being killed" (stop external work). The docstring should spell this out -- otherwise the obvious impl is to put DBX/BQ cancel in cleanup, which passes tests but breaks the first triggerer rollout.

@ashb
Copy link
Copy Markdown
Member

ashb commented Apr 21, 2026

otherwise the obvious impl is to put DBX/BQ cancel in cleanup, which passes tests but breaks the first triggerer rollout.

And I know of at least one person who did exactly this.

@amoghrajesh amoghrajesh changed the title Add on_cancel() hook to BaseTrigger to handle user actions on triggers Add on_kill() hook to BaseTrigger to handle user actions on triggers Apr 21, 2026
Comment thread airflow-core/src/airflow/jobs/triggerer_job_runner.py Outdated
@amoghrajesh amoghrajesh force-pushed the on-cancel-basetrigger branch from d2a8966 to ea3e28c Compare April 22, 2026 10:35
@amoghrajesh amoghrajesh merged commit 1812de8 into apache:main Apr 22, 2026
141 checks passed
@amoghrajesh amoghrajesh deleted the on-cancel-basetrigger branch April 22, 2026 12:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:Triggerer full tests needed We need to run full set of tests for this PR to merge

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Deferrable operator tasks do not call on_kill method when fail or restarted

4 participants