Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IntegrityError inserting into task_fail table with null execution_date from TI.handle_failure_with_callback #18943

Closed
2 tasks done
robinedwards opened this issue Oct 13, 2021 · 0 comments
Labels
affected_version:2.2 Issues Reported for 2.2 area:core kind:bug This is a clearly a bug priority:critical Showstopper bug that should be patched immediately
Milestone

Comments

@robinedwards
Copy link
Contributor

robinedwards commented Oct 13, 2021

Apache Airflow version

2.2.0 (latest released)

Operating System

Debian GNU/Linux 11 (bullseye)

Versions of Apache Airflow Providers

apache-airflow-providers-amazon @ file:///root/.cache/pypoetry/artifacts/c9/69/16/ffa2eb7a2e6e850a7048eaf66b6c40c990ef7c58149f20d3d3f333a2e9/apache_airflow_providers_amazon-2.2.0-py3-none-any.whl                                                                                                                                                                                         
apache-airflow-providers-celery @ file:///root/.cache/pypoetry/artifacts/6e/1b/2f/f968318a7474e979af4dc53893ecafe8cd11a98a94077a9c3c27304eb7/apache_airflow_providers_celery-2.1.0-py3-none-any.whl                                                                                                                                                                                         
apache-airflow-providers-ftp @ file:///root/.cache/pypoetry/artifacts/8b/9a/dd/79a36c62bc7f37f98d0ea33652570e19272e8a7a2297db13a6785698d1/apache_airflow_providers_ftp-2.0.1-py3-none-any.whl 
apache-airflow-providers-http @ file:///root/.cache/pypoetry/artifacts/52/28/81/03a89147daf7daceb55f1218189d1c4af01c33c45849b568769ca6765f/apache_airflow_providers_http-2.0.1-py3-none-any.whl                                                                                                                                                                                             
apache-airflow-providers-imap @ file:///root/.cache/pypoetry/artifacts/1c/5d/c5/269e8a8098e7017a26a2a376eb3020e1a864775b7ff310ed39e1bd503d/apache_airflow_providers_imap-2.0.1-py3-none-any.whl                                                                                                                                                                                             
apache-airflow-providers-postgres @ file:///root/.cache/pypoetry/artifacts/fb/69/ac/e8e25a0f6a4b0daf162c81c9cfdbb164a93bef6bd652c1c00eee6e0815/apache_airflow_providers_postgres-2.3.0-py3-none-any.whl                                                                                                                                                                                     
apache-airflow-providers-redis @ file:///root/.cache/pypoetry/artifacts/cf/2b/56/75563b6058fe45b70f93886dd92541e8349918eeea9d70c703816f2639/apache_airflow_providers_redis-2.0.1-py3-none-any.whl                                                                                                                                                                                           
apache-airflow-providers-sqlite @ file:///root/.cache/pypoetry/artifacts/61/ba/e9/c0b4b7ef2599dbd902b32afc99f2620d8a616b3072122e90f591de4807/apache_airflow_providers_sqlite-2.0.1-py3-none-any.whl  

Deployment

Other Docker-based deployment

Deployment details

AWS ECS, Celery Executor, Postgres 13, S3 Logging, Sentry integration

What happened

Noticed our Sentry getting a lot of integrity errors inserting into the task_fail table with a null execution date.

This seemed to be caused specifically by zombie task failures (We use AWS ECS Spot instances).

Specifically this callback from the dag file processor:

def handle_failure_with_callback(

Adds a task_fail here:

session.add(TaskFail(task, self.execution_date, self.start_date, self.end_date))

This blows up when it flushes further down the method. This i believe is because when the task instance is refreshed from the database the self.dag_run property is not populated. The proxy from ti.execution_date to ti.dag_run.execution_date then returns None causing our NOT NULL violation.

What you expected to happen

Insert into task_fail successfully and trigger callback

How to reproduce

Run this dag:

import logging
import time
from datetime import datetime

from airflow import DAG
from airflow.operators.python import PythonOperator


def long_running_task():
    for i in range(60):
        time.sleep(5)
        logging.info("Slept for 5")


def log_failure_dag(*args, **kwargs):
    logging.error("Our failure callback")


dag = DAG(
    dag_id="test_null_task_fail",
    schedule_interval='@daily',
    catchup=True,
    start_date=datetime(2021, 10, 9),
    max_active_runs=1,
    max_active_tasks=1,
    on_failure_callback=log_failure_dag,
)

with dag:
    PythonOperator(
        task_id="long_running",
        python_callable=long_running_task,
        on_failure_callback=log_failure_dag
    )

Kill the celery worker whilst its executing the long_running tasks. Wait for the zombie reaper of the scheduler to begin and call the failure handler.

Anything else

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@robinedwards robinedwards added area:core kind:bug This is a clearly a bug labels Oct 13, 2021
@ashb ashb added this to the Airflow 2.2.1 milestone Oct 13, 2021
@ashb ashb added the priority:critical Showstopper bug that should be patched immediately label Oct 13, 2021
@kaxil kaxil added the affected_version:2.2 Issues Reported for 2.2 label Oct 13, 2021
robinedwards added a commit to robinedwards/airflow that referenced this issue Oct 14, 2021
The dag_run property isn't populated by refresh_from_db or when this is
called from the failure handler when reaping zombies. This resulted in
an IntegrityError violating the NOT NULL constraint on task_fail
ashb pushed a commit to astronomer/airflow that referenced this issue Oct 14, 2021
The dag_run property isn't populated by refresh_from_db or when this is
called from the failure handler when reaping zombies. This resulted in
an IntegrityError violating the NOT NULL constraint on task_fail
@kaxil kaxil closed this as completed in 2966e3c Oct 14, 2021
jedcunningham pushed a commit that referenced this issue Oct 14, 2021
…NULL (#18979)

Fixes #18943 null exec date on insert to task_fail

The dag_run property isn't populated by refresh_from_db or when this is
called from the failure handler when reaping zombies. This resulted in
an IntegrityError violating the NOT NULL constraint on task_fail

(cherry picked from commit 2966e3c)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.2 Issues Reported for 2.2 area:core kind:bug This is a clearly a bug priority:critical Showstopper bug that should be patched immediately
Projects
None yet
Development

No branches or pull requests

3 participants