Skip to content

on_kill Hook not executed on task stop #51005

@hendricjabs

Description

@hendricjabs

Apache Airflow version

3.0.1

If "Other Airflow 2 version" selected, which one?

No response

What happened?

When stopping a running task manually via UI, the on_kill hook is not executed, and no log message appears.

What you think should happen instead?

The on_kill hook should be called and so the log message should be printed out.

How to reproduce

I use the following Operator to test this issue:


from time import sleep
from typing import TYPE_CHECKING

from airflow.models.baseoperator import BaseOperator
import time
import requests

if TYPE_CHECKING:
    from airflow.sdk.definitions.context import Context


class TestSmoothOperator(BaseOperator):
    """Operator that logs a YouTube link to Sade song "Smooth Operator"."""

    ui_color = "#e8f7e4"
    yt_link: str = "https://www.youtube.com/watch?v=4TYv2PhG89A"

    def __init__(self, **kwargs) -> None:
        super().__init__(**kwargs)

    def pre_execute(self, context):
        print("pre Execute!")

    def post_execute(self, context):
        print("post Execute!")

    def execute(self, context: Context):
        # Number of times to loop

        num_iterations = 3

        for i in range(num_iterations):
            print(f"Iteration {i + 1}: Sleeping for 20 seconds...")
            time.sleep(20)  # Sleep for 20 seconds
            print(f"Iteration {i + 1}: Woke up after 20 seconds!")
        self.log.info("Enjoy Sade - Smooth Operator: %s", self.yt_link)

    def post_execute(self, context):
        print("post Execute!")

    def on_kill(self) -> None:
        self.log.info("Logging on Kill!")
        print("please someone kill me!")

I use the following DAG:

import os
from datetime import timedelta

from airflow import DAG

import airflow_runtime.tools.airflow_tools
import airflow_runtime.tools.default_args
import airflow_runtime.tools.emr_spark
from airflow_runtime.tools.test_smooth_operator import TestSmoothOperator

DAG_ID = os.path.basename(__file__).replace(".py", "")
schedule = None

with DAG(
    DAG_ID,
    default_args=airflow_runtime.tools.default_args.default_args,
    max_active_runs=1,
    dagrun_timeout=timedelta(minutes=30),
    on_success_callback=airflow_runtime.tools.airflow_tools.cleanup_xcom,
    tags=["vscode", "development"],
    schedule=schedule,
    catchup=False,
) as dag:
    op_kwargs = {"instance_names": ["my-instance"]}

    start_instance = TestSmoothOperator(
        task_id="test_smoothOperator",
    )

    start_instance
  1. Start the Task
  2. Stop the Task
  3. Check the logs

Operating System

AWS EKS / Kubernetes; Helm chart 1.17.0-dev

Versions of Apache Airflow Providers

Package Name Version Description
apache-airflow-providers-amazon 9.7.0 Amazon integration (including Amazon Web Services (AWS) https://aws.amazon.com/)
apache-airflow-providers-celery 3.10.6 Celery https://docs.celeryq.dev/en/stable/
apache-airflow-providers-cncf-kubernetes 10.4.3 Kubernetes https://kubernetes.io/
apache-airflow-providers-common-compat 1.6.1 Common Compatibility Provider - providing compatibility code for previous Airflow versions
apache-airflow-providers-common-io 1.5.4 Common IO Provider
apache-airflow-providers-common-messaging 1.0.1 Common Messaging Provider
apache-airflow-providers-common-sql 1.27.0 Common SQL Provider https://en.wikipedia.org/wiki/SQL
apache-airflow-providers-docker 4.3.1 Docker https://www.docker.com/
apache-airflow-providers-elasticsearch 6.2.2 Elasticsearch https://www.elastic.co/elasticsearch
apache-airflow-providers-fab 2.0.2 Flask App Builder https://flask-appbuilder.readthedocs.io/
apache-airflow-providers-ftp 3.12.3 File Transfer Protocol (FTP) https://tools.ietf.org/html/rfc114
apache-airflow-providers-git 0.0.2 Distributed version control system (GIT) https://git-scm.com/
apache-airflow-providers-google 15.1.0 Google services including: - Google Ads https://ads.google.com/ - Google Cloud (GCP) https://cloud.google.com/ - Google Firebase https://firebase.google.com/ - Google LevelDB https://github.com/google/leveldb/ - Google Marketing Platform https://marketingplatform.google.com/ - Google Workspace https://workspace.google.com/(formerly Google Suite)
apache-airflow-providers-grpc 3.7.3 gRPC https://grpc.io/
apache-airflow-providers-hashicorp 4.1.1 Hashicorp including Hashicorp Vault https://www.vaultproject.io/
apache-airflow-providers-http 5.2.2 Hypertext Transfer Protocol (HTTP) https://www.w3.org/Protocols/
apache-airflow-providers-microsoft-azure 12.3.1 Microsoft Azure https://azure.microsoft.com/
apache-airflow-providers-mysql 6.2.2 MySQL https://www.mysql.com/
apache-airflow-providers-odbc 4.9.2 ODBC https://github.com/mkleehammer/pyodbc/wiki
apache-airflow-providers-openlineage 2.2.0 OpenLineage https://openlineage.io/
apache-airflow-providers-postgres 6.1.3 PostgreSQL https://www.postgresql.org/
apache-airflow-providers-redis 4.0.2 Redis https://redis.io/
apache-airflow-providers-sendgrid 4.0.1 Sendgrid https://sendgrid.com/
apache-airflow-providers-sftp 5.2.1 SSH File Transfer Protocol (SFTP) https://tools.ietf.org/wg/secsh/draft-ietf-secsh-filexfer/
apache-airflow-providers-slack 9.0.5 Slack https://slack.com/services integration including: - Slack API https://api.slack.com/ - Slack Incoming Webhook https://api.slack.com/messaging/webhooks
apache-airflow-providers-smtp 2.0.3 Simple Mail Transfer Protocol (SMTP) https://tools.ietf.org/html/rfc5321
apache-airflow-providers-snowflake 6.3.0 Snowflake https://www.snowflake.com/
apache-airflow-providers-ssh 4.0.1 Secure Shell (SSH) https://tools.ietf.org/html/rfc4251
apache-airflow-providers-standard 1.1.0

Deployment

Official Apache Airflow Helm Chart

Deployment details

Kubernetes 1.30
Airflow 3.0.1

Anything else?

The problem occurs every time.

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

Labels

affected_version:3.0Issues Reported for 3.0area:corekind:bugThis is a clearly a bugpriority:mediumBug that should be fixed before next release but would not block a release

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions