waldoppper opened a new issue, #39078: URL: https://github.com/apache/airflow/issues/39078
### Apache Airflow version Other Airflow 2 version (please specify below) ### If "Other Airflow 2 version" selected, which one? 2.8.3 ### What happened? I'm chasing an issue involving a hard-to-reproduce issue with deferrable operators being "stuck" in Queued state. In debugging, I noticed what seems to be a defficiency in the kubernetes_executor's override of [cleanup_stuck_queued_tasks](https://github.com/apache/airflow/blob/b56c7cd3352bf8f493e081bda895058320122b51/airflow/executors/base_executor.py#L404-L415) is not actually `fail`ing the instance. ### What you think should happen instead? **Background**: In debugging, I focused on the logs available at default log-level, which included ``` Marking task instance <TaskInstance: my_dag.my_group.my_task scheduled__2024-03-25T20:00:00+00:00 [queued]> stuck in queued as failed. If the task instance has available retries, it will be retried. ``` In reviewing the [code printing this message](https://github.com/apache/airflow/blob/2.8.3/airflow/jobs/scheduler_job_runner.py#L1549-L1583), it seems clear that [the expectation of a base_executor subclass](https://github.com/apache/airflow/blob/b56c7cd3352bf8f493e081bda895058320122b51/airflow/executors/base_executor.py#L404-L415) is to `fail` the task-instances. For reference, the celery_executor [does](https://github.com/apache/airflow/blob/2.8.3/airflow/providers/celery/executors/celery_executor.py#L461). **Problem**: The kubernetes_executor [does not](https://github.com/apache/airflow/blob/2.8.3/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py#L578-L617). **Solution**: It seems to me like it should. ### How to reproduce The true root cause of my issue is still a mystery to me. This is an attempt at fixing this safety net. ### Operating System debian ### Versions of Apache Airflow Providers apache-airflow-providers-cncf-kubernetes==8.0.0 ### Deployment Other 3rd-party Helm chart ### Deployment details _No response_ ### Anything else? _No response_ ### Are you willing to submit PR? - [X] 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]
