kiwi-ru opened a new issue, #31212:
URL: https://github.com/apache/airflow/issues/31212
### Apache Airflow version
Other Airflow 2 version (please specify below)
### What happened
Airflow Version: 2.4.3
Zombie detection doesn't keep the queue of already detected zombies and can
submit multiple callbacks to a DAG processor. If those callbacks aren't
processed in one iteration of DAG parsing and split between two different
iterations, a task instance can be marked UP_FOR_RETRY and immediately after
that - FAILED. For example, it may happen if `retry_callback` takes too long
(more than `zombie_detection_interval`). And despite configured retries, the
task instance won't be retried.
### What you think should happen instead
Task instance should be detected as zombie once and, for example, go into
`DETECTED_ZOMBIE` state.
### How to reproduce
1. Deploy the DAG:
```
#!/usr/bin/env python
import time
from datetime import datetime
from airflow import DAG
from airflow.operators.bash import BashOperator
from airflow.operators.empty import EmptyOperator
DAG_ID = 'with_retry_callback'
default_args = {
'start_date': datetime(2022, 1, 1),
}
def retry_callback(context):
time.sleep(20.0)
with DAG(
DAG_ID,
schedule_interval='0 8 * * *',
catchup=False,
default_args=default_args,
) as dag:
(
EmptyOperator(task_id='start')
>> BashOperator(
task_id='task',
bash_command='sleep 7200',
retries=1,
on_retry_callback=retry_callback,
)
>> EmptyOperator(task_id='end')
)
```
2. Trigger zombie detection by, for example, shutting down an Airflow worker
where it was running
3. DAG will be marked as FAILED despite `retries=1`
### Operating System
-
### Versions of Apache Airflow Providers
_No response_
### Deployment
Google Cloud Composer
### Deployment details
_No response_
### Anything else
Relevant part of Airflow Scheduler logs when it happens:
```
2023-05-11 15:30:23.798 CEST Detected zombie job: {'full_filepath':
'/home/airflow/gcs/dags/retry-callback.py', 'processor_subdir':
'/home/airflow/gcs/dags', 'msg': "{'DAG Id': 'with_retry_callback', 'Task Id':
'task', 'Run Id': 'manual__2023-05-11T12:19:28.069563Z', 'Hostname':
'airflow-worker-j67sw', 'External Executor Id':
'f5ce9e66-4ee1-4814-aead-283445d464f8'}", 'simple_task_instance':
<airflow.models.taskinstance.SimpleTaskInstance object at 0x7f533a18ed00>,
'is_failure_callback': True}
2023-05-11 15:30:24.407 CEST {'DAG Id': 'with_retry_callback', 'Task Id':
'task', 'Run Id': 'manual__2023-05-11T12:19:28.069563Z', 'Hostname':
'airflow-worker-j67sw', 'External Executor Id':
'f5ce9e66-4ee1-4814-aead-283445d464f8'}
2023-05-11 15:30:26.101 CEST Marking task as UP_FOR_RETRY.
dag_id=with_retry_callback, task_id=task, execution_date=20230511T121928,
start_date=20230511T121931, end_date=20230511T133024
2023-05-11 15:30:33.903 CEST Detected zombie job: {'full_filepath':
'/home/airflow/gcs/dags/retry-callback.py', 'processor_subdir':
'/home/airflow/gcs/dags', 'msg': "{'DAG Id': 'with_retry_callback', 'Task Id':
'task', 'Run Id': 'manual__2023-05-11T12:19:28.069563Z', 'Hostname':
'airflow-worker-j67sw', 'External Executor Id':
'f5ce9e66-4ee1-4814-aead-283445d464f8'}", 'simple_task_instance':
<airflow.models.taskinstance.SimpleTaskInstance object at 0x7f533a2987c0>,
'is_failure_callback': True}
2023-05-11 15:30:44.014 CEST Detected zombie job: {'full_filepath':
'/home/airflow/gcs/dags/retry-callback.py', 'processor_subdir':
'/home/airflow/gcs/dags', 'msg': "{'DAG Id': 'with_retry_callback', 'Task Id':
'task', 'Run Id': 'manual__2023-05-11T12:19:28.069563Z', 'Hostname':
'airflow-worker-j67sw', 'External Executor Id':
'f5ce9e66-4ee1-4814-aead-283445d464f8'}", 'simple_task_instance':
<airflow.models.taskinstance.SimpleTaskInstance object at 0x7f533a270d30>,
'is_failure_callback': True}
2023-05-11 15:31:16.748 CEST {'DAG Id': 'with_retry_callback', 'Task Id':
'task', 'Run Id': 'manual__2023-05-11T12:19:28.069563Z', 'Hostname':
'airflow-worker-j67sw', 'External Executor Id':
'f5ce9e66-4ee1-4814-aead-283445d464f8'}
2023-05-11 15:31:17.420 CEST Marking task as FAILED.
dag_id=with_retry_callback, task_id=task, execution_date=20230511T121928,
start_date=20230511T121931, end_date=20230511T133116
2023-05-11 15:31:17.491 CEST {'DAG Id': 'with_retry_callback', 'Task Id':
'task', 'Run Id': 'manual__2023-05-11T12:19:28.069563Z', 'Hostname':
'airflow-worker-j67sw', 'External Executor Id':
'f5ce9e66-4ee1-4814-aead-283445d464f8'}
2023-05-11 15:31:17.509 CEST Marking task as FAILED.
dag_id=with_retry_callback, task_id=task, execution_date=20230511T121928,
start_date=20230511T121931, end_date=20230511T133117
```
### Are you willing to submit PR?
- [ ] Yes I am willing to submit a PR!
### Code of Conduct
- [X] I agree to follow this project's [Code of
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
--
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]