You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/01/07 15:42:24 UTC

[GitHub] [airflow] Overbryd opened a new issue #13542: Task stuck in "scheduled" or "queued" state, pool has open slots

Overbryd opened a new issue #13542:
URL: https://github.com/apache/airflow/issues/13542


   **Apache Airflow version**: `2.0.0`
   
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`):
   
   ```
   Client Version: version.Info{Major:"1", Minor:"19", GitVersion:"v1.19.3", GitCommit:"1e11e4a2108024935ecfcb2912226cedeafd99df", GitTreeState:"clean", BuildDate:"2020-10-14T12:50:19Z", GoVersion:"go1.15.2", Compiler:"gc", Platform:"darwin/amd64"}
   Server Version: version.Info{Major:"1", Minor:"17+", GitVersion:"v1.17.14-gke.1600", GitCommit:"7c407f5cc8632f9af5a2657f220963aa7f1c46e7", GitTreeState:"clean", BuildDate:"2020-12-07T09:22:27Z", GoVersion:"go1.13.15b4", Compiler:"gc", Platform:"linux/amd64"}
   ```
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**: GKE
   - **OS** (e.g. from /etc/os-release):
   - **Kernel** (e.g. `uname -a`):
   - **Install tools**:
   - **Others**:
     - Airflow metadata database is hooked up to a PostgreSQL instance
   
   **What happened**:
   
   * Airflow 2.0.0 running on the `KubernetesExecutor` has many tasks stuck in "scheduled" or "queued" state which never get resolved.
   * The setup has a `default_pool` of 16 slots.
   * Currently no slots are used (see Screenshot), but all slots are queued.
   * No work is executed any more. The Executor or Scheduler is stuck.
   * There are many many tasks stuck in "scheduled" state
     * Tasks in "scheduled" state say `('Not scheduling since there are %s open slots in pool %s and require %s pool slots', 0, 'default_pool', 1)`
       That is simply not true, because there is nothing running on the cluster and there are always 16 tasks stuck in "queued".
   * There are many tasks stuck in "queued" state
     * Tasks in "queued" state say `Task is in the 'running' state which is not a valid state for execution. The task must be cleared in order to be run.`
       That is also not true. Nothing is running on the cluster and Airflow is likely just lying to itself. It seems the KubernetesExecutor and the scheduler easily go out of sync.
   
   **What you expected to happen**:
   
   * Airflow should resolve scheduled or queued tasks by itself once the pool has available slots
   * Airflow should use all available slots in the pool
   * It should be possible to clear a couple hundred tasks and expect the system to stay consistent
   
   **How to reproduce it**:
   
   * Vanilla Airflow 2.0.0 with `KubernetesExecutor` on Python `3.7.9`
   * `requirements.txt`
   
     ```
     pyodbc==4.0.30
     pycryptodomex==3.9.9
     apache-airflow-providers-google==1.0.0
     apache-airflow-providers-odbc==1.0.0
     apache-airflow-providers-postgres==1.0.0
     apache-airflow-providers-cncf-kubernetes==1.0.0
     apache-airflow-providers-sftp==1.0.0
     apache-airflow-providers-ssh==1.0.0
     ```
   
   * The only reliable way to trigger that weird bug is to clear the task state of many tasks at once. (> 300 tasks)
   
   **Anything else we need to know**:
   
   Don't know, as always I am happy to help debug this problem.
   The scheduler/executer seems to go out of sync and never back in sync again with the state of the world.
   
   We actually planned to upscale our Airflow installation with many more simultaneous tasks. With these severe yet basic scheduling/queuing problems we cannot move forward at all.
   
   Another strange, likely unrelated observation, the scheduler always uses 100% of the CPU. Burning it. Even with no scheduled or now queued tasks, its always very very busy.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853117451


   More data, if it can benefit:
   Airflow version: 2.1.0
   Kubernetes version: 1.18.9
   
   I'm using Airflow `2.1.0` (upgraded from `1.10.15` in our development environment) and trying to use the `KubernetesExecutor`. I am running _extremely_ basic DAGs that exist solely to test. In `airflow.cfg` I set `delete_worker_pods = False` so that I could try and examine what's going on. As far as I can tell, my worker pods are being created, immediately considering themselves successful (not sure if they actually receive a task to complete), and terminating. I don't think the pods are "successful but not writing the success state to the DB" because one of my tasks is a simple `CREATE TABLE` statement on my data warehouse, and the table does not appear.
   
   
   Here are the results of various logs:
   
   If I run `kubectl get pods`, I see my `airflow-webserver` and `airflow-scheduler` pods, as well as some "Completed" worker pods.
   
   ```
   NAME                                                           READY   STATUS      RESTARTS   AGE
   airflow-scheduler                                              1/1     Running     0          45m
   airflow-webserver-1                                            2/2     Running     0          56m
   airflow-webserver-2                                            2/2     Running     0          56m
   airflow-webserver-3                                            2/2     Running     0          45m
   airflow-webserver-4                                            2/2     Running     0          45m
   airflow-webserver-5                                            2/2     Running     0          45m
   <GENERATED_WORKER_POD_1>                                       0/1     Completed   0          15m
   <GENERATED_WORKER_POD_2>                                       0/1     Completed   0          56m
   <GENERATED_WORKER_POD_3>                                       0/1     Completed   0          45m
   ```
   
   `kubectl logs <my_worker_pod>` - I can tell that the pod was briefly alive but that did nothing, because the only output is a line that always appears as a side effect of the Docker image config. In contrast, if I get the logs from one of my functional `1.10.15` pods, I can see the task start:
   ```
   [2021-06-02 13:33:01,358] {__init__.py:50} INFO - Using executor LocalExecutor
   [2021-06-02 13:33:01,358] {dagbag.py:417} INFO - Filling up the DagBag from /usr/local/airflow/dags/my_dag.py
   ...
   etc
   ```
   
   `kubectl describe pod <my_worker_pod>` - the event log is quite tame:
   ```
   Events:
     Type    Reason     Age    From               Message
     ----    ------     ----   ----               -------
     Normal  Scheduled  3m21s  default-scheduler  Successfully assigned <GENERATED_POD_NAME> to <EC2_INSTANCE>
     Normal  Pulling    3m19s  kubelet            Pulling image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Pulled     3m19s  kubelet            Successfully pulled image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Created    3m19s  kubelet            Created container base
     Normal  Started    3m19s  kubelet            Started container base
   ```
   
   `kubectl logs airflow-scheduler` - I've trimmed the logs significantly, but here are the statements mentioning a stuck task
   ```
   [2021-06-02 14:49:42,742] {kubernetes_executor.py:369} INFO - Attempting to finish pod; pod_id: <GENERATED_POD_NAME>; state: None; annotations: {'dag_id': '<TEST_DAG>', 'task_id': '<TASK_ID>', 'execution_date': '2021-05-30T00:00:00+00:00', 'try_number': '1'}
   [2021-06-02 14:49:42,748] {kubernetes_executor.py:546} INFO - Changing state of (TaskInstanceKey(dag_id='<TEST_DAG>', task_id='<TASK_ID>', execution_date=datetime.datetime(2021, 5, 30, 0, 0, tzinfo=tzlocal()), try_number=1), None, '<GENERATED_POD_NAME>', 'airflow', '1000200742') to None
   [2021-06-02 14:49:42,751] {scheduler_job.py:1212} INFO - Executor reports execution of <TASK_ID> execution_date=2021-05-30 00:00:00+00:00 exited with status None for try_number 1
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-924363913


   > We're continuing to see an issue in `2.1.3` where DAG tasks appear to be running for many hours, even days, but are making no visible progress. This situation persists through scheduler restarts and is not resolved until we clear each stuck task manually twice. The first time we clear the task we see an odd rendering in the UI with a dark blue border that looks like this:
   > 
   > <img alt="Screen Shot 2021-09-21 at 7 40 43 AM" width="637" src="https://user-images.githubusercontent.com/74351/134193377-7f826066-37cc-4f48-bb6f-b2b2224a6be7.png">
   > 
   > The second time we clear the task the border changes to light green and it usually completes as expected.
   > 
   > This is a pretty frustrating situation because the only known path to remediation is manual intervention. As previously stated we're deploying to ECS, with each Airflow process as its own ECS service in an ECS cluster; this generally works well except as noted here.
   
   Hey @maxcountryman,
   
   That sounds very annoying. Sorry to hear that.
   I guess you are using the KubernetesExecutor?
   Next time that happens could you do the following two things:
   1. Send a USR2 kill command to the scheduler and list the output here. Example: `pkill -f -USR2 "airflow scheduler"`
   2. Check the TaskInstance that is stuck and especially the `external executor id` attribute of the TaskInstance. You can find this on the detailed view of a task instance.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jbkc85 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jbkc85 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-896304736


   @hafid-d / @trucnguyenlam - have you tried separating DAGs into separate pools?  The more pools that are available the less 'lanes' you'll have that are piled up.  Just beware that in doing this, you are also somewhat breaking the parallelism - but it does organize your DAGs much better over time!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] tienhung2812 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
tienhung2812 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-921858288


   Hi, just got this issue again on `2.1.3` but the behavior quite different, all of schedulers stuck into a DAG check loop, but restart the scheduler will fix it, all other DAG as stuck in `scheduled`
   
   ```
    [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,511] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks
   airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,512] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.aa.aa_20.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,512] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks
   airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,513] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.aa.ff_46.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,513] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks`
   airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,514] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.ff.gg_121.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   ```
   
   The current system have about >= 150 active dags but we have a`abc_dag` have more than 500 tasks inside
   I have already increase the scheduler config to handling more task
   - max_tis_per_query: 512
   - max_dagruns_to_create_per_loop: 50
   - max_dagruns_per_loop_to_schedule: 70
   I have also add a scheduler healthcheck but it can not detect this issue 
   
   Environment:
   - Celery Executor
   - MySQL 8.0.23-14
   - Airflow: 2.1.3 
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] pelaprat commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
pelaprat commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821575562


   We are experiencing this bug in our Airlfow implementation as well. This is my first time participating in reporting a bug on the Apache Airflow project. Out of curiosity, who is responsible on the Apache Airflow project for prioritizing these issues?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-814068888


   @lukas-at-harren -- Can you check the Airflow Webserver -> Admin -> Pools  and then in the row with your pool (`mssql_dwh`) check the Used slots. And click on the number in Used slots, it should take you to the TaskInstance page that should show the currently "running" taskinstances in that Pool.
   
   It is possible that they are not actually running but somehow got in that state in DB. If you see 3 entries over here, please mark those tasks as success or failed, that should clear your pool.
   
   ![image](https://user-images.githubusercontent.com/8811558/113708634-64508400-96d9-11eb-8505-2c293f0d2f44.png)
   
   ![image](https://user-images.githubusercontent.com/8811558/113708679-72060980-96d9-11eb-8c11-6c84a9c64801.png)
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-824575868


   I can observe the same problem with version 2.0.2:
   
   * Tasks fail, because a DAG/task has gone missing (we are using dynamically created DAGs, and they can go missing)
   * The scheduler keeps those queued
   * The pool gradually fills up with these queued tasks
   * The whole operation stops, because of this behaviour
   
   My current remedy:
   
   * Manually remove those queued tasks
   
   My desired solution:
   
   When a DAG/task goes missing while it is queued, it should end up in a failed state.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jbkc85 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jbkc85 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-876052790


   > > We're only making use of the `default_pool` and tasks are moving from `Queued` to `Running` there but aren't entering the queue despite being in a `Scheduled` state.
   > 
   > We had a similar issue and increased the `default_pool` 10x, from 128 to 1000. This fixed it completely for us, however, we are using Celery Executor though.
   > Did you already attempt something similar? Wondering if this helps in your case as well.
   
   We just updated it.  We will let you know in the morning - hopefully we see some improvement!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853693757


   I have resolved this issue for my environment! I'm not sure if this is the same "bug" as others, or a different issue with similar symptoms. But here we go
   
   ----
   
   In my airflow Docker image, the entrypoint is just a bootstrap script that accepts `webserver` or `scheduler` as arguments, and accepts the appropriate command.
   ```
   # Installing python libs, jars, etc
   ...
   ENTRYPOINT ["/bootstrap.sh"]
   ```
   
   `bootstrap.sh`:
   ```
   if [ "$1" = "webserver" ]
   then
   	exec airflow webserver
   fi
   
   if [ "$1" = "scheduler" ]
   then
   	exec airflow scheduler
   fi
   ```
   
   Previous to #12766, the KubernetesExecutor fed the `airflow tasks run` (or `airflow run` in older versions) into the `command` section of pod YAML.
   
   ```
   "containers": [
         {
           "args": [],
           "command": [
             "airflow",
             "run",
             "my_dag",
             "my_task",
             "2021-06-03T03:40:00+00:00",
             "--local",
             "--pool",
             "default_pool",
             "-sd",
             "/usr/local/airflow/dags/my_dag.py"
           ],
   ```
   This works fine for my setup -- the `command` just overrides my Docker's `ENTRYPOINT`, the pod executes its given command and terminates on completion. However, [this](https://github.com/apache/airflow/pull/12766/files#diff-681de8974a439f70dfa41705f5c1681ecce615fac6c4c715c1978d28d8f0da84L300) change moved the `airflow tasks run` issuance to the `args` section of the YAML. 
   ```
   'containers': [{'args': ['airflow',
                                      'tasks',
                                      'run',
                                      'my_dag,
                                      'my_task',
                                      '2021-06-02T00:00:00+00:00',
                                      '--local',
                                      '--pool',
                                      'default_pool',
                                      '--subdir',
                                      '/usr/local/airflow/dags/my_dag.py'],
                             'command': None,
   ```
   
   These new args do not match either `webserver` or `scheduler` in `bootstrap.sh`, therefore the script ends cleanly and so does the pod. Here is my solution, added to the bottom of `bootstrap.sh`:
   ```
   if [ "$1" = "airflow" ] && [ "$2" = "tasks" ] && [ "$3" = "run" ]
   then
   	exec "$@"
   fi
   ```
   Rather than just allow the pod to execute _whatever_ it's given in `args` (aka just running `exec "$@"` without a check), I decided to at least make sure the pod is being fed an `airflow run task` command.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FurcyPin edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
FurcyPin edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-948858685


   Hello, I think I ran into the same issue.
   
   ![2021-10-21-19:05:47](https://user-images.githubusercontent.com/6087587/138324724-d56b19d0-8dda-40e8-a62b-4b95f6155ba5.png)
   
   Here's all that the relevant info I could find, hoping this will help to solve it.
   
   * Stack: Cloud Composer 2 
   * Image version: composer-2.0.0-preview.3-airflow-2.1.2
   * Executor: I'm not sure what Cloud Composer uses, but the airflow.cfg in the bucket says "CeleryExecutor"
   
   - I am 100% sure that my DAG code does not contain any error (other similar tasks work fine).
   - I do not use any pool.
   - I do not have a trigger date in the future.
   
   I tried clearing the queued tasks, they immediately appeared as queued again, and the scheduler logged this message (and nothing else useful):
   ```
   could not queue task TaskInstanceKey(dag_id='my_dag', task_id='my_task', execution_date=datetime.datetime(2021, 10, 20, 0, 0, tzinfo=Timezone('UTC')), try_number=2)
   ```
   
   I tried restarting the scheduler by destroying the pod, it did not change anything.
   I tried destroying the Redis pod, but I did not have the necessary permission.
   
   I was running a large number of dags at the same time (more than 20), so it might be linked to `max_dagruns_per_loop_to_schedule`, so I increased it to a number larger than my number of dags,
   but the 3 tasks are still stuck in the queued state, even when I clear them.
   
   I'm running out of ideas... luckily it's only a POC, so if someone has some suggestions to what I could try next, I would be happy to try them.
   
   UPDATE : after a little more than 24 hours, the three tasks went into failed state by themselves. I cleared them and they ran with no problem...


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] avenkatraman commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
avenkatraman commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-815159787


   We're running into a similar problem on Airflow 1.10.14 with the Celery executor. We aren't running anything on k8s infra. We already have reboots of our scheduler every 5m and that didn't help. The problem did eventually seem to go away on its own but we have no idea how long it was like this. 
   
   180 slots in the default_pool, 0 used, 180 queued. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hyungryuk edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hyungryuk edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-950523692


   I solved it!. 
   I think my case is slightly different with this problem.
   In my case, tasks are stuck in scheduled state, and not turned into ququed or running state.
   I fixed this with "**AIRFLOW__CORE__PARALLELISM**" env variable.
   i've tried fix my dag's CONCURRENCY and MAX_ACTIBE_RUNS config values, but it dosen't help.
   changing **AIRFLOW__CORE__PARALLELISM** env variable is helpful!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945386663


   > i got same issue here. in my case, after i resizing airflow worker pod resouece size. (smaller than before.) it happend!. i'm not sure but it might related with your worker pod resource size.. i guess..
   
   Couple questions to help you further:
   - What version are you using? 
   - What state are your tasks in?
   - What is your setup? Celery Redis etc?
   - Is this only with TaskInstances that were already present before you scaled down or also with new TaskInstances?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945785740


   I will unsubscribe from this issue.
   
   I have not encountered this issue again (Airflow 2.1.2).
   But the following circumstances made this issue pop up again:
   
   * An Airflow DAG/task is being scheduled (not queued yet)
   * The Airflow DAG code is being updated, but it contains an error, so that the scheduler cannot load the code and the task that starts up exits immediately
       * Now a rare condition takes place: A task is scheduled, but not yet executed.
       * The same task will boot a container
       * The same task will exit immediately, because the container loads the faulty code and crashes without bringing up the task at all.
       * No failure is recorded on the task.
   * Then the scheduler thinks the task is queued, but the task crashed immediately (using KubernetesExecutor)
   
   How do I prevent this issue?
   
   I simply make sure the DAG code is 100% clean and loads both in the scheduler and the tasks that start up (using KubernetesExecutor).
   
   How do I recover from this issue?
    
   First, I fix the issue that prevents the DAG code from loading. I restart the scheduler.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945785740


   I will unsubscribe from this issue.
   
   I have not encountered this issue again (Airflow 2.1.2).
   But the following circumstances made this issue pop up again:
   
   * An Airflow DAG/task is being scheduled (not queued yet)
   * The Airflow DAG code is being updated, but it contains an error, so that the scheduler cannot load the code and the task that starts up exits immediately
       * Now a rare condition takes place: A task is scheduled, but not yet executed.
       * The same task will boot a container
       * The same task will exit immediately, because the container loads the faulty code and crashes without bringing up the task at all.
       * No failure is recorded on the task.
   * Then the scheduler thinks the task is queued, but the task crashed immediately (using KubernetesExecutor)
       * This leads to queued slots filling up over time.
       * Once all queued slots of a pool (or the default pool) are filled with queued (but never executed, immediately crashing) tasks, the scheduler and the whole system gets stuck.
   
   How do I prevent this issue?
   
   I simply make sure the DAG code is 100% clean and loads both in the scheduler and the tasks that start up (using KubernetesExecutor).
   
   How do I recover from this issue?
    
   First, I fix the issue that prevents the DAG code from loading. Then I restart the scheduler.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1060718792


   > Hello, found same issue when i used ver 2.2.4 (latest) maybe we have some workaround for this things ?
   
   @haninp  - this might be (and likely is - because MWAA which plays a role here has no 2.2.4 support yet) completely different issue. It's not helpful to say "I also have similar problem"  without specifying details, logs . 
   
   As a "workaround" (or diagnosis) I suggest you to follow this FAQ here: https://airflow.apache.org/docs/apache-airflow/stable/faq.html?highlight=faq#why-is-task-not-getting-scheduled and double check if your problem is not one of those with the configuration that is explained there.
   
   If you find you stil have a problem, then I invite you to describe it in detail in a separate issue (if this is something that is easily reproducible) or GitHub Discussion (if you have a problem but unsure how to reproduce it). Providing as many details such as your deployment details, logs, circumstances etc. are **crucial** to be able to help you. Just stating "I also have this problem" helps no-one (including yourself because you **might** thiink you delegated the problem and it will be solved, but in fact this might be a completely different problem.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ddcatgg commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ddcatgg commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-983261840


   I ran into the same issue, the scheduler's log:
   ```
   [2021-12-01 11:45:11,850] {scheduler_job.py:1206} INFO - Executor reports execution of jira_pull_5_min.jira_pull execution_date=2021-12-01 03:40:00+00:00 exited with status success for try_number 1
   [2021-12-01 11:46:26,870] {scheduler_job.py:941} INFO - 1 tasks up for execution:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,871] {scheduler_job.py:975} INFO - Figuring out tasks to run in Pool(name=data_etl_daily_jobs_pool) with 10 open slots and 1 task instances ready to be queued
   [2021-12-01 11:46:26,871] {scheduler_job.py:1002} INFO - DAG data_etl_daily_jobs has 0/16 running and queued tasks
   [2021-12-01 11:46:26,871] {scheduler_job.py:1063} INFO - Setting the following tasks to queued state:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,873] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 2 and queue default
   [2021-12-01 11:46:26,873] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1)
   ```
   Stucked task: dwd.dwd_ti_lgc_project
   The restart of the executor does not do anything.
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ddcatgg edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ddcatgg edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-983261840


   I ran into the same issue, the scheduler's log:
   ```
   [2021-12-01 11:45:11,850] {scheduler_job.py:1206} INFO - Executor reports execution of jira_pull_5_min.jira_pull execution_date=2021-12-01 03:40:00+00:00 exited with status success for try_number 1
   [2021-12-01 11:46:26,870] {scheduler_job.py:941} INFO - 1 tasks up for execution:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,871] {scheduler_job.py:975} INFO - Figuring out tasks to run in Pool(name=data_etl_daily_jobs_pool) with 10 open slots and 1 task instances ready to be queued
   [2021-12-01 11:46:26,871] {scheduler_job.py:1002} INFO - DAG data_etl_daily_jobs has 0/16 running and queued tasks
   [2021-12-01 11:46:26,871] {scheduler_job.py:1063} INFO - Setting the following tasks to queued state:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,873] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 2 and queue default
   [2021-12-01 11:46:26,873] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1)
   ```
   Stucked task: dwd.dwd_ti_lgc_project
   The restart of the executor does not do anything.
   
   Version: v2.0.1
   Git Version:.release:2.0.1+beb8af5ac6c438c29e2c186145115fb1334a3735


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JavierLopezT edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1007235056






-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] danmactough commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
danmactough commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1032153919


   > Thre is also 2.2.2 as of recently. Can you please upgrade and check it there @danmactough ?
   
   Wow @potiuk I totally missed that update! Huge news! I'll check that out and see if it helps.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-774772238


   @kaxil first of all, sorry for my late reply. I am still active on this issue, just so you know. I have been quite busy unfortunately.
   
   You asked whether this might be git-sync related.
   I can state that it is not git-sync related.
   I observed this issue with local clusters (Dags loaded using Docker volume mounts) as well as with installations that pack the code into the container.
   
   Its a bit hard to track down precisely, and I could only ever see it when using KubernetesExecutor.
   If I ever observe it again, I'll try to reproduce it more precisely (currently that cluster is running a low volume of jobs regularly, therefore all is smooth sailing so far).
   
   The only way I could trigger it manually once (as state in my original post) was when I was clearing a large number of tasks at once.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] maxcountryman commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
maxcountryman commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-850527299


   I'm on Airflow `2.1.0` deployed to ECS Fargate--seeing similar issues where they are 128 slots, a small number used, but queued tasks that aren't ever run. Perhaps relatedly I also see a task in a `none` state which I'm not able to track down from the UI. Cluster resources seem fine, using less than 50% CPU and memory.
   
   <img width="1297" alt="Screen Shot 2021-05-28 at 9 15 32 AM" src="https://user-images.githubusercontent.com/74351/120013160-43980280-bf95-11eb-95ce-5c6196993f7b.png">
   
   <img width="746" alt="Screen Shot 2021-05-28 at 9 14 58 AM" src="https://user-images.githubusercontent.com/74351/120013097-2fec9c00-bf95-11eb-9e1a-bd062b8fba0c.png">
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] danmactough commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
danmactough commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1030645181


   > @val2k I may know why increasing the `try_number` didn't work. The DAG file that @danmactough posted is missing `session.commit()` after `session.merge()`. After the addition, the DAG file worked as intended for me.
   
   @jpkoponen At least in v2.0.2 (which is the only v2.x version available on AWS MWAA), there's no reason to call `session.commit()` when using the `@provide_session` decorator. It [creates the session](https://github.com/apache/airflow/blob/2.0.2/airflow/utils/session.py#L69), and [calls `session.commit()` for you](https://github.com/apache/airflow/blob/2.0.2/airflow/utils/session.py#L32).


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-841375186


   Once 2.1 is out/in RC (next week) I will take a look at this issue


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-824894652


   @ashb The setup works like this (I am @overbryd, just a different account.)
   
   * There are dynamic DAGs that are part of the repository, and that gets pulled in regularly with a side-car container on the scheduler.
   * The worker has an init container that pulls the latest state of the repository.
   * Secondly I have DAGs that are built (very quickly) based on database query results.
   
   This scenario opens up the following edge cases/problems:
   
   * A DAG can go missing, after it has been queued when the entry is removed from the database, rare but happens.
   * A DAG can go missing, after it has been queued when a git push happens, which change the DAGs, before the worker started.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nitinpandey-154 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
nitinpandey-154 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849396016


   Any solution or workaround to fix this? This makes the scheduler very unreliable and the tasks are stuck in the queued state.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder (which is saved in the file share -PVC, and can be viewed via azure storage explorer). 
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment.  during debug of this stuck situation, the table 'dag', 'task_*', 'celery_*' had been truncated.
   ```
   - celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule
   - connections
   ```
   
   How would like me to fetch the log? Just the dag log run? 
   update: the one i can view via storage explorer - it is empty, since the task was not executed. i will check the worker itself. and update this comment.
   
   there are no errors
   ```
   $ kubectl -n airflow get pods
   NAME                                              READY   STATUS    RESTARTS   AGE
   airflow-scheduler-55549d985f-dbw7b                1/1     Running   0          134m
   airflow-web-6c8467fd74-9zkj5                      1/1     Running   0          134m
   airflow-worker-0                                  1/1     Running   0          115m
   airflow-worker-1                                  1/1     Running   0          125m
   nginx-ingress-nginx-controller-6d5794678d-8zdsl   1/1     Running   0          5h16m
   nginx-ingress-nginx-controller-6d5794678d-fzhwb   1/1     Running   0          5h15m
   telegraf-54cd7f8578-fgdrn                         1/1     Running   0          5h16m
   DKCPHMAC137:instructions mishi$ kubectl -n airflow logs airflow-worker-0
   *** installing global extra pip packages...
   Collecting flask_oauthlib==0.9.6
     Downloading Flask_OAuthlib-0.9.6-py3-none-any.whl (40 kB)
   Collecting cachelib
     Downloading cachelib-0.1.1-py3-none-any.whl (13 kB)
   Collecting requests-oauthlib<1.2.0,>=0.6.2
     Downloading requests_oauthlib-1.1.0-py2.py3-none-any.whl (21 kB)
   Requirement already satisfied: Flask in /home/airflow/.local/lib/python3.7/site-packages (from flask_oauthlib==0.9.6) (1.1.2)
   Collecting oauthlib!=2.0.3,!=2.0.4,!=2.0.5,<3.0.0,>=1.1.2
     Downloading oauthlib-2.1.0-py2.py3-none-any.whl (121 kB)
   Requirement already satisfied: requests>=2.0.0 in /home/airflow/.local/lib/python3.7/site-packages (from requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (2.23.0)
   Requirement already satisfied: click>=5.1 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (6.7)
   Requirement already satisfied: Werkzeug>=0.15 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (0.16.1)
   Requirement already satisfied: itsdangerous>=0.24 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (1.1.0)
   Requirement already satisfied: Jinja2>=2.10.1 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (2.11.2)
   Requirement already satisfied: urllib3!=1.25.0,!=1.25.1,<1.26,>=1.21.1 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (1.25.11)
   Requirement already satisfied: certifi>=2017.4.17 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (2020.11.8)
   Requirement already satisfied: idna<3,>=2.5 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (2.8)
   Requirement already satisfied: chardet<4,>=3.0.2 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (3.0.4)
   Requirement already satisfied: MarkupSafe>=0.23 in /home/airflow/.local/lib/python3.7/site-packages (from Jinja2>=2.10.1->Flask->flask_oauthlib==0.9.6) (1.1.1)
   Installing collected packages: cachelib, oauthlib, requests-oauthlib, flask-oauthlib
     Attempting uninstall: oauthlib
       Found existing installation: oauthlib 3.1.0
       Uninstalling oauthlib-3.1.0:
         Successfully uninstalled oauthlib-3.1.0
     Attempting uninstall: requests-oauthlib
       Found existing installation: requests-oauthlib 1.3.0
       Uninstalling requests-oauthlib-1.3.0:
         Successfully uninstalled requests-oauthlib-1.3.0
   Successfully installed cachelib-0.1.1 flask-oauthlib-0.9.6 oauthlib-2.1.0 requests-oauthlib-1.1.0
   WARNING: You are using pip version 20.2.4; however, version 21.0.1 is available.
   You should consider upgrading via the '/usr/local/bin/python -m pip install --upgrade pip' command.
   *** running worker...
   [2021-04-16 16:03:50,914] {settings.py:233} DEBUG - Setting up DB connection pool (PID 7)
   [2021-04-16 16:03:50,914] {settings.py:300} DEBUG - settings.prepare_engine_args(): Using pool settings. pool_size=5, max_overflow=10, pool_recycle=1800, pid=7
   [2021-04-16 16:03:51,233] {sentry.py:179} DEBUG - Could not configure Sentry: No module named 'blinker', using DummySentry instead.
   [2021-04-16 16:03:51,286] {__init__.py:45} DEBUG - Cannot import  due to  doesn't look like a module path
   [2021-04-16 16:03:51,685] {cli_action_loggers.py:42} DEBUG - Adding <function default_action_log at 0x7f7cda6afd40> to pre execution callback
   The 'worker' command is deprecated and removed in Airflow 2.0, please use 'celery worker' instead
   [2021-04-16 16:03:52,608] {cli_action_loggers.py:68} DEBUG - Calling callbacks: [<function default_action_log at 0x7f7cda6afd40>]
   [2021-04-16 16:03:53,288: DEBUG/MainProcess] | Worker: Preparing bootsteps.
   [2021-04-16 16:03:53,293: DEBUG/MainProcess] | Worker: Building graph...
   [2021-04-16 16:03:53,293: DEBUG/MainProcess] | Worker: New boot order: {Timer, Hub, Pool, Autoscaler, StateDB, Beat, Consumer}
   [2021-04-16 16:03:53,316: DEBUG/MainProcess] | Consumer: Preparing bootsteps.
   [2021-04-16 16:03:53,317: DEBUG/MainProcess] | Consumer: Building graph...
   [2021-04-16 16:03:53,393: DEBUG/MainProcess] | Consumer: New boot order: {Connection, Events, Mingle, Gossip, Agent, Tasks, Control, Heart, event loop}
   [2021-04-16 16:03:53,427: DEBUG/MainProcess] | Worker: Starting Hub
   [2021-04-16 16:03:53,427: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:53,427: DEBUG/MainProcess] | Worker: Starting Pool
   [2021-04-16 16:03:53,749] {settings.py:233} DEBUG - Setting up DB connection pool (PID 17)
   [2021-04-16 16:03:53,750] {settings.py:300} DEBUG - settings.prepare_engine_args(): Using pool settings. pool_size=5, max_overflow=10, pool_recycle=1800, pid=17
   [2021-04-16 16:03:53,951] {sentry.py:179} DEBUG - Could not configure Sentry: No module named 'blinker', using DummySentry instead.
   [2021-04-16 16:03:53,998] {__init__.py:45} DEBUG - Cannot import  due to  doesn't look like a module path
   [2021-04-16 16:03:54,013: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:54,013: DEBUG/MainProcess] | Worker: Starting Autoscaler
   [2021-04-16 16:03:54,014: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:54,014: DEBUG/MainProcess] | Worker: Starting Consumer
   [2021-04-16 16:03:54,014: DEBUG/MainProcess] | Consumer: Starting Connection
   [2021-04-16 16:03:54,081: INFO/MainProcess] Connected to redis://redis:**@redis-airflow-osweu-dev.redis.cache.windows.net:6379//
   [2021-04-16 16:03:54,081: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:54,081: DEBUG/MainProcess] | Consumer: Starting Events
   [2021-04-16 16:03:54,101: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:54,101: DEBUG/MainProcess] | Consumer: Starting Mingle
   [2021-04-16 16:03:54,102: INFO/MainProcess] mingle: searching for neighbors
   [2021-04-16 16:03:54,328] {cli_action_loggers.py:42} DEBUG - Adding <function default_action_log at 0x7fb8a5c12290> to pre execution callback
   [2021-04-16 16:03:54,603] {cli_action_loggers.py:68} DEBUG - Calling callbacks: [<function default_action_log at 0x7fb8a5c12290>]
   Starting flask
    * Serving Flask app "airflow.bin.cli" (lazy loading)
    * Environment: production
      WARNING: This is a development server. Do not use it in a production deployment.
      Use a production WSGI server instead.
    * Debug mode: off
   [2021-04-16 16:03:54,954] {_internal.py:122} INFO -  * Running on http://0.0.0.0:8793/ (Press CTRL+C to quit)
   [2021-04-16 16:03:55,194: INFO/MainProcess] mingle: sync with 1 nodes
   [2021-04-16 16:03:55,195: DEBUG/MainProcess] mingle: processing reply from celery@airflow-worker-1
   [2021-04-16 16:03:55,195: INFO/MainProcess] mingle: sync complete
   [2021-04-16 16:03:55,195: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:55,195: DEBUG/MainProcess] | Consumer: Starting Gossip
   [2021-04-16 16:03:55,229: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:55,230: DEBUG/MainProcess] | Consumer: Starting Tasks
   [2021-04-16 16:03:55,240: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:55,240: DEBUG/MainProcess] | Consumer: Starting Control
   [2021-04-16 16:03:55,266: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:55,267: DEBUG/MainProcess] | Consumer: Starting Heart
   [2021-04-16 16:03:55,280: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 16:03:55,280: DEBUG/MainProcess] | Consumer: Starting event loop
   [2021-04-16 16:03:55,281: DEBUG/MainProcess] | Worker: Hub.register Autoscaler...
   [2021-04-16 16:03:55,281: DEBUG/MainProcess] | Worker: Hub.register Pool...
   [2021-04-16 16:03:55,281: INFO/MainProcess] celery@airflow-worker-0 ready.
   [2021-04-16 16:03:55,281: DEBUG/MainProcess] basic.qos: prefetch_count->8
   [2021-04-16 16:03:56,928: DEBUG/MainProcess] celery@airflow-worker-1 joined the party
   
   
   **$ kubectl -n airflow logs airflow-worker-1**
   *** installing global extra pip packages...
   Collecting flask_oauthlib==0.9.6
     Downloading Flask_OAuthlib-0.9.6-py3-none-any.whl (40 kB)
   Collecting requests-oauthlib<1.2.0,>=0.6.2
     Downloading requests_oauthlib-1.1.0-py2.py3-none-any.whl (21 kB)
   Collecting oauthlib!=2.0.3,!=2.0.4,!=2.0.5,<3.0.0,>=1.1.2
     Downloading oauthlib-2.1.0-py2.py3-none-any.whl (121 kB)
   Collecting cachelib
     Downloading cachelib-0.1.1-py3-none-any.whl (13 kB)
   Requirement already satisfied: Flask in /home/airflow/.local/lib/python3.7/site-packages (from flask_oauthlib==0.9.6) (1.1.2)
   Requirement already satisfied: requests>=2.0.0 in /home/airflow/.local/lib/python3.7/site-packages (from requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (2.23.0)
   Requirement already satisfied: itsdangerous>=0.24 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (1.1.0)
   Requirement already satisfied: Jinja2>=2.10.1 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (2.11.2)
   Requirement already satisfied: Werkzeug>=0.15 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (0.16.1)
   Requirement already satisfied: click>=5.1 in /home/airflow/.local/lib/python3.7/site-packages (from Flask->flask_oauthlib==0.9.6) (6.7)
   Requirement already satisfied: certifi>=2017.4.17 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (2020.11.8)
   Requirement already satisfied: idna<3,>=2.5 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (2.8)
   Requirement already satisfied: urllib3!=1.25.0,!=1.25.1,<1.26,>=1.21.1 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (1.25.11)
   Requirement already satisfied: chardet<4,>=3.0.2 in /home/airflow/.local/lib/python3.7/site-packages (from requests>=2.0.0->requests-oauthlib<1.2.0,>=0.6.2->flask_oauthlib==0.9.6) (3.0.4)
   Requirement already satisfied: MarkupSafe>=0.23 in /home/airflow/.local/lib/python3.7/site-packages (from Jinja2>=2.10.1->Flask->flask_oauthlib==0.9.6) (1.1.1)
   Installing collected packages: oauthlib, requests-oauthlib, cachelib, flask-oauthlib
     Attempting uninstall: oauthlib
       Found existing installation: oauthlib 3.1.0
       Uninstalling oauthlib-3.1.0:
         Successfully uninstalled oauthlib-3.1.0
     Attempting uninstall: requests-oauthlib
       Found existing installation: requests-oauthlib 1.3.0
       Uninstalling requests-oauthlib-1.3.0:
         Successfully uninstalled requests-oauthlib-1.3.0
   Successfully installed cachelib-0.1.1 flask-oauthlib-0.9.6 oauthlib-2.1.0 requests-oauthlib-1.1.0
   WARNING: You are using pip version 20.2.4; however, version 21.0.1 is available.
   You should consider upgrading via the '/usr/local/bin/python -m pip install --upgrade pip' command.
   *** running worker...
   [2021-04-16 15:53:39,557] {settings.py:233} DEBUG - Setting up DB connection pool (PID 6)
   [2021-04-16 15:53:39,558] {settings.py:300} DEBUG - settings.prepare_engine_args(): Using pool settings. pool_size=5, max_overflow=10, pool_recycle=1800, pid=6
   [2021-04-16 15:53:39,926] {sentry.py:179} DEBUG - Could not configure Sentry: No module named 'blinker', using DummySentry instead.
   [2021-04-16 15:53:39,982] {__init__.py:45} DEBUG - Cannot import  due to  doesn't look like a module path
   [2021-04-16 15:53:40,446] {cli_action_loggers.py:42} DEBUG - Adding <function default_action_log at 0x7f424221bd40> to pre execution callback
   The 'worker' command is deprecated and removed in Airflow 2.0, please use 'celery worker' instead
   [2021-04-16 15:53:41,458] {cli_action_loggers.py:68} DEBUG - Calling callbacks: [<function default_action_log at 0x7f424221bd40>]
   [2021-04-16 15:53:42,387: DEBUG/MainProcess] | Worker: Preparing bootsteps.
   [2021-04-16 15:53:42,393: DEBUG/MainProcess] | Worker: Building graph...
   [2021-04-16 15:53:42,393: DEBUG/MainProcess] | Worker: New boot order: {StateDB, Beat, Timer, Hub, Pool, Autoscaler, Consumer}
   [2021-04-16 15:53:42,418: DEBUG/MainProcess] | Consumer: Preparing bootsteps.
   [2021-04-16 15:53:42,419: DEBUG/MainProcess] | Consumer: Building graph...
   [2021-04-16 15:53:42,517: DEBUG/MainProcess] | Consumer: New boot order: {Connection, Events, Mingle, Gossip, Heart, Tasks, Control, Agent, event loop}
   [2021-04-16 15:53:42,556: DEBUG/MainProcess] | Worker: Starting Hub
   [2021-04-16 15:53:42,556: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:42,556: DEBUG/MainProcess] | Worker: Starting Pool
   [2021-04-16 15:53:42,904] {settings.py:233} DEBUG - Setting up DB connection pool (PID 16)
   [2021-04-16 15:53:42,905] {settings.py:300} DEBUG - settings.prepare_engine_args(): Using pool settings. pool_size=5, max_overflow=10, pool_recycle=1800, pid=16
   [2021-04-16 15:53:43,085] {sentry.py:179} DEBUG - Could not configure Sentry: No module named 'blinker', using DummySentry instead.
   [2021-04-16 15:53:43,144] {__init__.py:45} DEBUG - Cannot import  due to  doesn't look like a module path
   [2021-04-16 15:53:43,172: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:43,172: DEBUG/MainProcess] | Worker: Starting Autoscaler
   [2021-04-16 15:53:43,172: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:43,172: DEBUG/MainProcess] | Worker: Starting Consumer
   [2021-04-16 15:53:43,173: DEBUG/MainProcess] | Consumer: Starting Connection
   [2021-04-16 15:53:43,253: INFO/MainProcess] Connected to redis://redis:**@redis-airflow-osweu-dev.redis.cache.windows.net:6379//
   [2021-04-16 15:53:43,253: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:43,253: DEBUG/MainProcess] | Consumer: Starting Events
   [2021-04-16 15:53:43,300: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:43,300: DEBUG/MainProcess] | Consumer: Starting Mingle
   [2021-04-16 15:53:43,300: INFO/MainProcess] mingle: searching for neighbors
   [2021-04-16 15:53:43,533] {cli_action_loggers.py:42} DEBUG - Adding <function default_action_log at 0x7fe2c52ec290> to pre execution callback
   [2021-04-16 15:53:43,860] {cli_action_loggers.py:68} DEBUG - Calling callbacks: [<function default_action_log at 0x7fe2c52ec290>]
   Starting flask
    * Serving Flask app "airflow.bin.cli" (lazy loading)
    * Environment: production
      WARNING: This is a development server. Do not use it in a production deployment.
      Use a production WSGI server instead.
    * Debug mode: off
   [2021-04-16 15:53:44,132] {_internal.py:122} INFO -  * Running on http://0.0.0.0:8793/ (Press CTRL+C to quit)
   [2021-04-16 15:53:44,382: INFO/MainProcess] mingle: sync with 1 nodes
   [2021-04-16 15:53:44,383: DEBUG/MainProcess] mingle: processing reply from celery@airflow-worker-0
   [2021-04-16 15:53:44,383: INFO/MainProcess] mingle: sync complete
   [2021-04-16 15:53:44,383: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:44,383: DEBUG/MainProcess] | Consumer: Starting Gossip
   [2021-04-16 15:53:44,413: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:44,414: DEBUG/MainProcess] | Consumer: Starting Heart
   [2021-04-16 15:53:44,431: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:44,431: DEBUG/MainProcess] | Consumer: Starting Tasks
   [2021-04-16 15:53:44,439: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:44,440: DEBUG/MainProcess] | Consumer: Starting Control
   [2021-04-16 15:53:44,473: DEBUG/MainProcess] ^-- substep ok
   [2021-04-16 15:53:44,473: DEBUG/MainProcess] | Consumer: Starting event loop
   [2021-04-16 15:53:44,473: DEBUG/MainProcess] | Worker: Hub.register Autoscaler...
   [2021-04-16 15:53:44,473: DEBUG/MainProcess] | Worker: Hub.register Pool...
   [2021-04-16 15:53:44,474: INFO/MainProcess] celery@airflow-worker-1 ready.
   [2021-04-16 15:53:44,474: DEBUG/MainProcess] basic.qos: prefetch_count->8
   [2021-04-16 15:53:45,888: DEBUG/MainProcess] celery@airflow-worker-0 joined the party
   [2021-04-16 16:03:34,318: DEBUG/MainProcess] celery@airflow-worker-0 left
   [2021-04-16 16:03:54,170: DEBUG/MainProcess] pidbox received method hello(from_node='celery@airflow-worker-0', revoked={}) [reply_to:{'exchange': 'reply.celery.pidbox', 'routing_key': '86623118-5415-3478-a9be-f48f1670b61b'} ticket:07d58af6-f3f2-4404-a926-94eab1e2955c]
   [2021-04-16 16:03:54,170: INFO/MainProcess] sync with celery@airflow-worker-0
   [2021-04-16 16:03:55,281: DEBUG/MainProcess] celery@airflow-worker-0 joined the party
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] trucnguyenlam edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
trucnguyenlam edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-883251087


   @kaxil we're still experiencing this issue in version 2.1.1 even after tuning `parallelism` and `pool` size to 1024


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JD-V commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JD-V commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-997599254


   I am facing the same issue in airflow-2.2.2 with kubernetes Executor. 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945785740


   I will unsubscribe from this issue.
   
   I have not encountered this issue again (Airflow 2.1.2).
   But the following circumstances made this issue pop up again:
   
   * An Airflow DAG/task is being scheduled (not queued yet)
   * The Airflow DAG code is being updated, but it contains an error, so that the scheduler cannot load the code and the task that starts up exits immediately
       * Now a rare condition takes place: A task is scheduled, but not yet executed.
       * The same task will boot a container
       * The same task will exit immediately, because the container loads the faulty code and crashes without bringing up the task at all.
       * No failure is recorded on the task.
   * Then the scheduler thinks the task is queued, but the task crashed immediately (using KubernetesExecutor)
       * This leads to queued slots filling up over time.
       * Once all queued slots of a pool (or the default pool) are filled with queued (but never executed, immediately crashing) tasks, the scheduler and the whole system gets stuck.
   
   How do I prevent this issue?
   
   I simply make sure the DAG code is 100% clean and loads both in the scheduler and the tasks that start up (using KubernetesExecutor).
   
   How do I recover from this issue?
    
   First, I fix the issue that prevents the DAG code from loading. I restart the scheduler.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] trucnguyenlam commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
trucnguyenlam commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-883251087


   @kaxil we're still experiencing this issue in version 2.1.1 even after tuning `parallelism` and `pool` size


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614


   I am also noticing this issue with my Airflow v2.1.2 instance, where the task is forever in queued state, and in the scheduler log I see 
   ```
   airflow-scheduler1_1  | [2021-08-04 12:45:23,286] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   ```
   
   When I sent the SIGUSR2 signal to scheduler I recived:
   
   ```
   airflow-scheduler1_1  | SIGUSR2 received, printing debug
   airflow-scheduler1_1  | --------------------------------------------------------------------------------
   airflow-scheduler1_1  | [2021-08-04 13:07:37,230] {base_executor.py:305} INFO - executor.queued (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,231] {base_executor.py:307} INFO - executor.running (1)
   airflow-scheduler1_1  | 	TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,233] {base_executor.py:311} INFO - executor.event_buffer (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,234] {celery_executor.py:372} INFO - executor.tasks (1)
   airflow-scheduler1_1  | 	(TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 1ed20983-dbd3-47c6-8404-345fa0fb75ff>)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,235] {celery_executor.py:377} INFO - executor.adopted_task_timeouts (0)
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jbkc85 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jbkc85 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-897078856


   > hi @jbkc85 thank you for the suggestion! May I know how to create different pools in Airflow ? many thanks!
   
   Pools are made through the Admin interface.  Then you can reference them in the DAGs: https://airflow.apache.org/docs/apache-airflow/stable/concepts/pools.html.
   
   Its a bit of overhead, but once set up you have a little more control over which lanes fill up and which ones don't.  Make sure to read about it before implementing!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay removed a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay removed a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614


   I am also noticing this issue with my Airflow v2.1.2 instance, where the task is forever in queued state, and in the scheduler log I see 
   ```
   airflow-scheduler1_1  | [2021-08-04 12:45:23,286] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   ```
   
   When I sent the SIGUSR2 signal to scheduler I received:
   
   ```
   airflow-scheduler1_1  | SIGUSR2 received, printing debug
   airflow-scheduler1_1  | --------------------------------------------------------------------------------
   airflow-scheduler1_1  | [2021-08-04 13:07:37,230] {base_executor.py:305} INFO - executor.queued (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,231] {base_executor.py:307} INFO - executor.running (1)
   airflow-scheduler1_1  | 	TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,233] {base_executor.py:311} INFO - executor.event_buffer (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,234] {celery_executor.py:372} INFO - executor.tasks (1)
   airflow-scheduler1_1  | 	(TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 1ed20983-dbd3-47c6-8404-345fa0fb75ff>)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,235] {celery_executor.py:377} INFO - executor.adopted_task_timeouts (0)
   ```
   
   So basically it is trying to queue a running task ? and failing the `if` check 
   ```
           if task_instance.key not in self.queued_tasks and task_instance.key not in self.running:
               self.log.info("Adding to queue: %s", command)
               self.queued_tasks[task_instance.key] = (command, priority, queue, task_instance)
           else:
               self.log.error("could not queue task %s", task_instance.key)
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hafid-d commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hafid-d commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-896144592


   Same here, facing the issue. Even restarting the scheduler doesn't fix the problem. My dags are still stuck in running state


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder (which is saved in the file share -PVC, and can be viewed via azure storage explorer). 
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment.  during debug of this stuck situation, the table 'dag', 'task_*', 'celery_*' had been truncated.
   ```
   - celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule
   - connections
   ```
   
   How would like me to fetch the log? Just the dag log run? 
   update: the one i can view via storage explorer - it is empty, since the task was not executed. i will check the worker itself. and update this comment.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JavierLopezT commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1007235056


   This has started happening to us as well. Randomly (3 nights over 10 days), all tasks get stuck in queued state and we have to manually clear the state for it to recover. We have seen that it is related to the decrease of this metric in datadog:
   
   ![image](https://user-images.githubusercontent.com/11339132/148517399-95681f0c-260e-48cb-9b38-b7d0556d8df9.png)
   
   Airflow 2.1.1 in EC2
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ddcatgg edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ddcatgg edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-983261840


   I ran into the same issue, the scheduler's log:
   ```
   [2021-12-01 11:45:11,850] {scheduler_job.py:1206} INFO - Executor reports execution of jira_pull_5_min.jira_pull execution_date=2021-12-01 03:40:00+00:00 exited with status success for try_number 1
   [2021-12-01 11:46:26,870] {scheduler_job.py:941} INFO - 1 tasks up for execution:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,871] {scheduler_job.py:975} INFO - Figuring out tasks to run in Pool(name=data_etl_daily_jobs_pool) with 10 open slots and 1 task instances ready to be queued
   [2021-12-01 11:46:26,871] {scheduler_job.py:1002} INFO - DAG data_etl_daily_jobs has 0/16 running and queued tasks
   [2021-12-01 11:46:26,871] {scheduler_job.py:1063} INFO - Setting the following tasks to queued state:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,873] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 2 and queue default
   [2021-12-01 11:46:26,873] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1)
   ```
   Stucked task: dwd.dwd_ti_lgc_project
   The restart of the scheduler, webserver and executor does not do anything.
   
   After clear state of the task, the executor does not received the task.
   
   Version: v2.0.1
   Git Version:.release:2.0.1+beb8af5ac6c438c29e2c186145115fb1334a3735


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-823225516


   @lukas-at-harren @pelaprat  Can you test it with recently released 2.0.2 release please


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] MatrixManAtYrService edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
MatrixManAtYrService edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849031715


   While trying to recreate this, I wrote a [stress test](https://github.com/MatrixManAtYrService/airflow-git-sync/blob/master/scheduler_stress.py) which I ran overnight on my local microk8s cluster (release:2.0.1+beb8af5ac6c438c29e2c186145115fb1334a3735 configured like [this](https://github.com/MatrixManAtYrService/airflow-git-sync/blob/master/zsh.stdin)).
   
   I was hoping that it would get fully stuck by the time I woke.  Instead there were only two stuck tasks: 
    
   ![stucktasks](https://user-images.githubusercontent.com/5834582/119712787-34db0f80-be1e-11eb-9a41-875dd88c0566.gif)
   
   Deleting the scheduler pod and letting kubernetes recreate it caused the two stuck tasks to complete.   At about 1:00 PM I cleared the state of all previous tasks.  For a little while, the scheduler managed to both backfill the cleared tasks and keep up with scheduled runs, but then something happened that caused most of the tasks to get stuck.
   
   <img width="825" alt="Screen Shot 2021-05-26 at 9 58 19 PM" src="https://user-images.githubusercontent.com/5834582/119764748-34b73000-be6f-11eb-99b0-c481905db56b.png">
   
   Things were still limping along after that, but I never again saw more than three tasks running at once. This time, resetting the scheduler pod did **not** remedy the situation--it just resumed to its prior anemic state.  Here's a dump of the database and a snapshot of the scheduler logs right after a restart:
   
   [db_and_scheduler_logs.tar.gz](https://github.com/apache/airflow/files/6551081/db_and_scheduler_logs.tar.gz)
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-762456811


   @Overbryd Did the suggestion in above comment help?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821292708


   @minnieshi Do you have access to logs from the worker tasks? If so, can you include any errors there?
   
   @lukas-at-harren Well done! The second part of your proposed solution "The worker must fail properly (with its task ending in a "failed" state) when he cannot find the DAG + task he was tasked with" would apply to my CeleryExecutor problem as well, given my workers were timing out attempting to find their DAGs.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] danmactough commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
danmactough commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1047306527


   > @danmactough Did this update help? We're tracking this issue before upgrading.
   
   @WattsInABox Unfortunately, the AWS MWAA "upgrade" path is not an upgrade path, but rather a "stand up a new cluster and move all your DAGs with no way to retain historical Airflow metadata" path. Which ok fine, maybe that's what we should do in any case, but it does mean testing out 2.2.2 is going to require some planning.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lfreina commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lfreina commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849769325


   I am having more or less the same issue with Kubernetes Executor. I sent the SIGUSR2 to the scheduler and got this:
   
   ```
   [2021-05-27 16:15:55,628] {kubernetes_executor.py:208} INFO - Event: tutorialtestrunthisfirst.ef1f345cc84b46beb4863e1e1a6121f8 is Running
   [2021-05-27 16:19:15,850] {scheduler_job.py:759} INFO - --------------------------------------------------------------------------------
   SIGUSR2 received, printing debug
   --------------------------------------------------------------------------------
   [2021-05-27 16:19:15,851] {base_executor.py:305} INFO - executor.queued (0)
   	
   [2021-05-27 16:19:15,851] {base_executor.py:307} INFO - executor.running (2)
   	TaskInstanceKey(dag_id='tutorial_test', task_id='run_this_first', execution_date=datetime.datetime(2021, 5, 26, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='hello_world', task_id='hello_task', execution_date=datetime.datetime(2021, 5, 26, 12, 0, tzinfo=Timezone('UTC')), try_number=1)
   [2021-05-27 16:19:15,851] {base_executor.py:311} INFO - executor.event_buffer (0)
   	
   [2021-05-27 16:19:15,851] {scheduler_job.py:762} INFO - --------------------------------------------------------------------------------
   ```
   
   Not much info there. I am using a simple DAG here. The task just gets stuck in queued state but nothing happens. Maybe I am doing something wrong but no error is shown and this might point out the problem we are all experiencing here. 
   ```
   from datetime import datetime
   from airflow import DAG
   
   # For airflow 2.1.0
   from airflow.operators.python import PythonOperator
   from airflow.operators.dummy import DummyOperator
   
   # For airflow 1.10.10
   #from airflow.operators.dummy_operator import DummyOperator
   #from airflow.operators.python_operator import PythonOperator
   
   
   def print_hello(**kwargs):
       return 'Hello world!'
   
   
   def print_goodbye(**kwargs):
       return 'Goodbye world'
   
   
   dag = DAG('hello_world', description='Simple tutorial DAG',
             schedule_interval='0 12 * * *',
             start_date=datetime(2021, 3, 20), 
             catchup=False)
   
   dummy_operator = DummyOperator(task_id='dummy_task', retries=3, dag=dag)
   
   hello_operator = PythonOperator(task_id='hello_task', python_callable=print_hello, dag=dag)
   
   goodbye_operator = PythonOperator(task_id='print_goodbye', python_callable=print_goodbye, dag=dag)
   
   dummy_operator >> hello_operator >> goodbye_operator
   ```
   
   I did test this DAG in minikube in a similar setup as I did for issue #15305. This DAG works with airflow 1.10.10 but does not with 2.1.0. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jonathonbattista edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jonathonbattista edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-770431504


   We are experiencing these symptoms on `1.10.14` as well. Doesn't seem `git-sync` related.
   
   `120` tasks are supposedly running with`0` slots are open, but nothing is spawning.
   
   Does any work have a clue why this is happening?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817750364


   Working out a solution that would check the container status when the pod is pending


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-875932438


   > We're only making use of the `default_pool` and tasks are moving from `Queued` to `Running` there but aren't entering the queue despite being in a `Scheduled` state.
   
   
   We had a similar issue and increased the `default_pool` 10x, from 128 to 1000. This fixed it completely for us, however, we are using Celery Executor though. 
   Did you already attempt something similar? Wondering if this helps in your case as well. 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812742063


   I ran into this issue due to the scheduler over-utilizing CPU because our `min_file_process_interval` was set to 0 (the default prior to 2.0), which in airflow 2.0 causes 100% CPU utilization by constantly pinging for DAG files. Setting this parameter to 60 fixed the issue.
   The stack I observed this on:
   host: AWS ECS Cluster
   executor: CeleryExecutor
   queue: AWS SQS Queue
   
   The behavior I observed was that the scheduler would mark tasks are "queued", but never actually send them to the queue (I think the scheduler does actual queueing via the executor). My manual workaround until correcting the `min_file_process_interval` param was to stop the scheduler, clear queued tasks, and then start a new scheduler. The new scheduler would temporarily properly send tasks to the queue, before degenerating to marking tasks as queued without sending to the queue.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] grepthat commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
grepthat commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-774974042


   I'm seeing this behaviour as well. I could not reliably reproduce it, but my experience matches that of @Overbryd - especially when clearing many tasks that can run in parallel. I noticed that the indefinitely queued tasks produce an error in the scheduler log:
   
   ```
   [2021-01-20 17:17:52,393] {base_executor.py:82} ERROR - could not queue task ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2)
   ```
   
   Looking further at the log I notice that, the task gets processed normally first, but then gets picked up again leading to the mentioned error.
   ```
   [2021-01-20 17:16:18,454] {scheduler_job.py:1100} INFO - Sending ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2) to executor with priority 1 and queue celery
   [2021-01-20 17:16:18,454] {base_executor.py:79} INFO - Adding to queue: ['airflow', 'tasks', 'run', '==dag-id==', '==task-id==', '2021-01-12T17:39:37.661475+00:00', '--local', '--pool', 'default_pool', '--subdir', '/opt/airflow/dags/dag.py']
   [2021-01-20 17:16:18,469] {kubernetes_executor.py:532} INFO - Add task ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2) with command [] with executor_config {}
   [2021-01-20 17:16:18,498] {kubernetes_executor.py:299} INFO - Kubernetes job is (==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2)
   [2021-01-20 17:16:25,158] {kubernetes_executor.py:568} INFO - Changing state of (==TaskInstanceKey== tzinfo=tzutc()), try_number=2), None, '==pod-id==', 'airflow', '8928202') to None
   [2021-01-20 17:16:25,192] {kubernetes_executor.py:617} INFO - Deleted pod: ==TaskInstanceKey== tzinfo=tzutc()), try_number=2) in namespace airflow
   [2021-01-20 17:16:25,357] {kubernetes_executor.py:568} INFO - Changing state of (==TaskInstanceKey== tzinfo=tzutc()), try_number=2), None, '==pod-id==', 'airflow', '8928203') to None
   [2021-01-20 17:16:25,363] {kubernetes_executor.py:617} INFO - Deleted pod: ==TaskInstanceKey== tzinfo=tzutc()), try_number=2) in namespace airflow
   [2021-01-20 17:16:25,364] {kubernetes_executor.py:568} INFO - Changing state of (==TaskInstanceKey== tzinfo=tzutc()), try_number=2), None, '==pod-id==', 'airflow', '8928204') to None
   [2021-01-20 17:16:25,369] {kubernetes_executor.py:617} INFO - Deleted pod: ==TaskInstanceKey== tzinfo=tzutc()), try_number=2) in namespace airflow
   # ⬇ task gets picked up again
   [2021-01-20 17:17:30,649] {scheduler_job.py:1100} INFO - Sending ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2) to executor with priority 1 and queue celery
   [2021-01-20 17:17:30,649] {base_executor.py:79} INFO - Adding to queue: ['airflow', 'tasks', 'run', '==dag-id==', '==task-id==', '2021-01-12T17:39:37.661475+00:00', '--local', '--pool', 'default_pool', '--subdir', '/opt/airflow/dags/dag.py']
   [2021-01-20 17:17:30,655] {kubernetes_executor.py:532} INFO - Add task ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2) with command 
   [2021-01-20 17:17:30,658] {kubernetes_executor.py:299} INFO - Kubernetes job is (==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2), 
   [2021-01-20 17:17:52,389] {scheduler_job.py:1058} INFO - Setting the following tasks to queued state:
       <TaskInstance: ==TaskInstanceKey== 2021-01-12 17:39:37.661475+00:00 [scheduled]>
   [2021-01-20 17:17:52,392] {scheduler_job.py:1100} INFO - Sending ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2) to executor with priority 1 and queue celery
   [2021-01-20 17:17:52,393] {base_executor.py:82} ERROR - could not queue task ==TaskInstanceKey== tzinfo=Timezone('UTC')), try_number=2)
   ```
   
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817118815


   I observed this again last night and the reason was image pull error inside the pods. 
   
   When there's image pull error, the pod is still in 'Pending' phase and airflow sees it as such therefore tasks stay queued. If you don't do anything, after a very long time, the tasks will be cleared and the dag marked as failed.
   
   The kubernetes pod watcher does not watch the containers inside the pods. It watches only the pods and report the status as pending therefore tasks are queued.
   
   After much digging, the only solution I can think of is using timeout to delete the pods because I couldn't find a way we could watch the containers inside the pods and report the container status.
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817718310


   @ephraimbuddy ok. I have now set `AIRFLOW__KUBERNETES__DELETE_WORKER_PODS` to `False`. Let me see if it triggers again.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] trucnguyenlam commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
trucnguyenlam commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-883251087


   @kaxil we're still experiencing this issue in version 2.1.1 even after tuning `parallelism` and `pool` size


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jpkoponen commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jpkoponen commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1031198456


   > > @val2k I may know why increasing the `try_number` didn't work. The DAG file that @danmactough posted is missing `session.commit()` after `session.merge()`. After the addition, the DAG file worked as intended for me.
   > 
   > @jpkoponen At least in v2.0.2 (which is the only v2.x version available on AWS MWAA), there's no reason to call `session.commit()` when using the `@provide_session` decorator. It [creates the session](https://github.com/apache/airflow/blob/2.0.2/airflow/utils/session.py#L69), and [calls `session.commit()` for you](https://github.com/apache/airflow/blob/2.0.2/airflow/utils/session.py#L32).
   
   Okay, I didn't know that. In that case, I'm confused why the `try_number` increased with `commit()` but didn't without it. 🤷 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JavierLopezT edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1007235056


   This has started happening to us as well. Randomly (3 nights over 10), all tasks get stuck in queued state and we have to manually clear the state for it to recover. We have seen that it is related to the decline of this metric in datadog:
   
   ![Captura de pantalla 2022-01-07 a las 10 01 55](https://user-images.githubusercontent.com/11339132/148519482-eb52b494-7b76-4029-b50a-b45065b4e485.png)
   
   
   
   Airflow 2.1.1 in EC2 with LocalExecutor
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JavierLopezT edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1007235056


   This has started happening to us as well. Randomly (3 nights over 10), all tasks get stuck in queued state and we have to manually clear the state for it to recover. We have seen that it is related to the decline of this metric in datadog:
   
   ![Captura de pantalla 2022-01-07 a las 10 01 55](https://user-images.githubusercontent.com/11339132/148519482-eb52b494-7b76-4029-b50a-b45065b4e485.png)
   
   
   
   Airflow 2.1.1 in EC2
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] cdibble commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
cdibble commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1020844854


   > 
   
   I am having a similar experience. I'm on `Airflow v2.2.2` with `KubernetesExecutor` in AWS EKS. I observe when a DAG run kicks off, tasks are quickly queued, but some relatively small fraction of the queued tasks are moved to a running state. At the same time, kubernetes reports `FailedCreatePodSandBox` on tasks that are in the queued state. My scheduler has plenty of resources, but I see the `OSError: handle is closed` error coming up in the scheduler logs right around when queued tasks stop being moved to the running state. Those tasks stay queued until some of the already-executing tasks complete. In my case, those tasks will eventually be set to running and complete, but I can never seem to get the concurrency I want. There's plenty of room in the pool, and I am fairly certain I am not limited by the various concurrency/parallelism settings.
   
   Below are the scheduler logs when I get a similar `OSError` as @danmactough as well as the kubectl events logs from the time when a new DAG run was executing and tasks were moving from scheduled to queued to executing. 
   
   scheduler logs
   ```
   [2022-01-25 05:49:54,253] {kubernetes_executor.py:147} INFO - Event: <redacted_A> had an event of type MODIFIED
   [2022-01-25 05:49:54,253] {kubernetes_executor.py:213} INFO - Event: <redacted_A> is Running
   [2022-01-25 05:49:54,654] {kubernetes_executor.py:147} INFO - Event: <redacted_B> had an event of type MODIFIED
   [2022-01-25 05:49:54,654] {kubernetes_executor.py:213} INFO - Event: <redacted_B> is Running
   [2022-01-25 05:49:55,054] {kubernetes_executor.py:147} INFO - Event: <redacted_C> had an event of type MODIFIED
   [2022-01-25 05:49:55,054] {kubernetes_executor.py:201} INFO - Event: <redacted_C> Pending
   [2022-01-25 05:49:55,653] {kubernetes_executor.py:147} INFO - Event: <redacted_D> had an event of type MODIFIED
   [2022-01-25 05:49:55,653] {kubernetes_executor.py:213} INFO - Event: <redacted_D> is Running
   [2022-01-25 05:50:06,704] {kubernetes_executor.py:454} INFO - Found 15 queued task instances
   [2022-01-25 05:50:45,076] {processor.py:244} WARNING - Killing DAGFileProcessorProcess (PID=12643)
   Process ForkProcess-22:
   Traceback (most recent call last):
     File "/usr/local/lib/python3.8/multiprocessing/process.py", line 315, in _bootstrap
       self.run()
     File "/usr/local/lib/python3.8/multiprocessing/process.py", line 108, in run
       self._target(*self._args, **self._kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 277, in _run_processor_manager
       processor_manager.start()
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 510, in start
       return self._run_parsing_loop()
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 570, in _run_parsing_loop
       self._collect_results_from_processor(processor)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 894, in _collect_results_from_processor
       if processor.result is not None:
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/processor.py", line 322, in result
       if not self.done:
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/processor.py", line 287, in done
       if self._parent_channel.poll():
     File "/usr/local/lib/python3.8/multiprocessing/connection.py", line 255, in poll
       self._check_closed()
     File "/usr/local/lib/python3.8/multiprocessing/connection.py", line 136, in _check_closed
       raise OSError("handle is closed")
   OSError: handle is closed
   [2022-01-25 05:50:45,858] {manager.py:308} WARNING - DagFileProcessorManager (PID=12136) exited with exit code 1 - re-launching
   [2022-01-25 05:50:45,861] {manager.py:163} INFO - Launched DagFileProcessorManager with pid: 12873
   [2022-01-25 05:50:45,868] {settings.py:52} INFO - Configured default timezone Timezone('UTC')
   [2022-01-25 05:50:45,870] {settings.py:462} INFO - Loaded airflow_local_settings from /opt/airflow/config/airflow_local_settings.py .
   [2022-01-25 05:50:53,756] {scheduler_job.py:1114} INFO - Resetting orphaned tasks for active dag runs
   [2022-01-25 05:51:07,317] {kubernetes_executor.py:454} INFO - Found 15 queued task instances
   ```
   
   kubectl events
   ```
   
   60m         Normal    Created                  pod/<redacted_A>                               Created container base
   60m         Warning   Failed                   pod/<redacted_B>                               Error: context deadline exceeded
   60m         Normal    Pulled                   pod/<redacted_C>                               Container image "k8s.gcr.io/git-sync/git-sync:v3.3.0" already present on machine
   60m         Normal    Pulled                   pod/<redacted_D>                               Container image "k8s.gcr.io/git-sync/git-sync:v3.3.0" already present on machine
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_E>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "gefs50etlv4extracts3getatmos2000.88a63f73338a4f38b67f248ded8726dc": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_F>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to start sandbox container for pod "<redacted_F>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_G>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_G>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_H>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_H>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_I>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_I>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_J>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_J>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_K>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_K>": operation timeout: context deadline exceeded
   ```
   
   Happy to provide more info if requested.
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945785740


   I will be unsubscribing from this issue.
   
   I have not encountered this issue again (Airflow 2.1.2).
   But the following circumstances made this issue pop up again:
   
   * An Airflow DAG/task is being scheduled (not queued yet)
   * The Airflow DAG code is being updated, but it contains an error, so that the scheduler cannot load the code and the task that starts up exits immediately
   * Then the scheduler thinks the task is queued, but the task crashed immediately (using KubernetesExecutor)
   
   How do I prevent this issue?
   
   I simply make sure the DAG code is 100% clean and loads both in the scheduler and the tasks that start up (using KubernetesExecutor).
   
   How do I recover from this issue?
    
   First, I fix the issue that prevents the DAG code from loading. I restart the scheduler.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945785740


   I will unsubscribe from this issue.
   
   I have not encountered this issue again (Airflow 2.1.2).
   But the following circumstances made this issue pop up again:
   
   * An Airflow DAG/task is being scheduled (not queued yet)
   * The Airflow DAG code is being updated, but it contains an error, so that the scheduler cannot load the code and the task that starts up exits immediately
   * Then the scheduler thinks the task is queued, but the task crashed immediately (using KubernetesExecutor)
   
   How do I prevent this issue?
   
   I simply make sure the DAG code is 100% clean and loads both in the scheduler and the tasks that start up (using KubernetesExecutor).
   
   How do I recover from this issue?
    
   First, I fix the issue that prevents the DAG code from loading. I restart the scheduler.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder (which is saved in the file share -PVC, and can be viewed via azure storage explorer). 
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment.  during debug of this stuck situation, the table 'dag', 'task_*', 'celery_*' had been truncated.
   ```
   - celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule
   - connections
   ```
   
   How would like me to fetch the log? Just the dag log run? 
   update: the one i can view via storage explorer - it is empty, since the task was not executed. i will check the worker itself. and update this comment.
   
   there are no errors. i will attach the log anyhow.
   [log-workers.txt](https://github.com/apache/airflow/files/6327151/log-workers.txt)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lintingbin2009 edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lintingbin2009 edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-839409350


   We also have this problem. Every few days there will be a task that cannot be scheduled in the queue state forever. After restarting the scheduler, the task status will become up_for_retry and continue to run. So our current solution is to restart the scheduler every hour.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911454020


   The workaround for now would be to increase `[scheduler] max_dagruns_per_loop_to_schedule` to a higher value, default is 20


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] tienhung2812 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
tienhung2812 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-929988205


   > Hi, just got this issue again on `2.1.3` but the behavior quite different, all of schedulers stuck into a DAG check loop, but restart the scheduler will fix it, all other DAG as stuck in `scheduled`
   > 
   > ```
   >  [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,511] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,512] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.aa.aa_20.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,512] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,513] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.aa.ff_46.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,513] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks`
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,514] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.ff.gg_121.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > ```
   > 
   > The current system have about >= 150 active dags but we have a`abc_dag` have more than 500 tasks inside I have already increase the scheduler config to handling more task
   > 
   > * max_tis_per_query: 512
   > * max_dagruns_to_create_per_loop: 50
   > * max_dagruns_per_loop_to_schedule: 70
   >   I have also add a scheduler healthcheck but it can not detect this issue
   > 
   > Environment:
   > 
   > * Celery Executor
   > * MySQL 8.0.23-14
   > * Airflow: 2.1.3
   
   I have updated to `2.1.4` and it works properly, this might as same as https://github.com/apache/airflow/issues/18131


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hyungryuk edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hyungryuk edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-950523692


   I solved it!. 
   I think my case is slightly different with this problem.
   In my case, tasks are stuck in scheduled state, and not turned into ququed or running state.
   I fixed this with "**AIRFLOW__CORE__PARALLELISM**" env variable.
   i've tried fix my dag's CONCURRENCY and MAX_ACTIBE_RUNS config values, but it dosen't help.
   changing **AIRFLOW__CORE__PARALLELISM** env variable was solved this problem!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value schedulerNumRuns is set  -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.
   - when checking log on scheduler, it has error like this
   `[2021-04-16 13:06:36,392] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 3497') Celery Task ID: ('XXXXXXXX_YYY_test', 'Task_blahblahblah', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)`
   
   - i reported here: https://github.com/helm/charts/issues/19399 but found the issue is already closed.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder, which is saved in the file share (PVC), can be viewed via azure storage explorer. 
   The environment is set up new, and migrated a few tables from old environment. How would like me to fetch the log? Just the dag log run? 
   
   update: the one i can view via storage log - it is empty, since the task was not executed.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-813831995


   @kaxil I have checked, `min_file_process_interval` is set to `30`, however the problem is still there for me.
   
   @SalmonTimo I have a pretty high CPU utilisation (60%), albeit the scheduler settings are default. But why? Does this matter?
   
   ––
   
   Same issue, new day: I have Airflow running, the scheduler running, but the whole cluster has 103 scheduled tasks and 3 queued tasks, but nothing is running at all. I highly doubt that `min_file_process_interval` is the root of the problem.
   I suggest somebody mark that issue with a higher priority, I do not think that "regularly restarting the scheduler" is a reasonable solution.
   
   --
   
   What we need here is some factual inspection of the Python process.
   I am no Python expert, however I am proficient and know myself around in other VMs (Erlang, Ruby).
   
   Following that stack trace idea, I just learned that Python cannot dump a process (https://stackoverflow.com/a/141826/128351), unfortunately, otherwise I would have provided you with such a process dump of my running "scheduler".
   
   I am very happy to provide you with some facts about my stalled scheduler, if you tell me how you would debug such an issue.
   
   What I currently have:
   
   * CPU utilisation of the scheduler is still pretty high (around 60%).
   * `AIRFLOW__SCHEDULER__MIN_FILE_PROCESS_INTERVAL` is set to `30`
   * `AIRFLOW__SCHEDULER__DAG_DIR_LIST_INTERVAL` is set to `10`
   * Log output of scheduler:
   
   ```
   [2021-04-06 05:19:56,201] {scheduler_job.py:1063} INFO - Setting the following tasks to queued state:
   
   [2021-04-06 05:19:57,865] {scheduler_job.py:941} INFO - 15 tasks up for execution:
   
   # ... snip ...
   
   [2021-04-06 05:19:57,876] {scheduler_job.py:975} INFO - Figuring out tasks to run in Pool(name=mssql_dwh) with 0 open slots and 15 task instances ready to be queued
   [2021-04-06 05:19:57,882] {scheduler_job.py:985} INFO - Not scheduling since there are 0 open slots in pool mssql_dwh
   ```
   
   What I find striking, is the message `INFO - Not scheduling since there are 0 open slots in pool mssql_dwh`.
   That is a pool configured for max 3 slots. However no single task is running. Bluntly, I guess there is a bug with the stack:
   
   * KubernetesExecutor
   * Pools
   
   I fear bug is, that the "scheduler" might be loosing track of running tasks on Kubernetes.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] thejens commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
thejens commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-815708831


   We observe the same, I set up a dag that clears these tasks automatically by checking how long they've been queued (though this might sometimes get stuck itself, however that's resolved by restarting the scheduler frequently). 
   
   This also happens to sensors in "up_for_reschedule" by the way. I think it has something to do with a lot of dags/tasks triggering at the same time, we have a lot happening on even hours, and especially at UTC midnight. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911103938


   I'm experiencing a new, similar issue on `2.1.3`, where the DAGruns themselves get stuck in a queued state. The scheduler will accumulate queued runs but they never pop off. I'd be happy to share logs if someone can tell me which file they would like to see or what text to grep for.
   ![image](https://user-images.githubusercontent.com/7014837/131776954-bffb4ac9-e303-462c-8703-2934178f1a6b.png)
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911307179


   > I'm experiencing a new, similar issue on `2.1.3`, where the DAGruns themselves get stuck in a queued state. The scheduler will accumulate queued runs but they never pop off. I'd be happy to share logs if someone can tell me which file they would like to see or what text to grep for.
   > ![image](https://user-images.githubusercontent.com/7014837/131776954-bffb4ac9-e303-462c-8703-2934178f1a6b.png)
   
   Check the dag details page for the running tasks, it's possible there's a running task but due to the number of running as well as queued dagruns, the running is not shown here in the UI. Check the page ('Details') in dag view.
   
   If you have 20 dagruns, and only 2 is running, you won't see the running on the  page you are looking at, currently in the UI


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] renanleme edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
renanleme edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-803572439


   I had the same problem today and I think I found the problem.
   
   I'm testing with:
   Apache Airflow version: `2.0.1`
   Executor: `Celery`
   Running locally
   
   I was testing one dag and after changing a few parameters in one of the tasks in the dag file and cleaning the tasks, the task got stuck on scheduled state.
   **The problem:** The changes that I made broke the task, something like this:
   ```
   airflow-worker_1     | airflow.exceptions.AirflowException: Invalid arguments were passed to GoogleCloudStorageToBigQueryOperator (task_id: load_dag). Invalid arguments were:
   airflow-worker_1     | **kwargs: {'gcp_conn_id': 'bigquery_default'}
   ```
   So, the worker was refusing to execute because I was passing an invalid argument to the task. **_The problem is that the worker doesn't notify the scheduler/web that the file is wrong_** (no alert of a broken dag was being show in the Airflow home page).
   
   After updating the task parameter and cleaning the task, it ran successfully.
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-841102410


   @zachliu What executor are you using?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812742063


   I ran into this issue due to the scheduler over-utilizing CPU because our `min_file_process_interval` was set to 0 (the default prior to 2.0), which in airflow 2.0 causes 100% CPU utilization by constantly refreshing DAG files. Setting this parameter to 60 fixed the issue.
   The stack I observed this on:
   host: AWS ECS Cluster
   executor: CeleryExecutor
   queue: AWS SQS Queue
   
   The behavior I observed was that the scheduler would mark tasks are "queued", but never actually send them to the queue (I think the scheduler does actual queueing via the executor). My manual workaround until correcting the `min_file_process_interval` param was to stop the scheduler, clear queued tasks, and then start a new scheduler. The new scheduler would temporarily properly send tasks to the queue, before degenerating to marking tasks as queued without sending to the queue.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
uranusjr edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-811058508


   Is this related to #14924?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-813831995


   @kaxil I have checked, `min_file_process_interval` is set to `30`, however the problem is still there for me.
   
   @SalmonTimo I have a pretty high CPU utilisation (60%), albeit the scheduler settings are default. But why? Does this matter?
   
   ––
   
   Same issue, new day: I have Airflow running, the scheduler running, but the whole cluster has 103 scheduled tasks and 3 queued tasks, but nothing is running at all. I highly doubt that `min_file_process_interval` is the root of the problem.
   I suggest somebody mark that issue with a higher priority, I do not think that "regularly restarting the scheduler" is a reasonable solution.
   
   --
   
   What we need here is some factual inspection of the Python process.
   I am no Python expert, however I am proficient and know myself around in other VMs (Erlang, Ruby).
   
   Following that stack trace idea, I just learned that Python cannot dump a process (https://stackoverflow.com/a/141826/128351), unfortunately, otherwise I would have provided you with such a process dump of my running "scheduler".
   
   I am very happy to provide you with some facts about my stalled scheduler, if you tell me how you would debug such an issue.
   
   What I currently have:
   
   * CPU utilisation of the scheduler is still pretty high (around 60%).
   * `AIRFLOW__SCHEDULER__MIN_FILE_PROCESS_INTERVAL` is set to `30`
   * `AIRFLOW__SCHEDULER__DAG_DIR_LIST_INTERVAL` is set to `10`
   * Log output of scheduler:
   
   ```
   [2021-04-06 05:19:56,201] {scheduler_job.py:1063} INFO - Setting the following tasks to queued state:
   
   [2021-04-06 05:19:57,865] {scheduler_job.py:941} INFO - 15 tasks up for execution:
   
   # ... snip ...
   
   [2021-04-06 05:19:57,876] {scheduler_job.py:975} INFO - Figuring out tasks to run in Pool(name=mssql_dwh) with 0 open slots and 15 task instances ready to be queued
   [2021-04-06 05:19:57,882] {scheduler_job.py:985} INFO - Not scheduling since there are 0 open slots in pool mssql_dwh
   ```
   
   What I find striking, is the message `INFO - Not scheduling since there are 0 open slots in pool mssql_dwh`.
   That is a pool configured for max 3 slots. However no single task is running. I fear bug is, that the "scheduler" might be loosing track of running tasks on Kubernetes. Bluntly, I guess there is a bug in the components:
   
   * Scheduler
   * KubernetesExecutor
   * Pools


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] gabrielsyapse commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
gabrielsyapse commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-906778140


   what was committed in 2.1.3 that you think is the "fix". we tried upgrading to airflow 2 to find out the scheduler is broken in multiple ways, and so had to revert everything. i'm not interested in unusual config changes to dance around the issue when airflow 1.x has a scheduler that still works.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] maxcountryman commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
maxcountryman commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-924426797


   @Jorricks thanks for your note--we'll do both 1. and 2. when this happens next.
   
   We're actually using the Celery executor with Redis; I don't see any weird resource contention or other obvious issues between the ECS services.
   
   The overall status of the DagRun in these situations is "running".


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-877087945


   > https://github.com/apache/airflow/blob/b0f7f91fe29d1314b71c76de0f11d2dbe81c5c4a/airflow/jobs/scheduler_job.py#L336
   > 
   > . This particular line looks like it limits the query results per the amount of max executions of the task instances (via parallelism). Doesn't this hinder the actual results to what CAN be scheduled? There shouldn't be a limit here as the for loop underneath logically goes through the results and schedules the queued tasks right?
   
   I agree with you that this limit prevents actual TaskInstances from being scheduled in that run.
   This is also why I increased the `parallelism` option to 1000 and increased the `pool` size.
   However, I am a bit puzzled by that increasing these two doesn't solve your issue with Airflow 2.1.1.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] renanleme edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
renanleme edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-803572439


   I had the same problem today and I think I found the problem.
   
   I'm testing with:
   Apache Airflow version: `2.0.1`
   Executor: `Celery`
   Running locally
   
   I was testing one dag and after changing a few parameters in one of the tasks in the dag file and cleaning the tasks, the task got stuck on scheduled state.
   **The problem:** The changes that I made broke the task, something like this:
   ```
   airflow-worker_1     | airflow.exceptions.AirflowException: Invalid arguments were passed to GoogleCloudStorageToBigQueryOperator (task_id: load_dag). Invalid arguments were:
   airflow-worker_1     | **kwargs: {'gcp_conn_id': 'bigquery_default'}
   ```
   So, the worker was refusing to execute because I was passing an invalid argument to the task. **_The problem is that the worker doesn't notify (or update the task status to running) the scheduler/web that the file is wrong_** (no alert of a broken dag was being show in the Airflow home page).
   
   After updating the task parameter and cleaning the task, it ran successfully.
   
   _Ps.: Probably is not the same problem that the OP is having but it's also related to task stuck on scheduled_


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] RicardoPedrotti commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
RicardoPedrotti commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-809324370


   I'm facing the same issue as OP and unfortunately what @renanleme said does not apply to my situation.
   I read a lot about the `AIRFLOW__SCHEDULER__RUN_DURATION` that @jonathonbattista mentioned to be the solution, but first of all, this config does not appear to be implemented in airflow 2.X and second,  as the [Scheduler documentation](https://airflow.apache.org/docs/apache-airflow/stable/scheduler.html) says: 
   > scheduler is designed to run as a persistent service in an Airflow production environment.
   
   I wouldn't mind restarting the scheduler, but it is not clear for me the reason of the hanging queued tasks. In my environment, it appears to be very random.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-904405340


   Can we try 2.1.3 and see if this problem still exists?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] gabrielsyapse commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
gabrielsyapse commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-907296907


   @potiuk I am interested in what commit in 2.1.3 fixes this issue.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-811765010


   I have replicated this, will be working on it 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] zachliu commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
zachliu commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-840859576


   in my case (Airflow 2.0.2), tasks seem to get stuck in `queued` state if the tasks happen to run during the deployment of Airflow (we are using [ecs deploy](https://github.com/fabfuel/ecs-deploy) to deploy) :thinking: 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817758093


   @ephraimbuddy I now found a failing case.
   
   * Task remained in `queued` state
   * The pod is in `Error` state. However, the error log is very strange, as it does not happen with other tasks of the same type (with the same codebase, nothing changed).
   
   ```
   [2021-04-12 11:48:03,704] {dagbag.py:448} INFO - Filling up the DagBag from /home/airflow/dags/<redacted>.py
   Traceback (most recent call last):
     File "/usr/local/bin/airflow", line 8, in <module>
       sys.exit(main())
     File "/usr/local/lib/python3.7/site-packages/airflow/__main__.py", line 40, in main
       args.func(args)
     File "/usr/local/lib/python3.7/site-packages/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/usr/local/lib/python3.7/site-packages/airflow/utils/cli.py", line 89, in wrapper
       return f(*args, **kwargs)
     File "/usr/local/lib/python3.7/site-packages/airflow/cli/commands/task_command.py", line 217, in task_run
       dag = get_dag(args.subdir, args.dag_id)
     File "/usr/local/lib/python3.7/site-packages/airflow/utils/cli.py", line 189, in get_dag
       'parse.'.format(dag_id)
   airflow.exceptions.AirflowException: dag_id could not be found: <redacted>. Either the dag did not exist or it failed to parse.
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jpkoponen commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jpkoponen commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029720475


   I have had the same issue happen randomly and I was forwarded here by AWS support. I am considering using the utility DAG posted by @danmactough above, but I would like to make it automatic. The idea is to make it a periodic DAG that would go through all the DAGs, e.g. once per hour, and find the tasks that are stuck if there are any. I would suppose that it is possible somehow. However, I am afraid that the utility DAG itself would take too much resources and block other tasks. Do you @danmactough have any intuition if this is an applicable idea and why did you choose to make the DAG be triggered manually instead of automatic?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029786174


   @jpkoponen I adapted the @danmactough DAG to make it automatic and fit our use case. In filter DAGs that have been stuck for more than 5 minutes, and simply delete them. (In my case, changing the `try_number` and the state has no other effect than queuing the DAG again).
   
   ```
   import os
   import requests
   import time
   import json
   from datetime import datetime, timedelta
   from pprint import pprint
   
   from airflow import DAG
   from airflow.models.dagrun import DagRun
   from airflow.models.taskinstance import TaskInstance
   from airflow.operators.python import PythonOperator
   from airflow.utils import timezone
   from airflow.utils.db import provide_session
   from airflow.utils.state import State
   from dependencies.utils.var import DATADOG_API_KEY
   
   DAG_NAME = os.path.splitext(os.path.basename(__file__))[0]
   DEFAULT_ARGS = {
       "owner": "airflow",
       "depends_on_past": False,
       "email_on_failure": False,
       "email_on_retry": False,
       "execution_timeout": timedelta(minutes=10),
       "retries": 0,
   }
   
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       filter = [
           TaskInstance.state == State.QUEUED,
           TaskInstance.queued_dttm < datetime.now(timezone.utc) - timedelta(minutes=5)
       ]
   
       tis = session.query(TaskInstance).filter(*filter).all()
       print(f"Task instances: {tis}")
       print(f"Updating {len(tis)} task instances")
   
   
       for ti in tis:
           dr = (
               session.query(DagRun)
               .filter(DagRun.run_id == ti.dag_run.run_id)
               .first()
           )
   
           dagrun = (
               dict(
                   id=dr.id,
                   dag_id=dr.dag_id,
                   execution_date=dr.execution_date,
                   start_date=dr.start_date,
                   end_date=dr.end_date,
                   _state=dr._state,
                   run_id=dr.run_id,
                   creating_job_id=dr.creating_job_id,
                   external_trigger=dr.external_trigger,
                   run_type=dr.run_type,
                   conf=dr.conf,
                   last_scheduling_decision=dr.last_scheduling_decision,
                   dag_hash=dr.dag_hash,
               )
               if dr
               else {}
           )
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
   
   
           dr.state = State.FAILED
           print(f"Deleting {str(ti)}.")
           session.delete(ti)
   
       session.commit()
       print("Done.")
   
   
   with DAG(
       DAG_NAME,
       description="Utility DAG to fix TaskInstances stuck in queued state",
       default_args=DEFAULT_ARGS,
       schedule_interval="*/5 * * * *",
       start_date=datetime(year=2021, month=8, day=1),
       max_active_runs=1,
       catchup=False,
       default_view="graph",
       is_paused_upon_creation=False,
   ) as dag:
       PythonOperator(task_id="unstick_dag", python_callable=unstick_dag_callable)
   ```
      The DAG runs every 5 minutes and I never caught it in a queued state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hyungryuk edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hyungryuk edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-950523692


   I've solved it!. 
   I think my case is slightly different with this problem.
   In my case, tasks are stuck in scheduled state, and not turned into ququed or running state.
   I fixed this with "**AIRFLOW__CORE__PARALLELISM**" env variable.
   i've tried fix my dag's CONCURRENCY and MAX_ACTIBE_RUNS config values, but it dosen't help.
   changing **AIRFLOW__CORE__PARALLELISM** env variable was solved this problem!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1013862830


   Hello @danmactough,
   
   Thanks for your work ! Could you provide the whole code for your utility DAG please ? 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861521020


   @shivanshs9 did you try what I mentioned? Stopping the scheduler, then clearing the Task Instances and starting the scheduler again?
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-819411427


   @ephraimbuddy I found the root cause for the problem, and a way to reproduce it. Keep in mind my stack (Airflow + KubernetesExecutor), as this issue has been watered down by many different stacks and situations, ending with the same symptoms.
   
   Steps to reproduce:
   
   * Create a DAG and schedule some work for it.
   * While work is scheduled, remove the DAG.
   * Restart the scheduler.
   * Now the DAG does no longer exist, but it still exists in the database. And its scheduled tasks also still exist.
   * The scheduler dutifully schedules work for the non-existent DAG (<- this is a problem)
   * The KubernetesExecutor spawns a new worker pod
   * The worker pod is awfully surprised that there is no DAG for the work he was tasked with
   * The worker pod commits suicide without telling anybody (<- this is a problem)
   * The scheduler faithfully keeps the task in "queued" state, although the worker is no more
   
   Solution:
   
   * The scheduler should not schedule work for tasks that are no longer in the DagBag
   * The worker must fail properly (with its task ending in a "failed" state) when he cannot find the DAG + task he was tasked with


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] renanleme commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
renanleme commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-803572439


   I had the same problem today and I think I found the problem.
   
   I'm testing with:
   Apache Airflow version: `2.0.1`
   Executor: `Celery`
   Running locally
   
   I was testing one dag and after changing a few parameters in one of the tasks in the dag file and cleaning the tasks, the task got stuck on scheduled state.
   **The problem:** The changes that I made broke the task, something like this:
   ```
   airflow-worker_1     | airflow.exceptions.AirflowException: Invalid arguments were passed to GoogleCloudStorageToBigQueryOperator (task_id: load_dag). Invalid arguments were:
   airflow-worker_1     | **kwargs: {'gcp_conn_id': 'bigquery_default'}
   ```
   So, the worker was refusing to execute because I was passing an invalid argument to the task. **_The problem is that the worker doesn't notify the scheduler/web that the file is wrong_** (no alert of a broken dag was being show in the Airflow home page).
   
   After updating the task parameter and cleaning again the task, it ran successfully.
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] rrbarbosa edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
rrbarbosa edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-844789569


   We are also experiencing tasks getting stuck in Airflow 2.0.1 with the Kubernetes Executor. We created a "heartbeat" DAG that runs every 10m, when it doesn't, we get an alert.
   
   In the last occurrence, the scheduler seem to think the heartbeat DAG reached its `max_active_runs (1 of 1)`, so it refused to schedule a new task. As far as I can tell, the previous execution succeeded and the pod correctly was killed. 
   
   That task remained in the `scheduled` state, and the log `Reset the following X orphaned TaskInstances` listed the only task in this DAG every 5min. After ~1h hour, the DAG was finally executed again.
   
   Our current workaround is a cronjob to restart the scheduler pod every few hours.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-811058508


   Is this and #14924 related?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-810866892


   Back on this. I am currently observing the behaviour again.
   
   I can confirm:
   
   * The solution description of @renanleme does not apply to my case. Definitely not.
   * Restarting the scheduler is a workaround the problem
   
   The issue persists with `2.0.1` please update the tag accordingly.
   
   The issue is definitely "critical" as it halts THE ENTIRE airflow operation...!


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] tothandor commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
tothandor commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-818568914


   I also have these symptoms with Airflow 2.0.1, though I am using a Celery executor.
   I have inspected the workers with `strace` and it seems, that they're unable to read from some pipe:
   ```bash
   # strace -p 259969
   strace: Process 259969 attached
   read(8,
   ```
   
   If I check the file descriptors, it's gone missing:
   ```
   # ls -l /proc/259969/fd/8                                                                                                                                                                                         
   lr-x------. 1 airflow airflow 64 Apr 13 10:31 /proc/259969/fd/8 -> 'pipe:[864564522]'
   ```
   
   And the parent process is unable to write to a socket:
   ```bash
   # strace -p 252882                                                                                                                                                                                              
   strace: Process 252882 attached
   write(6, "\0\0\1t\200\4\225i\1\0\0\0\0\0\0\214\34airflow.utils.d"..., 376
   ```
   And the socket as well is missing.
   ```bash
   # ls -l /proc/252882/fd/6                                                                                                                                                                                         
   lrwx------. 1 airflow airflow 64 Apr 13 10:30 /proc/252882/fd/6 -> 'socket:[864501296]'
   ```
   After killing the stucked workers things start to work again for a while.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nitinpandey-154 edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
nitinpandey-154 edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-846808019


   Environment:
   - Airflow 2.1.0
   - Docker : apache/airflow:2.1.0-python3.8
   - Executor : Celery
   
   I am facing the same issue again.  This happens randomly and even after cleaning up the tasks instances and/or restarting the container is not fixing the issue.
   
   airflow-scheduler    | [2021-05-24 06:46:52,008] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 21, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 100 and queue airflow_maintenance
   airflow-scheduler    | [2021-05-24 06:46:52,008] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 21, tzinfo=Timezone('UTC')), try_number=1)
   airflow-scheduler    | [2021-05-24 06:46:52,008] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 24, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 100 and queue airflow_maintenance
   airflow-scheduler    | [2021-05-24 06:46:52,009] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 24, tzinfo=Timezone('UTC')), try_number=1)
   
   The only solution that works is manually deleting the dag.. which isn't a feasible option. This should be a very high priority as it breaks the scheduling.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - airflow 1.10.14
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value schedulerNumRuns is set  -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.
   - when checking log on scheduler, it has error like this
   `[2021-04-16 13:06:36,392] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 3497') Celery Task ID: ('XXXXXXXX_YYY_test', 'Task_blahblahblah', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)`
   
   - i reported here: https://github.com/helm/charts/issues/19399 but found the issue is already closed.
   
   **I tried to experiment the ...., which did not help as i expected.**
   - Uploaded the dag with new name/id. enabled, cleared the dag (otherwise the 1st task just stuck at the 'queued' state)
   and 1st task is at the 'scheduled' state and stuck there.
   - check scheduler log:
   - `[2021-04-16 15:58:51,991] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 1851')
   Celery Task ID: ('XXXXXX_min_test_3', 'Load_product_to_source', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/executors/celery_executor.py", line 117, in fetch_celery_task_state
       res = (celery_task[0], celery_task[1].state)`
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-869620583


   @Vbubblery _A_ fix was included in 2.1.1, but it is not yet clear if that fixes all the cases of this behaviour or not.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay removed a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay removed a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614


   I am also noticing this issue with my Airflow v2.1.2 instance, where the task is forever in queued state, and in the scheduler log I see 
   ```
   airflow-scheduler1_1  | [2021-08-04 12:45:23,286] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   ```
   
   When I sent the SIGUSR2 signal to scheduler I received:
   
   ```
   airflow-scheduler1_1  | SIGUSR2 received, printing debug
   airflow-scheduler1_1  | --------------------------------------------------------------------------------
   airflow-scheduler1_1  | [2021-08-04 13:07:37,230] {base_executor.py:305} INFO - executor.queued (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,231] {base_executor.py:307} INFO - executor.running (1)
   airflow-scheduler1_1  | 	TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,233] {base_executor.py:311} INFO - executor.event_buffer (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,234] {celery_executor.py:372} INFO - executor.tasks (1)
   airflow-scheduler1_1  | 	(TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 1ed20983-dbd3-47c6-8404-345fa0fb75ff>)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,235] {celery_executor.py:377} INFO - executor.adopted_task_timeouts (0)
   ```
   
   So basically it is trying to queue a running task ? and failing the `if` check 
   ```
           if task_instance.key not in self.queued_tasks and task_instance.key not in self.running:
               self.log.info("Adding to queue: %s", command)
               self.queued_tasks[task_instance.key] = (command, priority, queue, task_instance)
           else:
               self.log.error("could not queue task %s", task_instance.key)
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614






-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-819436342


   Thank you @lukas-at-harren, you captured it correctly


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614


   I am also noticing this issue with my Airflow v2.1.2 instance, where the task is forever in queued state, and in the scheduler log I see 
   ```
   airflow-scheduler1_1  | [2021-08-04 12:45:23,286] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   ```
   
   When I sent the SIGUSR2 signal to scheduler I received:
   
   ```
   airflow-scheduler1_1  | SIGUSR2 received, printing debug
   airflow-scheduler1_1  | --------------------------------------------------------------------------------
   airflow-scheduler1_1  | [2021-08-04 13:07:37,230] {base_executor.py:305} INFO - executor.queued (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,231] {base_executor.py:307} INFO - executor.running (1)
   airflow-scheduler1_1  | 	TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,233] {base_executor.py:311} INFO - executor.event_buffer (0)
   airflow-scheduler1_1  | 	
   airflow-scheduler1_1  | [2021-08-04 13:07:37,234] {celery_executor.py:372} INFO - executor.tasks (1)
   airflow-scheduler1_1  | 	(TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 1ed20983-dbd3-47c6-8404-345fa0fb75ff>)
   airflow-scheduler1_1  | [2021-08-04 13:07:37,235] {celery_executor.py:377} INFO - executor.adopted_task_timeouts (0)
   ```
   
   So basically it is trying to queue a running task ? and failing the `if` check 
   ```
           if task_instance.key not in self.queued_tasks and task_instance.key not in self.running:
               self.log.info("Adding to queue: %s", command)
               self.queued_tasks[task_instance.key] = (command, priority, queue, task_instance)
           else:
               self.log.error("could not queue task %s", task_instance.key)
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817761518


   > @ephraimbuddy I now found a failing case.
   > 
   > * Task remained in `queued` state
   > * The pod is in `Error` state. However, the error log is very strange, as it does not happen with other tasks of the same type (with the same codebase, nothing changed).
   > 
   > ```
   > [2021-04-12 11:48:03,704] {dagbag.py:448} INFO - Filling up the DagBag from /home/airflow/dags/<redacted>.py
   > Traceback (most recent call last):
   >   File "/usr/local/bin/airflow", line 8, in <module>
   >     sys.exit(main())
   >   File "/usr/local/lib/python3.7/site-packages/airflow/__main__.py", line 40, in main
   >     args.func(args)
   >   File "/usr/local/lib/python3.7/site-packages/airflow/cli/cli_parser.py", line 48, in command
   >     return func(*args, **kwargs)
   >   File "/usr/local/lib/python3.7/site-packages/airflow/utils/cli.py", line 89, in wrapper
   >     return f(*args, **kwargs)
   >   File "/usr/local/lib/python3.7/site-packages/airflow/cli/commands/task_command.py", line 217, in task_run
   >     dag = get_dag(args.subdir, args.dag_id)
   >   File "/usr/local/lib/python3.7/site-packages/airflow/utils/cli.py", line 189, in get_dag
   >     'parse.'.format(dag_id)
   > airflow.exceptions.AirflowException: dag_id could not be found: <redacted>. Either the dag did not exist or it failed to parse.
   > ```
   
   Good info, similar to what I got this morning https://github.com/apache/airflow/issues/14974#issuecomment-817577477. Thanks for checking


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-819411427


   @ephraimbuddy I found the root cause for _my_ problem, and a way to reproduce it. Keep in mind my stack (Airflow + KubernetesExecutor), as this issue has been watered down by many different stacks and situations, ending with the same symptoms.
   
   Steps to reproduce:
   
   * Create a DAG and schedule some work for it.
   * While work is scheduled, remove the DAG.
   * Restart the scheduler.
   * Now the DAG does no longer exist, but it still exists in the database. And its scheduled tasks also still exist.
   * The scheduler dutifully schedules work for the non-existent DAG (<- this is a problem)
   * The KubernetesExecutor spawns a new worker pod
   * The worker pod is awfully surprised that there is no DAG for the work he was tasked with
   * The worker pod commits suicide without telling anybody (<- this is a problem)
   * The scheduler faithfully keeps the task in "queued" state, although the worker is no more
   
   Solution:
   
   * The scheduler should not schedule work for tasks that are no longer in the DagBag
   * The worker must fail properly (with its task ending in a "failed" state) when he cannot find the DAG + task he was tasked with
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817201181


   https://github.com/apache/airflow/pull/15263 will add a POD pending timeout which should help with the case similar to what @ephraimbuddy had


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder, which is saved in the file share (PVC), can be viewed via azure storage explorer. 
   The environment is set up new, and migrated a few tables from old environment. How would like me to fetch the log? Just the dag log run? 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-919078792


   Can someone test it with latest 2.2.0b1 @Overbryd ?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jbkc85 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jbkc85 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-876546094


   > > We just updated it. We will let you know in the morning - hopefully we see some improvement!
   > 
   > You can try the things mentioned here [in this comment](https://github.com/apache/airflow/issues/13542#issuecomment-861300533) if the issue still exists.
   
   Still the same issue - regardless of troubleshooting.  Also we run on a LocalExecutor, so we cant force a DAG Run.
   
   I did notice something though and I just want to verify: https://github.com/apache/airflow/blob/b0f7f91fe29d1314b71c76de0f11d2dbe81c5c4a/airflow/jobs/scheduler_job.py#L336.  This particular line looks like it limits the query results per the amount of max executions of the task instances (via parallelism).  Doesn't this hinder the actual results to what CAN be scheduled?  There shouldn't be a limit here as the for loop underneath logically goes through the results and schedules the queued tasks right?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861300533


   I currently work for a company where we have 100+ DAGS running with approximately a 1000 tasks.
   We were running into a similar issue.
   
   I figured out there is a bug in the `celery_executor` which I still want to fix myself and contribute.
   Summary of that problem:
   At the start of the scheduler, the celery_executor class instance of the scheduler picks up everything from 'dead' schedulers (your previous run). That is (if you run one scheduler) every TaskInstance in the Running, Queued or Scheduled state. Then once it verified that this task is not running(takes 10 minutes), it clears most of the references but forgets a crucial one, making it such that the scheduler can NEVER start this task anymore. You can still start it via the webserver because that has its own celery_executor class instance.
   
   What we noticed:
   - Many tasks were very slowly to be scheduled even though the workers were almost fully idle.
   - The TaskInstances were stuck on Queued or Scheduled.
   - Restarting the scheduler didn't work.
   
   What you can do to verify whether you have the same issue:
   - Stop the scheduler
   - Clear all TaskInstances that are Queued or Scheduled
   - Start the scheduler
   
   Our fix:
   - Increase the airflow.cfg parallelism -> from 32 to 320. This is what deadlocks your scheduler in that case.
   - Increase the default pool size (for a speedup) -> from 128 to 1000
   - For any task that the scheduler can't run anymore. Do the procedure mentioned above or kick-start it yourself by clicking the task instance followed by "Ignore all deps", "Ignore Task states", "Ignore Task Deps" and finally "Run".
   
   Hope this helps anyone and saves you a couple days of debugging :)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-877087945


   > https://github.com/apache/airflow/blob/b0f7f91fe29d1314b71c76de0f11d2dbe81c5c4a/airflow/jobs/scheduler_job.py#L336
   > 
   > . This particular line looks like it limits the query results per the amount of max executions of the task instances (via parallelism). Doesn't this hinder the actual results to what CAN be scheduled? There shouldn't be a limit here as the for loop underneath logically goes through the results and schedules the queued tasks right?
   
   I agree with you that this limit prevents actual TaskInstances from being scheduled in that run. And I also think this is not an optimal implementation.
   This is also why I increased the `parallelism` option to 1000 and increased the `pool` size.
   However, I am a bit puzzled by that increasing these two doesn't solve your issue with Airflow 2.1.1.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mfjackson commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mfjackson commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-756526646


   @Overbryd I'm not sure if this helps, but I was dealing with a similar issue with our self-managed Airflow instance on GKE when we upgraded to 2.0.0 a couple of weeks ago. Are you using `gitSync`? If so, are you receiving an error in your logs surrounding the `knownHosts`?
   
   If so, we found that updating the [pod-template-file](https://github.com/apache/airflow/blob/master/chart/files/pod-template-file.kubernetes-helm-yaml) to this:
   
   ```
   # Licensed to the Apache Software Foundation (ASF) under one
   # or more contributor license agreements.  See the NOTICE file
   # distributed with this work for additional information
   # regarding copyright ownership.  The ASF licenses this file
   # to you under the Apache License, Version 2.0 (the
   # "License"); you may not use this file except in compliance
   # with the License.  You may obtain a copy of the License at
   #
   #   http://www.apache.org/licenses/LICENSE-2.0
   #
   # Unless required by applicable law or agreed to in writing,
   # software distributed under the License is distributed on an
   # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
   # KIND, either express or implied.  See the License for the
   # specific language governing permissions and limitations
   # under the License.
   ---
   apiVersion: v1
   kind: Pod
   metadata:
     name: dummy-name
   spec:
   {{- if .Values.dags.gitSync.enabled }}
     initContainers:
   {{- include "git_sync_container" (dict "Values" .Values "is_init" "true") | indent 8 }}
   {{- end }}
     containers:
       - args: []
         command: []
         envFrom:
         {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 6 }}
         env:
           - name: AIRFLOW__CORE__EXECUTOR
             value: LocalExecutor
   {{- include "standard_airflow_environment" . | indent 6}}
   {{- include "custom_airflow_environment" . | indent 6 }}
         image: {{ template "pod_template_image" . }}
         imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
         name: base
         ports: []
         volumeMounts:
           - mountPath: {{ template "airflow_logs" . }}
             name: airflow-logs
           - name: config
             mountPath: {{ template "airflow_config_path" . }}
             subPath: airflow.cfg
             readOnly: true
   {{- if .Values.scheduler.airflowLocalSettings }}
           - name: config
             mountPath: {{ template "airflow_local_setting_path" . }}
             subPath: airflow_local_settings.py
             readOnly: true
   {{- end }}
   {{- if .Values.dags.gitSync.knownHosts }}
           - mountPath: /etc/git-secret/known_hosts
             name: config
             subPath: known_hosts
             readOnly: true
   {{- end }}
   {{- if .Values.dags.gitSync.sshKeySecret }}
           - mountPath: /etc/git-secret/ssh
             name: git-sync-ssh-key
             subPath: ssh
   {{- end }}
   {{- if or .Values.dags.gitSync.enabled .Values.dags.persistence.enabled }}
           - mountPath: {{ include "airflow_dags_mount_path" . }}
             name: dags
             readOnly: true
   {{- if .Values.dags.persistence.enabled }}
             subPath: {{.Values.dags.gitSync.dest }}/{{ .Values.dags.gitSync.subPath }}
   {{- end }}
   {{- end }}
     hostNetwork: false
     {{- if or .Values.registry.secretName .Values.registry.connection }}
     imagePullSecrets:
       - name: {{ template "registry_secret" . }}
     {{- end }}
     restartPolicy: Never
     securityContext:
       runAsUser: {{ .Values.uid }}
     nodeSelector: {{ toYaml .Values.nodeSelector | nindent 4 }}
     affinity: {{ toYaml .Values.affinity | nindent 4 }}
     tolerations: {{ toYaml .Values.tolerations | nindent 4 }}
     serviceAccountName: '{{ .Release.Name }}-worker'
     volumes:
     {{- if .Values.dags.persistence.enabled }}
     - name: dags
       persistentVolumeClaim:
         claimName: {{ template "airflow_dags_volume_claim" . }}
     {{- else if .Values.dags.gitSync.enabled }}
     - name: dags
       emptyDir: {}
     {{- end }}
     {{- if and  .Values.dags.gitSync.enabled  .Values.dags.gitSync.sshKeySecret }}
   {{- include "git_sync_ssh_key_volume" . | indent 2 }}
     {{- end }}
     - emptyDir: {}
       name: airflow-logs
     - configMap:
         name: {{ include "airflow_config" . }}
       name: config
   ```
   
   The changes to the original file are as follows:
   
   - Line 56: set "name: config"
   - Line 58: insert "readOnly: true"
   - Delete lines 98-103
   
   This allowed gitSync to work (after also passing a `knownHosts` value in the `values.yaml` file, along with the other necessary configurations) and processes to move from a persistent "queued" state to a "running" state.
   
   If this isn't the issue, the other area that you may want to look into is making sure that your service account binding annotations are properly set for you scheduler, webserver, and workers in your `values.yaml` file. If they're not properly bound to a GCP service account with adequate pod creation permissions, the initial task may queue, but won't run.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029786174


   @jpkoponen I adapted the @danmactough DAG to make it automatic and fit our use case. It filters DAGs that have been stuck for more than 5 minutes, and simply deletes them. (In my case, changing the `try_number` and the state has no other effect than queuing the task again).
   
   ```python
   import os
   import requests
   import time
   import json
   from datetime import datetime, timedelta
   from pprint import pprint
   
   from airflow import DAG
   from airflow.models.dagrun import DagRun
   from airflow.models.taskinstance import TaskInstance
   from airflow.operators.python import PythonOperator
   from airflow.utils import timezone
   from airflow.utils.db import provide_session
   from airflow.utils.state import State
   
   DAG_NAME = os.path.splitext(os.path.basename(__file__))[0]
   DEFAULT_ARGS = {
       "owner": "airflow",
       "depends_on_past": False,
       "email_on_failure": False,
       "email_on_retry": False,
       "execution_timeout": timedelta(minutes=10),
       "retries": 0,
   }
   
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       filter = [
           TaskInstance.state == State.QUEUED,
           TaskInstance.queued_dttm < datetime.now(timezone.utc) - timedelta(minutes=5)
       ]
   
       tis = session.query(TaskInstance).filter(*filter).all()
       print(f"Task instances: {tis}")
       print(f"Updating {len(tis)} task instances")
   
   
       for ti in tis:
           dr = (
               session.query(DagRun)
               .filter(DagRun.run_id == ti.dag_run.run_id)
               .first()
           )
   
           dagrun = (
               dict(
                   id=dr.id,
                   dag_id=dr.dag_id,
                   execution_date=dr.execution_date,
                   start_date=dr.start_date,
                   end_date=dr.end_date,
                   _state=dr._state,
                   run_id=dr.run_id,
                   creating_job_id=dr.creating_job_id,
                   external_trigger=dr.external_trigger,
                   run_type=dr.run_type,
                   conf=dr.conf,
                   last_scheduling_decision=dr.last_scheduling_decision,
                   dag_hash=dr.dag_hash,
               )
               if dr
               else {}
           )
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
   
   
           dr.state = State.FAILED
           print(f"Deleting {str(ti)}.")
           session.delete(ti)
   
       session.commit()
       print("Done.")
   
   
   with DAG(
       DAG_NAME,
       description="Utility DAG to fix TaskInstances stuck in queued state",
       default_args=DEFAULT_ARGS,
       schedule_interval="*/5 * * * *",
       start_date=datetime(year=2021, month=8, day=1),
       max_active_runs=1,
       catchup=False,
       default_view="graph",
       is_paused_upon_creation=False,
   ) as dag:
       PythonOperator(task_id="unstick_dag", python_callable=unstick_dag_callable)
   ```
      The DAG runs every 5 minutes and I never caught it in a queued state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1053580062


   Also I suggest to open issue to the MWAA support - maybe this is simply some problem with MWAA configuration.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817714499


   > @ephraimbuddy no I do not have that issue.
   > 
   > When I try to observe it closely, it always goes like this:
   > 
   > * There are 3 tasks in the `mssql_dwh` pool. All of them have the state `queued`. Nothing is running. Nothing is started. The scheduler does not start anything new, because the pool has 0 available slots.
   > * Then I clear those 3 tasks.
   > * The scheduler immediately picks some tasks and puts them into `queued` state. Meanwhile, Kubernetes starts the pods.
   > * If I am lucky, some of the tasks get executed properly, and the scheduler continues what it is supposed to do.
   > * But not long, it starts to accumulate "dead" tasks in `queued` state. Those are NOT running in Kubernetes.
   > * I checked the scheduler for error logs, and I can see some log lines like these
   > 
   > ```
   > ERROR - Executor reports task instance <TaskInstance: <redacted> 2021-04-10 00:00:00+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
   > ```
   > 
   > @kaxil So think there must be some kind of race condition between the scheduler and the Kubernetes pod startup.
   > Some tasks finish really quickly (successfully so) and the scheduler KEEPS them in `queued` state.
   
   When the tasks are queued and nothing is happening, can you describe the task pods and check what's happening inside. I feel something went wrong inside the pods containers. Please post what you got when you describe the pod here if you can replicate again 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-924363913






-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911420829


   Thanks @easontm , It's a bug. I have a PR trying to address that see https://github.com/apache/airflow/pull/17945


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853117451


   More data, if it can benefit:
   I'm using Airflow `2.1.0` (upgraded from `1.10.15` in our development environment) and trying to use the `KubernetesExecutor`. I am running _extremely_ basic DAGs that exist solely to test. In `airflow.cfg` I set `delete_worker_pods = False` so that I could try and examine what's going on. As far as I can tell, my worker pods are being created, immediately considering themselves successful (not sure if they actually receive a task to complete), and terminating. I don't think the pods are "successful but not writing the success state to the DB" because one of my tasks is a simple `CREATE TABLE` statement on my data warehouse, and the table does not appear.
   
   
   Here are the results of various logs:
   
   If I run `kubectl get pods`, I see my `airflow-webserver` and `airflow-scheduler` pods, as well as some "Completed" worker pods.
   
   ```
   NAME                                                           READY   STATUS      RESTARTS   AGE
   airflow-scheduler                                              1/1     Running     0          45m
   airflow-webserver-1                                            2/2     Running     0          56m
   airflow-webserver-2                                            2/2     Running     0          56m
   airflow-webserver-3                                            2/2     Running     0          45m
   airflow-webserver-4                                            2/2     Running     0          45m
   airflow-webserver-5                                            2/2     Running     0          45m
   <GENERATED_WORKER_POD_1>                                       0/1     Completed   0          15m
   <GENERATED_WORKER_POD_2>                                       0/1     Completed   0          56m
   <GENERATED_WORKER_POD_3>                                       0/1     Completed   0          45m
   ```
   
   `kubectl logs <my_worker_pod>` - I can tell that the pod was briefly alive but that did nothing, because the only output is a line that always appears as a side effect of the Docker image config. In contrast, if I get the logs from one of my functional `1.10.15` pods, I can see the task start:
   ```
   [2021-06-02 13:33:01,358] {__init__.py:50} INFO - Using executor LocalExecutor
   [2021-06-02 13:33:01,358] {dagbag.py:417} INFO - Filling up the DagBag from /usr/local/airflow/dags/my_dag.py
   ...
   etc
   ```
   
   `kubectl describe pod <my_worker_pod>` - the event log is quite tame:
   ```
   Events:
     Type    Reason     Age    From               Message
     ----    ------     ----   ----               -------
     Normal  Scheduled  3m21s  default-scheduler  Successfully assigned <GENERATED_POD_NAME> to <EC2_INSTANCE>
     Normal  Pulling    3m19s  kubelet            Pulling image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Pulled     3m19s  kubelet            Successfully pulled image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Created    3m19s  kubelet            Created container base
     Normal  Started    3m19s  kubelet            Started container base
   ```
   That's it. No further events.
   
   `kubectl logs airflow-scheduler` - I've trimmed the logs significantly, but here are the statements mentioning a stuck task
   ```
   [2021-06-02 14:49:42,742] {kubernetes_executor.py:369} INFO - Attempting to finish pod; pod_id: <GENERATED_POD_NAME>; state: None; annotations: {'dag_id': '<TEST_DAG>', 'task_id': '<TASK_ID>', 'execution_date': '2021-05-30T00:00:00+00:00', 'try_number': '1'}
   [2021-06-02 14:49:42,748] {kubernetes_executor.py:546} INFO - Changing state of (TaskInstanceKey(dag_id='<TEST_DAG>', task_id='<TASK_ID>', execution_date=datetime.datetime(2021, 5, 30, 0, 0, tzinfo=tzlocal()), try_number=1), None, '<GENERATED_POD_NAME>', 'airflow', '1000200742') to None
   [2021-06-02 14:49:42,751] {scheduler_job.py:1212} INFO - Executor reports execution of <TASK_ID> execution_date=2021-05-30 00:00:00+00:00 exited with status None for try_number 1
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - airflow 1.10.14
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value schedulerNumRuns is set  -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.
   - when checking log on scheduler, it has error like this
   `[2021-04-16 13:06:36,392] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 3497') Celery Task ID: ('XXXXXXXX_YYY_test', 'Task_blahblahblah', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)`
   
   - i reported here: https://github.com/helm/charts/issues/19399 but found the issue is already closed.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812742063






-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817698207


   @ephraimbuddy no I do not have that issue.
   
   When I try to observe it closely, it always goes like this:
   
   * There are 3 tasks in the `mssql_dwh` pool. All of them have the state `queued`. Nothing is running. Nothing is started. The scheduler does not start anything new, because the pool has 0 available slots.
   * Then I clear those 3 tasks.
   * The scheduler immediately picks some tasks and puts them into `queued` state. Meanwhile, Kubernetes starts the pods.
   * If I am lucky, some of the tasks get executed properly, and the scheduler continues what it is supposed to do.
   * But not long, it starts to accumulate "dead" tasks in `queued` state. Those are NOT running in Kubernetes.
   * I checked the scheduler for error logs, and I can see some log lines like these
   
   ```
   ERROR - Executor reports task instance <TaskInstance: <redacted> 2021-04-10 00:00:00+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
   ```
   
   @kaxil So think there must be some kind of race condition between the scheduler and the Kubernetes pod startup.
   Some tasks finish really quickly (successfully so) and the scheduler KEEPS them in `queued` state.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jonathonbattista edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jonathonbattista edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-770431504


   We are experiencing these symptoms on `1.10.14` as well. Doesn't seem `git-sync` related.
   
   Does any work have a clue why this is happening?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder (which is saved in the file share -PVC, and can be viewed via azure storage explorer). 
   
   updated my comment earlier


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jhonatanrt commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jhonatanrt commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-852487656


   > I'm on Airflow `2.1.0` deployed to ECS Fargate--seeing similar issues where they are 128 slots, a small number used, but queued tasks that aren't ever run. Perhaps relatedly I also see a task in a `none` state which I'm not able to track down from the UI. Cluster resources seem fine, using less than 50% CPU and memory.
   > 
   > I also saw an error from Sentry around this time:
   > 
   > ```
   > could not queue task ["amplitude.events","run-amplitude-events-crawler","2021-05-27T04:15:00.000000Z",1]
   > ```
   > 
   > <img alt="Screen Shot 2021-05-28 at 9 15 32 AM" width="1297" src="https://user-images.githubusercontent.com/74351/120013160-43980280-bf95-11eb-95ce-5c6196993f7b.png">
   > 
   > <img alt="Screen Shot 2021-05-28 at 9 14 58 AM" width="746" src="https://user-images.githubusercontent.com/74351/120013097-2fec9c00-bf95-11eb-9e1a-bd062b8fba0c.png">
   Have you  had any problem to deploy airflow-scheduler in ecs fargate?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861354234


   @Jorricks Amazing! Shout (here, with a direct ping, or on Airflow slack) if I can help at all.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hyungryuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hyungryuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-950523692


   I solved it!. 
   I think my case is slightly different with this problem.
   In my case, tasks are stuck in scheduled state, and not turned into ququed or running state.
   I fixed this with "AIRFLOW__CORE__PARALLELISM" env variable.
   i've tried fix my dag's CONCURRENCY and MAX_ACTIBE_RUNS config values, but it dosen't help.
   change AIRFLOW__CORE__PARALLELISM env variable is helpful!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] timgriffiths edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
timgriffiths edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-854356862


   Environment:
   
   Airflow 2.1.0
   Docker : apache/airflow:2.1.0-python3.8
   Executor : Celery
   
   So I have also hit this today, the way i was able to replicate this 
   
   -> have a py file that generates move than 1 dag
   -> then have a user break 1 of the dag from this resultant chain
   -> all the other still valid dags will be triggered in the scheduler but fail in the Worker but it looks like the exception is not passed back to the Scheduler so it permanently stays in a "queued state"
   
   > [2021-06-04 01:11:22,927: INFO/ForkPoolWorker-40] Executing command in Celery: ['airflow', 'tasks', 'run', 'myworkingjob', 'taskname-abc', '2021-06-04T01:11:21.775097+00:00', '--local', '--pool', 'pool', '--subdir', '/opt/airflow/dags/mm-dynamic-dags.py']
   [2021-06-04 01:11:22,940: INFO/ForkPoolWorker-34] Filling up the DagBag from /opt/airflow/dags/mm-dynamic-dags.py
   [2021-06-04 01:11:27,993: ERROR/ForkPoolWorker-39] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dagbag.py", line 317, in _load_modules_from_file
       loader.exec_module(new_module)
     File "<frozen importlib._bootstrap_external>", line 848, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 187, in <module>
       create_dag("{}/{}".format(global_dags_base_url, dag_file))
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 123, in create_dag
       dag = DAG(
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dag.py", line 277, in __init__
       validate_key(dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/helpers.py", line 44, in validate_key
       raise AirflowException(
   airflow.exceptions.AirflowException: The key (test-test+abc) has to be made of alphanumeric characters, dashes, dots and underscores exclusively
   [2021-06-04 01:11:27,996: ERROR/ForkPoolWorker-39] Failed to execute task dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse..
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 116, in _execute_in_fork
       args.func(args)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 91, in wrapper
       return f(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/commands/task_command.py", line 219, in task_run
       dag = get_dag(args.subdir, args.dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 191, in get_dag
       raise AirflowException(
   airflow.exceptions.AirflowException: dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse.
   [2021-06-04 01:11:28,017: ERROR/ForkPoolWorker-39] Task airflow.executors.celery_executor.execute_command[66453177-4042-4c6a-882b-bd0fcdbda0d8] raised unexpected: AirflowException('Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm')
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 412, in trace_task
       R = retval = fun(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 704, in __protected_call__
       return self.run(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 87, in execute_command
       _execute_in_fork(command_to_exec)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 98, in _execute_in_fork
       raise AirflowException('Celery command failed on host: ' + get_hostname())
   airflow.exceptions.AirflowException: Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm
   [2021-06-04 01:11:28,091: ERROR/ForkPoolWorker-37] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] maxcountryman edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
maxcountryman edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-850527299


   I'm on Airflow `2.1.0` deployed to ECS Fargate--seeing similar issues where they are 128 slots, a small number used, but queued tasks that aren't ever run. Perhaps relatedly I also see a task in a `none` state which I'm not able to track down from the UI. Cluster resources seem fine, using less than 50% CPU and memory.
   
   I also saw an error from Sentry around this time:
   
   ```
   could not queue task ["amplitude.events","run-amplitude-events-crawler","2021-05-27T04:15:00.000000Z",1]
   ```
   
   <img width="1297" alt="Screen Shot 2021-05-28 at 9 15 32 AM" src="https://user-images.githubusercontent.com/74351/120013160-43980280-bf95-11eb-95ce-5c6196993f7b.png">
   
   <img width="746" alt="Screen Shot 2021-05-28 at 9 14 58 AM" src="https://user-images.githubusercontent.com/74351/120013097-2fec9c00-bf95-11eb-9e1a-bd062b8fba0c.png">
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - airflow 1.10.14
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value schedulerNumRuns is set  -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.
   - when checking log on scheduler, it has error like this
   `[2021-04-16 13:06:36,392] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 3497') Celery Task ID: ('XXXXXXXX_YYY_test', 'Task_blahblahblah', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)`
   
   - i reported here: https://github.com/helm/charts/issues/19399 but found the issue is already closed.
   
   **I tried to experiment the ...., which did not help as i expected.**
   - Uploaded the dag with new name/id. enabled, cleared the dag (otherwise the 1st task just stuck at the 'queued' state)
   and 1st task is at the 'scheduled' state and stuck there.
   - check scheduler log:
   - `[2021-04-16 15:58:51,991] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 1851')
   Celery Task ID: ('XXXXXX_min_test_3', 'Load_XXXX_to_YYYY', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/executors/celery_executor.py", line 117, in fetch_celery_task_state
       res = (celery_task[0], celery_task[1].state)`
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] trucnguyenlam edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
trucnguyenlam edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-883251087


   @kaxil we're still experiencing this issue in version 2.1.1 even after tuning `parallelism` and `pool` size to 1024


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] pelaprat commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
pelaprat commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-827627834


   Although I [was optimistic](https://github.com/apache/airflow/issues/13542#issuecomment-825117101), we've seen the behavior return where queued tasks are stuck in a queued state even though there are slots open. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853693757


   I have resolved this issue for my environment! I'm not sure if this is the same "bug" as others, or a different issue with similar symptoms. But here we go
   
   ----
   
   In my Airflow Docker image, the entrypoint is just a bootstrap script that accepts `webserver` or `scheduler` as arguments, and executes the appropriate command.
   ```
   # Installing python libs, jars, etc
   ...
   ENTRYPOINT ["/bootstrap.sh"]
   ```
   
   `bootstrap.sh`:
   ```
   if [ "$1" = "webserver" ]
   then
   	exec airflow webserver
   fi
   
   if [ "$1" = "scheduler" ]
   then
   	exec airflow scheduler
   fi
   ```
   
   Previous to #12766, the KubernetesExecutor fed the `airflow tasks run` (or `airflow run` in older versions) into the `command` section of pod YAML.
   
   ```
   "containers": [
         {
           "args": [],
           "command": [
             "airflow",
             "run",
             "my_dag",
             "my_task",
             "2021-06-03T03:40:00+00:00",
             "--local",
             "--pool",
             "default_pool",
             "-sd",
             "/usr/local/airflow/dags/my_dag.py"
           ],
   ```
   This works fine for my setup -- the `command` just overrides my Docker's `ENTRYPOINT`, the pod executes its given command and terminates on completion. However, [this](https://github.com/apache/airflow/pull/12766/files#diff-681de8974a439f70dfa41705f5c1681ecce615fac6c4c715c1978d28d8f0da84L300) change moved the `airflow tasks run` issuance to the `args` section of the YAML. 
   ```
   'containers': [{'args': ['airflow',
                                      'tasks',
                                      'run',
                                      'my_dag,
                                      'my_task',
                                      '2021-06-02T00:00:00+00:00',
                                      '--local',
                                      '--pool',
                                      'default_pool',
                                      '--subdir',
                                      '/usr/local/airflow/dags/my_dag.py'],
                             'command': None,
   ```
   
   These new args do not match either `webserver` or `scheduler` in `bootstrap.sh`, therefore the script ends cleanly and so does the pod. Here is my solution, added to the bottom of `bootstrap.sh`:
   ```
   if [ "$1" = "airflow" ] && [ "$2" = "tasks" ] && [ "$3" = "run" ]
   then
   	exec "$@"
   fi
   ```
   Rather than just allow the pod to execute _whatever_ it's given in `args` (aka just running `exec "$@"` without a check), I decided to at least make sure the pod is being fed an `airflow run task` command.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853117451


   More data, if it can benefit:
   I'm using Airflow `2.1.0` (upgraded from `1.10.15` in our development environment) and trying to use the `KubernetesExecutor`. I am running _extremely_ basic DAGs that exist solely to test. In `airflow.cfg` I set `delete_worker_pods = False` so that I could try and examine what's going on. As far as I can tell, my worker pods are being created, immediately considering themselves successful (not sure if they actually receive a task to complete), and terminating. I don't think the pods are "successful but not writing the success state to the DB" because one of my tasks is a simple `CREATE TABLE` statement on my data warehouse, and the table does not appear.
   
   
   Here are the results of various logs:
   
   If I run `kubectl get pods`, I see my `airflow-webserver` and `airflow-scheduler` pods, as well as some "Completed" worker pods.
   
   ```
   NAME                                                           READY   STATUS      RESTARTS   AGE
   airflow-scheduler                                              1/1     Running     0          45m
   airflow-webserver-1                                            2/2     Running     0          56m
   airflow-webserver-2                                            2/2     Running     0          56m
   airflow-webserver-3                                            2/2     Running     0          45m
   airflow-webserver-4                                            2/2     Running     0          45m
   airflow-webserver-5                                            2/2     Running     0          45m
   <GENERATED_WORKER_POD_1>                                       0/1     Completed   0          15m
   <GENERATED_WORKER_POD_2>                                       0/1     Completed   0          56m
   <GENERATED_WORKER_POD_3>                                       0/1     Completed   0          45m
   ```
   
   `kubectl logs <my_worker_pod>` - I can tell that the pod was briefly alive but that did nothing, because the only output is a line that always appears as a side effect of the Docker image config. In contrast, if I get the logs from one of my functional `1.10.15` pods, I can see the task start:
   ```
   [2021-06-02 13:33:01,358] {__init__.py:50} INFO - Using executor LocalExecutor
   [2021-06-02 13:33:01,358] {dagbag.py:417} INFO - Filling up the DagBag from /usr/local/airflow/dags/my_dag.py
   ...
   etc
   ```
   
   `kubectl describe pod <my_worker_pod>` - the event log is quite tame:
   ```
   Events:
     Type    Reason     Age    From               Message
     ----    ------     ----   ----               -------
     Normal  Scheduled  3m21s  default-scheduler  Successfully assigned <GENERATED_POD_NAME> to <EC2_INSTANCE>
     Normal  Pulling    3m19s  kubelet            Pulling image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Pulled     3m19s  kubelet            Successfully pulled image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Created    3m19s  kubelet            Created container base
     Normal  Started    3m19s  kubelet            Started container base
   ```
   
   `kubectl logs airflow-scheduler` - I've trimmed the logs significantly, but here are the statements mentioning a stuck task
   ```
   [2021-06-02 14:49:42,742] {kubernetes_executor.py:369} INFO - Attempting to finish pod; pod_id: <GENERATED_POD_NAME>; state: None; annotations: {'dag_id': '<TEST_DAG>', 'task_id': '<TASK_ID>', 'execution_date': '2021-05-30T00:00:00+00:00', 'try_number': '1'}
   [2021-06-02 14:49:42,748] {kubernetes_executor.py:546} INFO - Changing state of (TaskInstanceKey(dag_id='<TEST_DAG>', task_id='<TASK_ID>', execution_date=datetime.datetime(2021, 5, 30, 0, 0, tzinfo=tzlocal()), try_number=1), None, '<GENERATED_POD_NAME>', 'airflow', '1000200742') to None
   [2021-06-02 14:49:42,751] {scheduler_job.py:1212} INFO - Executor reports execution of <TASK_ID> execution_date=2021-05-30 00:00:00+00:00 exited with status None for try_number 1
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861304352


   > `kill -USR2 <pid of scheduler>` -- how you get the pid depends upon how and where you are running it :)
   > 
   > Likely exec in to the container, run `ps auxww` and find the oldest scheduler processs (you'll see some sub processes, possibly named helpfully).
   
   Thanks a lot @ashb. This command allowed me to investigate the issue better and finally helped me figure the problem out.
   Expect a PR from me this week :)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817118815


   I observed this again last night and the reason was image pull error inside the pods. 
   
   When there's image pull error, the pod is still in 'Pending' phase and airflow sees it as such therefore tasks stay queued. 
   
   The kubernetes pod watcher does not watch the containers inside the pods. It watches only the pods and report the status as pending therefore tasks are queued.
   
   After much digging, the only solution I can think of is using timeout to delete the pods because I couldn't find a way we could watch the containers inside the pods and report the container status.
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029786174


   @jpkoponen I adapted the @danmactough DAG to make it automatic and fit our use case. In filter DAGs that have been stuck for more than 5 minutes, and simply delete them. (In my case, changing the `try_number` and the state has no other effect than queuing the DAG again).
   
   ```python
   import os
   import requests
   import time
   import json
   from datetime import datetime, timedelta
   from pprint import pprint
   
   from airflow import DAG
   from airflow.models.dagrun import DagRun
   from airflow.models.taskinstance import TaskInstance
   from airflow.operators.python import PythonOperator
   from airflow.utils import timezone
   from airflow.utils.db import provide_session
   from airflow.utils.state import State
   
   DAG_NAME = os.path.splitext(os.path.basename(__file__))[0]
   DEFAULT_ARGS = {
       "owner": "airflow",
       "depends_on_past": False,
       "email_on_failure": False,
       "email_on_retry": False,
       "execution_timeout": timedelta(minutes=10),
       "retries": 0,
   }
   
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       filter = [
           TaskInstance.state == State.QUEUED,
           TaskInstance.queued_dttm < datetime.now(timezone.utc) - timedelta(minutes=5)
       ]
   
       tis = session.query(TaskInstance).filter(*filter).all()
       print(f"Task instances: {tis}")
       print(f"Updating {len(tis)} task instances")
   
   
       for ti in tis:
           dr = (
               session.query(DagRun)
               .filter(DagRun.run_id == ti.dag_run.run_id)
               .first()
           )
   
           dagrun = (
               dict(
                   id=dr.id,
                   dag_id=dr.dag_id,
                   execution_date=dr.execution_date,
                   start_date=dr.start_date,
                   end_date=dr.end_date,
                   _state=dr._state,
                   run_id=dr.run_id,
                   creating_job_id=dr.creating_job_id,
                   external_trigger=dr.external_trigger,
                   run_type=dr.run_type,
                   conf=dr.conf,
                   last_scheduling_decision=dr.last_scheduling_decision,
                   dag_hash=dr.dag_hash,
               )
               if dr
               else {}
           )
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
   
   
           dr.state = State.FAILED
           print(f"Deleting {str(ti)}.")
           session.delete(ti)
   
       session.commit()
       print("Done.")
   
   
   with DAG(
       DAG_NAME,
       description="Utility DAG to fix TaskInstances stuck in queued state",
       default_args=DEFAULT_ARGS,
       schedule_interval="*/5 * * * *",
       start_date=datetime(year=2021, month=8, day=1),
       max_active_runs=1,
       catchup=False,
       default_view="graph",
       is_paused_upon_creation=False,
   ) as dag:
       PythonOperator(task_id="unstick_dag", python_callable=unstick_dag_callable)
   ```
      The DAG runs every 5 minutes and I never caught it in a queued state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029786174


   @jpkoponen I adapted the @danmactough DAG to make it automatic and fit our use case. It filters tasks that have been stuck for more than 5 minutes, and simply deletes them. (In my case, changing the `try_number` and the state has no other effect than queuing the task again).
   
   ```python
   import os
   import requests
   import time
   import json
   from datetime import datetime, timedelta
   from pprint import pprint
   
   from airflow import DAG
   from airflow.models.dagrun import DagRun
   from airflow.models.taskinstance import TaskInstance
   from airflow.operators.python import PythonOperator
   from airflow.utils import timezone
   from airflow.utils.db import provide_session
   from airflow.utils.state import State
   
   DAG_NAME = os.path.splitext(os.path.basename(__file__))[0]
   DEFAULT_ARGS = {
       "owner": "airflow",
       "depends_on_past": False,
       "email_on_failure": False,
       "email_on_retry": False,
       "execution_timeout": timedelta(minutes=10),
       "retries": 0,
   }
   
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       filter = [
           TaskInstance.state == State.QUEUED,
           TaskInstance.queued_dttm < datetime.now(timezone.utc) - timedelta(minutes=5)
       ]
   
       tis = session.query(TaskInstance).filter(*filter).all()
       print(f"Task instances: {tis}")
       print(f"Updating {len(tis)} task instances")
   
   
       for ti in tis:
           dr = (
               session.query(DagRun)
               .filter(DagRun.run_id == ti.dag_run.run_id)
               .first()
           )
   
           dagrun = (
               dict(
                   id=dr.id,
                   dag_id=dr.dag_id,
                   execution_date=dr.execution_date,
                   start_date=dr.start_date,
                   end_date=dr.end_date,
                   _state=dr._state,
                   run_id=dr.run_id,
                   creating_job_id=dr.creating_job_id,
                   external_trigger=dr.external_trigger,
                   run_type=dr.run_type,
                   conf=dr.conf,
                   last_scheduling_decision=dr.last_scheduling_decision,
                   dag_hash=dr.dag_hash,
               )
               if dr
               else {}
           )
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
   
   
           dr.state = State.FAILED
           print(f"Deleting {str(ti)}.")
           session.delete(ti)
   
       session.commit()
       print("Done.")
   
   
   with DAG(
       DAG_NAME,
       description="Utility DAG to fix TaskInstances stuck in queued state",
       default_args=DEFAULT_ARGS,
       schedule_interval="*/5 * * * *",
       start_date=datetime(year=2021, month=8, day=1),
       max_active_runs=1,
       catchup=False,
       default_view="graph",
       is_paused_upon_creation=False,
   ) as dag:
       PythonOperator(task_id="unstick_dag", python_callable=unstick_dag_callable)
   ```
      The DAG runs every 5 minutes and I never caught it in a queued state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jpkoponen edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jpkoponen edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029887270


   Thank you @val2k !
   
   It's interesting that increasing `try_number` and modifying the `STATE` did not work for you. I will first try the automatic method without deleting hoping that it will work for me.
   
   I just wish there was a way to test the DAG other than waiting for the issue to arise again. I think we have had it happen only twice in the past two weeks. 😄 
   
   Edit: Actually, we didn't have it happen in prod but in dev there were two instances last night. I tried to use the utility DAG to increase their `try_number` but it didn't work. 😞 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] cdibble commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
cdibble commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1020844854


   > 
   
   I am having a similar experience. I'm on `Airflow v2.2.2` with `KubernetesExecutor` in AWS EKS. I observe when a DAG run kicks off, tasks are quickly queued, but some relatively small fraction of the queued tasks are moved to a running state. At the same time, kubernetes reports `FailedCreatePodSandBox` on tasks that are in the queued state. My scheduler has plenty of resources, but I see the `OSError: handle is closed` error coming up in the scheduler logs right around when queued tasks stop being moved to the running state. Those tasks stay queued until some of the already-executing tasks complete. In my case, those tasks will eventually be set to running and complete, but I can never seem to get the concurrency I want. There's plenty of room in the pool, and I am fairly certain I am not limited by the various concurrency/parallelism settings.
   
   Below are the scheduler logs when I get a similar `OSError` as @danmactough as well as the kubectl events logs from the time when a new DAG run was executing and tasks were moving from scheduled to queued to executing. 
   
   scheduler logs
   ```
   [2022-01-25 05:49:54,253] {kubernetes_executor.py:147} INFO - Event: <redacted_A> had an event of type MODIFIED
   [2022-01-25 05:49:54,253] {kubernetes_executor.py:213} INFO - Event: <redacted_A> is Running
   [2022-01-25 05:49:54,654] {kubernetes_executor.py:147} INFO - Event: <redacted_B> had an event of type MODIFIED
   [2022-01-25 05:49:54,654] {kubernetes_executor.py:213} INFO - Event: <redacted_B> is Running
   [2022-01-25 05:49:55,054] {kubernetes_executor.py:147} INFO - Event: <redacted_C> had an event of type MODIFIED
   [2022-01-25 05:49:55,054] {kubernetes_executor.py:201} INFO - Event: <redacted_C> Pending
   [2022-01-25 05:49:55,653] {kubernetes_executor.py:147} INFO - Event: <redacted_D> had an event of type MODIFIED
   [2022-01-25 05:49:55,653] {kubernetes_executor.py:213} INFO - Event: <redacted_D> is Running
   [2022-01-25 05:50:06,704] {kubernetes_executor.py:454} INFO - Found 15 queued task instances
   [2022-01-25 05:50:45,076] {processor.py:244} WARNING - Killing DAGFileProcessorProcess (PID=12643)
   Process ForkProcess-22:
   Traceback (most recent call last):
     File "/usr/local/lib/python3.8/multiprocessing/process.py", line 315, in _bootstrap
       self.run()
     File "/usr/local/lib/python3.8/multiprocessing/process.py", line 108, in run
       self._target(*self._args, **self._kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 277, in _run_processor_manager
       processor_manager.start()
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 510, in start
       return self._run_parsing_loop()
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 570, in _run_parsing_loop
       self._collect_results_from_processor(processor)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 894, in _collect_results_from_processor
       if processor.result is not None:
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/processor.py", line 322, in result
       if not self.done:
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/dag_processing/processor.py", line 287, in done
       if self._parent_channel.poll():
     File "/usr/local/lib/python3.8/multiprocessing/connection.py", line 255, in poll
       self._check_closed()
     File "/usr/local/lib/python3.8/multiprocessing/connection.py", line 136, in _check_closed
       raise OSError("handle is closed")
   OSError: handle is closed
   [2022-01-25 05:50:45,858] {manager.py:308} WARNING - DagFileProcessorManager (PID=12136) exited with exit code 1 - re-launching
   [2022-01-25 05:50:45,861] {manager.py:163} INFO - Launched DagFileProcessorManager with pid: 12873
   [2022-01-25 05:50:45,868] {settings.py:52} INFO - Configured default timezone Timezone('UTC')
   [2022-01-25 05:50:45,870] {settings.py:462} INFO - Loaded airflow_local_settings from /opt/airflow/config/airflow_local_settings.py .
   [2022-01-25 05:50:53,756] {scheduler_job.py:1114} INFO - Resetting orphaned tasks for active dag runs
   [2022-01-25 05:51:07,317] {kubernetes_executor.py:454} INFO - Found 15 queued task instances
   ```
   
   kubectl events
   ```
   
   60m         Normal    Created                  pod/<redacted_A>                               Created container base
   60m         Warning   Failed                   pod/<redacted_B>                               Error: context deadline exceeded
   60m         Normal    Pulled                   pod/<redacted_C>                               Container image "k8s.gcr.io/git-sync/git-sync:v3.3.0" already present on machine
   60m         Normal    Pulled                   pod/<redacted_D>                               Container image "k8s.gcr.io/git-sync/git-sync:v3.3.0" already present on machine
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_E>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "gefs50etlv4extracts3getatmos2000.88a63f73338a4f38b67f248ded8726dc": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_F>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to start sandbox container for pod "<redacted_F>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_G>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_G>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_H>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_H>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_I>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_I>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_J>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_J>": operation timeout: context deadline exceeded
   60m         Warning   FailedCreatePodSandBox   pod/<redacted_K>                               Failed to create pod sandbox: rpc error: code = Unknown desc = failed to create a sandbox for pod "<redacted_K>": operation timeout: context deadline exceeded
   ```
   
   Happy to provide more info if requested.
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] DVerzal commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
DVerzal commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1048969780


   We've upgraded to a 2.2.2 MWAA environment and are encountering the similar queuing behavior. Tasks remain in the queued state for about fifteen minutes before executing. This is in an extremely small dev environment we're testing. Unfortunately, even the unstick_tag task remains in a queued state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] danmactough commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
danmactough commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1011598836


   Airflow 2.0.2+e494306fb01f3a026e7e2832ca94902e96b526fa (MWAA on AWS)
   
   This happens to us a LOT: a DAG will be running, task instances will be marked as "queued", but nothing gets moved to "running".
   
   When this happened today (the first time today), I was able to track down the following error in the scheduler logs:
   
   ![2022-01-12 at 7 16 PM](https://user-images.githubusercontent.com/357481/149243393-0f0b5b91-d1f7-4a51-8a43-3eab644a49e7.png)
   
   At some point after the scheduler had that exception, I tried to clear the state of the queued task instances to get them to run. That resulting in the following logs:
   
   ![2022-01-12 at 7 18 PM](https://user-images.githubusercontent.com/357481/149243535-3ebfd0b1-31af-43aa-99e2-7ee5aa1dbaff.png)
   
   This corresponds to this [section of code](https://github.com/apache/airflow/blob/2.0.2/airflow/executors/base_executor.py#L73-L85):
   
   ![2022-01-12 at 10 38 AM](https://user-images.githubusercontent.com/357481/149171972-e9824366-6e85-4c2e-a00c-5ee66d466de8.png)
   
   My conclusion is that when the scheduler experienced that error, it entered a pathological state: it was running but had bad state in memory. Specifically, the queued task instances were in the `queued_tasks` or `running` in-memory cache, and thus any attempts to re-queue those tasks would fail as long as that scheduler process was running because the tasks would appear to already have been queued and/or running.
   
   Both caches use the [`TaskInstanceKey`](https://github.com/apache/airflow/blob/10023fdd65fa78033e7125d3d8103b63c127056e/airflow/models/taskinstance.py#L224-L230), which is made up of `dag_id` (which we can't change), `task_id` (which we can't change), `execution_date` (nope, can't change), and `try_number` (🎉 we can change this!!).
   
   So to work around this, I created a utility DAG that will find all task instances in a "queued" or "None" state and increment the `try_number` field.
   
   The DAG runs as a single `PythonOperator` with the following callable:
   
   ```python
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       dag_id = dag_run.conf.get("dag_id")
       if not dag_id:
           raise AssertionError("dag_id was not provided")
       execution_date = dag_run.conf.get("execution_date")
       if not execution_date:
           raise AssertionError("execution_date was not provided")
       execution_date = parse(execution_date)
   
       filter = [
           or_(TaskInstance.state == State.QUEUED, TaskInstance.state == State.NONE),
           TaskInstance.dag_id == dag_id,
           TaskInstance.execution_date == execution_date,
       ]
       print(
           (
               f"DAG id: {dag_id}, Execution Date: {execution_date}, State: "
               f"""{dag_run.conf.get("state", f"{State.QUEUED} or {State.NONE}")}, """
               f"Filter: {[str(f) for f in filter]}"
           )
       )
   
       tis = session.query(TaskInstance).filter(*filter).all()
       dr = (
           session.query(DagRun)
           .filter(DagRun.dag_id == dag_id, DagRun.execution_date == execution_date)
           .first()
       )
       dagrun = (
           dict(
               id=dr.id,
               dag_id=dr.dag_id,
               execution_date=dr.execution_date,
               start_date=dr.start_date,
               end_date=dr.end_date,
               _state=dr._state,
               run_id=dr.run_id,
               creating_job_id=dr.creating_job_id,
               external_trigger=dr.external_trigger,
               run_type=dr.run_type,
               conf=dr.conf,
               last_scheduling_decision=dr.last_scheduling_decision,
               dag_hash=dr.dag_hash,
           )
           if dr
           else {}
       )
   
       print(f"Updating {len(tis)} task instances")
       print("Here are the task instances we're going to update")
       # Print no more than 100 tis so we don't lock up the session too long
       for ti in tis[:100]:
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
       if len(tis) > 100:
           print("Output truncated after 100 task instances")
   
       for ti in tis:
           ti.try_number = ti.next_try_number
           ti.state = State.NONE
           session.merge(ti)
   
       if dag_run.conf.get("activate_dag_runs", True):
           dr.state = State.RUNNING
           dr.start_date = timezone.utcnow()
   
       print("Done")
   ```
   
   Moments after I shipped this DAG, another DAG got stuck, and I had a chance to see if this utility DAG worked -- it did! 😅 
   
   -----
   
   Couple of thoughts:
   
   - I don't think my error is exactly the same as OP, as some very key conditions are not applicable to my case, but this issue appears to have many different and probably not at all related bugs that kind of manifest as "stuck DAGs" and this issue has pretty good Google juice -- I just hope my explanation and/or work-around help someone else.
   - The MWAA product from AWS is using an older version of Airflow, so the combination of factors that leads to this pathological state may no longer be possible in the current version of Airflow.
   - MWAA uses the CeleryExecutor, which I suspect is where the pathological state is coming from, not BaseExecutor directly.
   - All that being said, I'm surprised to see this critical state being kept in memory (`queued_tasks` and `running`), but I don't have a complete mental model of how the executor and the scheduler are distinct or not. My understanding is that this is scheduler code, but with the scheduler being high-availability (we're running 3 schedulers), in-memory state seems like something we should be using very judiciously and be flushing and rehydrating from the database regularly.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JavierLopezT edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1007235056


   This has started happening to us as well. Randomly (3 nights over 10), all tasks get stuck in queued state and we have to manually clear the state for it to recover. We have seen that it is related to the decrease of this metric in datadog:
   
   ![image](https://user-images.githubusercontent.com/11339132/148517399-95681f0c-260e-48cb-9b38-b7d0556d8df9.png)
   
   Airflow 2.1.1 in EC2
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] WattsInABox commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
WattsInABox commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1045166294


   @danmactough Did this update help? We're tracking this issue before upgrading.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] timgriffiths commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
timgriffiths commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-854356862


   Environment:
   
   Airflow 2.1.0
   Docker : apache/airflow:2.1.0-python3.8
   Executor : Celery
   
   So I have also hit this today, the way i was able to replicate this 
   
   -> have a py file that generates move than 1 dag
   -> then have a user break 1 of the dag from this resultant chain
   -> all the other still valid dags will be triggered in the scheduler but fail in the Worker but it looks like the exception is not passed back to the Scheduler so it permanently stays in a "queued state"
   
   `[2021-06-04 01:11:22,927: INFO/ForkPoolWorker-40] Executing command in Celery: ['airflow', 'tasks', 'run', 'myworkingjob', 'taskname-abc', '2021-06-04T01:11:21.775097+00:00', '--local', '--pool', 'pool', '--subdir', '/opt/airflow/dags/mm-dynamic-dags.py']
   [2021-06-04 01:11:22,940: INFO/ForkPoolWorker-34] Filling up the DagBag from /opt/airflow/dags/mm-dynamic-dags.py
   [2021-06-04 01:11:27,993: ERROR/ForkPoolWorker-39] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dagbag.py", line 317, in _load_modules_from_file
       loader.exec_module(new_module)
     File "<frozen importlib._bootstrap_external>", line 848, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 187, in <module>
       create_dag("{}/{}".format(global_dags_base_url, dag_file))
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 123, in create_dag
       dag = DAG(
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dag.py", line 277, in __init__
       validate_key(dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/helpers.py", line 44, in validate_key
       raise AirflowException(
   airflow.exceptions.AirflowException: The key (test-test+abc) has to be made of alphanumeric characters, dashes, dots and underscores exclusively
   [2021-06-04 01:11:27,996: ERROR/ForkPoolWorker-39] Failed to execute task dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse..
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 116, in _execute_in_fork
       args.func(args)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 91, in wrapper
       return f(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/commands/task_command.py", line 219, in task_run
       dag = get_dag(args.subdir, args.dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 191, in get_dag
       raise AirflowException(
   airflow.exceptions.AirflowException: dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse.
   [2021-06-04 01:11:28,017: ERROR/ForkPoolWorker-39] Task airflow.executors.celery_executor.execute_command[66453177-4042-4c6a-882b-bd0fcdbda0d8] raised unexpected: AirflowException('Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm')
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 412, in trace_task
       R = retval = fun(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 704, in __protected_call__
       return self.run(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 87, in execute_command
       _execute_in_fork(command_to_exec)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 98, in _execute_in_fork
       raise AirflowException('Celery command failed on host: ' + get_hostname())
   airflow.exceptions.AirflowException: Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm
   [2021-06-04 01:11:28,091: ERROR/ForkPoolWorker-37] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py`
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911359911


   Thanks @ephraimbuddy! That wasn't it but I did find something that _may_ be issue-worthy. I added some custom logging to `scheduler_job._start_queued_dagruns()` and noticed that the contents of `dag_runs` was the same 20 DAGruns in every loop (default count from config `max_dagruns_per_loop_to_schedule`). The DAGruns in question show up first when ordering by the following code from `dagrun.next_dagruns_to_examine()`:
   ```
   .order_by(
       nulls_first(cls.last_scheduling_decision, session=session),
       cls.execution_date,
   )
   ```
   
   This DAG is set to `max_active_runs=1` so all 20 examined queued DAGruns do not change state (because there is another running already). The problem arises because the `_start_queued_dagruns()` function (AFAIK) doesn't update `last_scheduling_decision`, so every time the query is run to get next DAGruns, the same ones appear (and continue to not be scheduled if the currently active DAGrun for that DAG takes a long time -- and it continues so long as there are more than 20 DAGruns queued).
   
   I think the `last_scheduling_decision` column needs to be updated somewhere here:
   ```
   if dag.max_active_runs and active_runs >= dag.max_active_runs:
       self.log.debug(
           "DAG %s already has %d active runs, not moving any more runs to RUNNING state %s",
           dag.dag_id,
           active_runs,
           dag_run.execution_date,
       )
   ```
   
   I was able to get around this issue currently by simply increasing the number of DAGruns handled per loop (and telling my users not to queue so many), but perhaps it should still be addressed.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder (which is saved in the file share -PVC, and can be viewed via azure storage explorer). 
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment.  during debug of this stuck situation, the table 'dag', 'task_*', 'celery_*' had been truncated.
   ```
   - celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule
   - connections
   ```
   
   How would like me to fetch the log? Just the dag log run? 
   update: the one i can view via storage explorer - it is empty, since the task was not executed. i will check the worker itself. and update this comment.
   
   there are no errors. i will attach the log anyhow.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] shivanshs9 edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
shivanshs9 edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861406671


   > `kill -USR2 <pid of scheduler>` -- how you get the pid depends upon how and where you are running it :)
   > 
   > Likely exec in to the container, run `ps auxww` and find the oldest scheduler processs (you'll see some sub processes, possibly named helpfully).
   
   @ashb @Jorricks  I'm facing a similar problem as OP on Airflow v2.0.1 with Celery Executor. If it helps, you can find the debug information from scheduler logs:
   
   ```
   [2021-06-15 07:13:48,623] {{dag_processing.py:1071}} INFO - Finding 'running' jobs without a recent heartbeat
   [2021-06-15 07:13:48,624] {{dag_processing.py:1075}} INFO - Failing jobs without heartbeat after 2021-06-15 07:08:48.624534+00:00
   [2021-06-15 07:13:49,539] {{scheduler_job.py:757}} INFO - --------------------------------------------------------------------------------
   SIGUSR2 received, printing debug
   --------------------------------------------------------------------------------
   [2021-06-15 07:13:49,539] {{base_executor.py:302}} INFO - executor.queued (0)
   	
   [2021-06-15 07:13:49,539] {{base_executor.py:307}} INFO - executor.running (40)
   	TaskInstanceKey(dag_id='ergo_job_collector', task_id='process_job_result', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 10, 753973, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 13, 22, 40, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='ergo_task_queuer', task_id='push_tasks', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 1, 316420, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=32)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_1', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 3, 23, 693482, tzinfo=Timezone('UTC')), try_number=41)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 13, 9, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 13, 9, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=39)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 11, 6, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 11, 6, 30, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 13, 22, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 13, 9, 16, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 11, 6, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 5, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=42)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_2', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797327, tzinfo=Timezone('UTC')), try_number=40)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 32, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 15, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 3, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 0, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 30, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=41)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=30)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 11, 6, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=39)
   [2021-06-15 07:13:49,540] {{base_executor.py:308}} INFO - executor.event_buffer (0)
   	
   [2021-06-15 07:13:49,540] {{celery_executor.py:387}} INFO - executor.tasks (27)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=30), <AsyncResult: 6960c6d0-7e21-4ef9-8f04-d95efdd9d706>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_2', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797327, tzinfo=Timezone('UTC')), try_number=40), <AsyncResult: 4a815d59-d824-4373-8f96-34272174cfc0>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=39), <AsyncResult: 5296e550-5efd-474a-9e34-897330504886>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=39), <AsyncResult: cad2a01b-cde3-473e-85bc-9f96eae7da7c>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: cae54db9-56a5-4a27-8226-1dc5867265f7>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 438e5379-ef8d-4bfe-8e55-cae461a3f62f>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 9a953b60-b222-4d4c-aff4-5b831caef71f>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 840e404c-1cee-4c18-8123-b354a63e8d80>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 4410416b-7132-4713-a78d-caa0ce3cacb0>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 91d5060a-ff5c-4997-832a-757c25069ac7>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 10, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ade2d701-f632-41d9-a901-40531619dfba>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: febb3b1e-0f63-41d3-83dc-58f5b11a39ca>)
   	(TaskInstanceKey(dag_id='ergo_job_collector', task_id='process_job_result', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 10, 753973, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 390f1451-7f44-4415-8c57-e2542ed6b6d2>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 30, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 7926f732-e671-4f64-afaf-7c352a85a929>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 32, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ba1edc8e-66b4-4a20-9cc6-5b0c5758d8c4>)
   	(TaskInstanceKey(dag_id='ergo_task_queuer', task_id='push_tasks', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 1, 316420, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 0ef80d4f-1c4c-4e62-b5ed-cebd0b3ba075>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: fea3edfa-477d-4fb0-a580-03ea91faaef1>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 4749177d-df70-49cf-a2fb-9b0a230d5ff4>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 0, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: bf04f105-efc7-41be-864c-8161c4973f6b>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 3, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 7c8c6550-cdcc-4405-816b-15da1dbeb79e>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 15, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ee255fe6-21d2-4d3f-8860-83e55265fea9>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 46b77d60-43b5-41e6-99eb-e407e0d74791>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 5, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ed5115f9-dec6-49da-a5bf-15860a907ab4>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=32), <AsyncResult: 49ee2d3f-eef1-4f7a-a5da-923e8cb8b914>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_1', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 3, 23, 693482, tzinfo=Timezone('UTC')), try_number=41), <AsyncResult: 910a1d67-658d-4fe9-954a-2d093b64c95d>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=42), <AsyncResult: 8061f49f-8050-4b73-84fd-4a0222369040>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=41), <AsyncResult: 262cf0ac-1132-4176-aba0-77ec4a2bddb7>)
   [2021-06-15 07:13:49,540] {{celery_executor.py:390}} INFO - executor.adopted_task_timeouts (0)
   	
   [2021-06-15 07:13:49,540] {{scheduler_job.py:760}} INFO - --------------------------------------------------------------------------------
   [2021-06-15 07:13:51,028] {{dag_processing.py:838}} INFO -
   ================================================================================
   DAG File Processing Stats
   
   File Path                                                                                           PID  Runtime      # DAGs    # Errors  Last Runtime    Last Run
   ------------------------------------------------------------------------------------------------  -----  ---------  --------  ----------  --------------  -------------------
   /home/airflow/.local/lib/python3.8/site-packages/airflow/smart_sensor_dags/__init__.py                                     0           0  0.09s           2021-06-15T07:13:50
   /opt/airflow/dags/dag_ergo.py                                                                                              2           0  0.47s           2021-06-15T07:13:51
   /home/airflow/.local/lib/python3.8/site-packages/airflow/smart_sensor_dags/smart_sensor_group.py  16087  0.00s             5           0  0.15s           2021-06-15T07:13:50
   /opt/airflow/dags/dag_dagen.py                                                                    16081  1.48s            90           0  4.10s           2021-06-15T07:13:49
   ================================================================================
   ```
   
   Workers have already finished executing these tasks but the scheduler is stuck and doesn't schedule any new tasks. The only fix that works is by manually terminating and restarting the scheduler pod.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853693757


   I have resolved this issue for my environment! I'm not sure if this is the same "bug" as others, or a different issue with similar symptoms. But here we go
   
   ----
   
   In my Airflow Docker image, the entrypoint is just a bootstrap script that accepts `webserver` or `scheduler` as arguments, and executes the appropriate command.
   ```
   # Installing python libs, jars, etc
   ...
   ENTRYPOINT ["/bootstrap.sh"]
   ```
   
   `bootstrap.sh`:
   ```
   if [ "$1" = "webserver" ]
   then
   	exec airflow webserver
   fi
   
   if [ "$1" = "scheduler" ]
   then
   	exec airflow scheduler
   fi
   ```
   
   Previous to #12766, the KubernetesExecutor fed the `airflow tasks run` (or `airflow run` in older versions) into the `command` section of pod YAML.
   
   ```
   "containers": [
         {
           "args": [],
           "command": [
             "airflow",
             "run",
             "my_dag",
             "my_task",
             "2021-06-03T03:40:00+00:00",
             "--local",
             "--pool",
             "default_pool",
             "-sd",
             "/usr/local/airflow/dags/my_dag.py"
           ],
   ```
   This works fine for my setup -- the `command` just overrides my Docker's `ENTRYPOINT`, the pod executes its given command and terminates on completion. However, [this](https://github.com/apache/airflow/pull/12766/files#diff-681de8974a439f70dfa41705f5c1681ecce615fac6c4c715c1978d28d8f0da84L300) change moved the `airflow tasks run` issuance to the `args` section of the YAML. 
   ```
   'containers': [{'args': ['airflow',
                                      'tasks',
                                      'run',
                                      'my_dag,
                                      'my_task',
                                      '2021-06-02T00:00:00+00:00',
                                      '--local',
                                      '--pool',
                                      'default_pool',
                                      '--subdir',
                                      '/usr/local/airflow/dags/my_dag.py'],
                             'command': None,
   ```
   
   These new args do not match either `webserver` or `scheduler` in `bootstrap.sh`, therefore the script ends cleanly and so does the pod. Here is my solution, added to the bottom of `bootstrap.sh`:
   ```
   if [ "$1" = "airflow" ] && [ "$2" = "tasks" ] && [ "$3" = "run" ]
   then
   	exec "$@"
   fi
   ```
   Rather than just allow the pod to execute _whatever_ it's given in `args` (aka just running `exec "$@"` without a check), I decided to at least make sure the pod is being fed an `airflow run task` command.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] pelaprat commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
pelaprat commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-825117101


   We've updated to Airflow 2.0.2 and, after one DAG run, the problem did not resurface. None of the 2,000 or so tasks that are executed by the DAG ended up in a zombie state. We're going to keep watch over this for the next few days, but the initial results look promising.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] shivanshs9 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
shivanshs9 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861583602


   > @shivanshs9 did you try what I mentioned? 
   > Stopping the scheduler, then clearing the Task Instances and starting the scheduler again?
   > 
   
   @Jorricks yeah, restarting the scheduler works. Haven't tried clearing the task instances, since I mark them as failed and then run them again. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] pelaprat commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
pelaprat commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-822527551


   I agree, this is a critical issue for us. At the moment, we have customers using our software that need to manually clear these tasks and/or restart the scheduler. This isn't a sustainable solution.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853117451


   More data, if it can benefit:
   I'm using Airflow `2.1.0` (upgraded from `1.10.15` in our development environment) and trying to use the `KubernetesExecutor`. I am running _extremely_ basic DAGs that exist solely to test. In `airflow.cfg` I set `delete_worker_pods = False` so that I could try and examine what's going on. As far as I can tell, my worker pods are being created, immediately considering themselves successful (not sure if they actually receive a task to complete), and terminating. I don't think the pods are "successful but not writing the success state to the DB" because one of my tasks is a simple `CREATE TABLE` statement on my data warehouse.
   
   
   Here are the results of various logs:
   
   If I run `kubectl get pods`, I see my `airflow-webserver` and `airflow-scheduler` pods, as well as some "Completed" worker pods.
   
   ```
   NAME                                                           READY   STATUS      RESTARTS   AGE
   airflow-scheduler                                              1/1     Running     0          45m
   airflow-webserver-1                                            2/2     Running     0          56m
   airflow-webserver-2                                            2/2     Running     0          56m
   airflow-webserver-3                                            2/2     Running     0          45m
   airflow-webserver-4                                            2/2     Running     0          45m
   airflow-webserver-5                                            2/2     Running     0          45m
   <GENERATED_WORKER_POD_1>                                       0/1     Completed   0          15m
   <GENERATED_WORKER_POD_2>                                       0/1     Completed   0          56m
   <GENERATED_WORKER_POD_3>                                       0/1     Completed   0          45m
   ```
   
   `kubectl logs <my_worker_pod>` - I can tell that the pod was briefly alive but that did nothing, because the only output is a line that always appears as a side effect of the Docker image config. In contrast, if I get the logs from one of my functional `1.10.15` pods, I can see the task start:
   ```
   [2021-06-02 13:33:01,358] {__init__.py:50} INFO - Using executor LocalExecutor
   [2021-06-02 13:33:01,358] {dagbag.py:417} INFO - Filling up the DagBag from /usr/local/airflow/dags/my_dag.py
   ...
   etc
   ```
   
   `kubectl describe pod <my_worker_pod>` - the event log is quite tame:
   ```
   Events:
     Type    Reason     Age    From               Message
     ----    ------     ----   ----               -------
     Normal  Scheduled  3m21s  default-scheduler  Successfully assigned <GENERATED_POD_NAME> to <EC2_INSTANCE>
     Normal  Pulling    3m19s  kubelet            Pulling image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Pulled     3m19s  kubelet            Successfully pulled image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Created    3m19s  kubelet            Created container base
     Normal  Started    3m19s  kubelet            Started container base
   ```
   That's it. No further events.
   
   `kubectl logs airflow-scheduler` - I've trimmed the logs significantly, but here are the statements mentioning a stuck task
   ```
   [2021-06-02 14:49:42,742] {kubernetes_executor.py:369} INFO - Attempting to finish pod; pod_id: <GENERATED_POD_NAME>; state: None; annotations: {'dag_id': '<TEST_DAG>', 'task_id': '<TASK_ID>', 'execution_date': '2021-05-30T00:00:00+00:00', 'try_number': '1'}
   [2021-06-02 14:49:42,748] {kubernetes_executor.py:546} INFO - Changing state of (TaskInstanceKey(dag_id='<TEST_DAG>', task_id='<TASK_ID>', execution_date=datetime.datetime(2021, 5, 30, 0, 0, tzinfo=tzlocal()), try_number=1), None, '<GENERATED_POD_NAME>', 'airflow', '1000200742') to None
   [2021-06-02 14:49:42,751] {scheduler_job.py:1212} INFO - Executor reports execution of <TASK_ID> execution_date=2021-05-30 00:00:00+00:00 exited with status None for try_number 1
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jbkc85 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jbkc85 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-877542235


   > This is also why I increased the `parallelism` option to 1000 and increased the `pool` size.
   
   ooh, so it wasn't just the pool size.  That makes more sense on why it worked.
   
   That being said, @caleboverman I think stumbled onto a great solution that not only eliminates the potential of (when this is fixed) parallelism, when airflow fixes the issue, ends up scheduling so many processes it takes down the Airflow box itself.  The gist of it, is just create more pools and use the pools as a way to help queue processes.  This of course works when you have a smaller installation, but really helped our stuff go through after we implemented the solution.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ohdearaugustin edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ohdearaugustin edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849435787


   We are facing the same issue, with the kubernetes executor. K8s: 1.19.3. We use the helmchart from https://airflow-helm.github.io/charts (8.0.6), which refers to Airflow Version 2.0.1.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nitinpandey-154 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
nitinpandey-154 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-846808019


   Environment:
   - Airflow 2.1.0
   - Docker : apache/airflow:2.1.0-python3.8
   - Executor : Celery
   
   I am facing the same issue again.  This happens randomly and even after cleaning up the tasks instances and/or restarting the container is not fixing the issue.
   
   airflow-scheduler    | [2021-05-24 06:46:52,008] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 21, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 100 and queue airflow_maintenance
   airflow-scheduler    | [2021-05-24 06:46:52,008] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 21, tzinfo=Timezone('UTC')), try_number=1)
   airflow-scheduler    | [2021-05-24 06:46:52,008] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 24, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 100 and queue airflow_maintenance
   airflow-scheduler    | [2021-05-24 06:46:52,009] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='airflow_health_checkup', task_id='send_heartbeat', execution_date=datetime.datetime(2021, 5, 24, 4, 24, tzinfo=Timezone('UTC')), try_number=1)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] maxcountryman commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
maxcountryman commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-924064575


   We're continuing to see an issue in `2.1.3` where DAG tasks appear to be running for many hours, even days, but are making no visible progress. This situation persists through scheduler restarts and is not resolved until we clear each stuck task manually twice. The first time we clear the task we see an odd rendering in the UI with a dark blue border that looks like this:
   
   <img width="637" alt="Screen Shot 2021-09-21 at 7 40 43 AM" src="https://user-images.githubusercontent.com/74351/134193377-7f826066-37cc-4f48-bb6f-b2b2224a6be7.png">
   
   The second time we clear the task the border changes to light green and it usually completes as expected.
   
   This is a pretty frustrating situation because the only known path to remediation is manual intervention. As previously stated we're deploying to ECS, with each Airflow process as its own ECS service in an ECS cluster; this generally works well except as noted here.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] gabrielsyapse removed a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
gabrielsyapse removed a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-906778140






-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817698207


   @ephraimbuddy no I do not have that issue.
   
   When I try to observe it closely, it always goes like this:
   
   * There are 3 tasks in the `mssql_dwh` pool. All of them have the state `queued`. Nothing is running. Nothing is started. The scheduler does not start anything new, because the pool has 0 available slots.
   * Then I clear those 3 tasks.
   * The scheduler immediately picks some tasks and puts them into `queued` state. Meanwhile, Kubernetes starts the pods.
   * If I am lucky, some of the tasks get executed properly, and the scheduler continues what it is supposed to do.
   * But not long, it starts to accumulate "dead" tasks in `queued` state. Those are NOT running in Kubernetes.
   * I checked the scheduler for error logs, and I can see some log lines like these
   
   ```
   ERROR - Executor reports task instance <TaskInstance: <redacted> 2021-04-10 00:00:00+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
   ```
   
   So think there must be some kind of race condition between the scheduler and the Kubernetes pod startup.
   Some tasks finish really quickly (successfully so) and the scheduler KEEPS them in `queued` state.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hafid-d commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hafid-d commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-897005699


   So I noticed when a dag is running alone everything goes well. Then, when another dag starts running at the same time, every thing gets stuck after few minutes. No more logs in the airflow celery worker, it doesnt receive anything from the scheduler. I increased the values mentionned above, as well as dag_concurrency. But nothing happened. Any idea how I can solve this issue ? thanks


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hafid-d commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hafid-d commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-896897010


   hi @jbkc85 thank you for the suggestion! May I know how to create different pools in Airflow ? many thanks!


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jpkoponen commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jpkoponen commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029955126


   @val2k I may know why increasing the `try_number` didn't work. The DAG file that @danmactough posted is missing `session.commit()` after `session.merge()`. After the addition, the DAG file worked as intended for me.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1053579845


   > We've upgraded to a 2.2.2 MWAA environment and are encountering the similar queuing behavior. Tasks remain in the queued state for about fifteen minutes before executing. This is in an extremely small dev environment we're testing. Unfortunately, even the unstick_tag task remains in a queued state.
   
   @DVerzal  I propose you followhttps://airflow.apache.org/docs/apache-airflow/stable/faq.html?highlight=faq#why-is-task-not-getting-scheduled  - and  review resourcesa and configuration of your Airlfow and open a new issue (if this does not help) with detailed information on your configuraiton and logs. It is likely different problem than what you describe.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029725433


   @cdibble what's your setting for `AIRFLOW__KUBERNETES__WORKER_PODS_CREATION_BATCH_SIZE`? Since your task instances gets to queued state, it's not a scheduler issue but your executor. Since you're on kubernetes, I suggest you increase the above configuration if you haven't done so


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - celery executor
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value -1),  it decided to run 3 out of 4 tasks, and the last one just stuck at the queued
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    stuck at the 'scheduled' state after clear the task.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] alete89 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
alete89 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-825688001


   As many others here, we're also experiencing 0 slots, and task scheduled but never being run on Airflow 1.10.14 on kubernetes.
   restarting the scheduler pod did triggered the run for those stuck tasks. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
JavierLopezT commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1007235056


   This has started happening to us as well. Randomly (3 nights over 10 days), all tasks get stuck in queued state and we have to manually clear the state for it to recover. We have seen that it is related to the decrease of this metric in datadog:
   
   ![image](https://user-images.githubusercontent.com/11339132/148517399-95681f0c-260e-48cb-9b38-b7d0556d8df9.png)
   
   Airflow 2.1.1 in EC2
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder (which is saved in the file share -PVC, and can be viewed via azure storage explorer). 
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment. 
   `- celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule`
   
   How would like me to fetch the log? Just the dag log run? 
   update: the one i can view via storage explorer - it is empty, since the task was not executed. i will check the worker itself. and update this comment.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-876187522


   > We just updated it. We will let you know in the morning - hopefully we see some improvement!
   
   You can try the things mentioned here [in this comment](https://github.com/apache/airflow/issues/13542#issuecomment-861300533) if the issue still exists. 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - airflow 1.10.14
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value schedulerNumRuns is set  -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.
   - when checking log on scheduler, it has error like this
   `[2021-04-16 13:06:36,392] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 3497') Celery Task ID: ('XXXXXXXX_YYY_test', 'Task_blahblahblah', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)`
   
   - i reported here: https://github.com/helm/charts/issues/19399 but found the issue is already closed.
   
   **I tried to experiment the "min_file_process_interval to be 30", which did not help as i expected.**
   - Uploaded the dag with new name/id. enabled, cleared the dag (otherwise the 1st task just stuck at the 'queued' state)
   and 1st task is at the 'scheduled' state and stuck there.
   - check scheduler log:
   - `[2021-04-16 15:58:51,991] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 1851')
   Celery Task ID: ('XXXXXX_min_test_3', 'Load_XXXX_to_YYYY', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/executors/celery_executor.py", line 117, in fetch_celery_task_state
       res = (celery_task[0], celery_task[1].state)`
   
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment.  during debug of this stuck situation, the table 'dag', 'task_*', 'celery_*' had been truncated.
   ```
   - celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule
   - connections
   ```
   
   the dag log itself is empty, since the task was not executed. 
   the worker has no errors. i will attach the log anyhow.
   [log-workers.txt](https://github.com/apache/airflow/files/6327151/log-workers.txt)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder, which is saved in the file share (PVC), can be viewed via azure storage explorer. 
   The environment is set up new, and migrated a few tables from old environment. How would like me to fetch the log? Just the dag log run? 
   
   update: the one i can view via storage explorer - it is empty, since the task was not executed. i will check the worker itself. and update this comment.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] laurikoobas commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
laurikoobas commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817101896


   I ran into the same issue and it was none of these causes. What I figured out and what fixed it was that I ran the Celery Flower (it runs on port 5555, run it by "airflow celery flower" command). And from there I could see all the celery workers that were in the pool of workers. There was some there that I didn't expect - apparently I had a temporary/testing setup of Airflow running against the same celery backend and the worker there was also trying to pick up the queued tasks.
   But the task definitions were different between the scheduler and that errant worker, so it didn't run. Of course the errors were also produced not at the main worker and I just didn't see them.
   When I stopped that errant worker then everything resumed working.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861300533


   We were running into a similar issue as we have 100+ dags and around a 1000 tasks.
   
   I figured out there is a bug in the `celery_executor` which I still want to fix myself and contribute.
   
   Summary of that problem:
   At the start of the scheduler, the celery_executor class instance of the scheduler picks up everything from 'dead' schedulers (your previous run). That is (if you run one scheduler) every TaskInstance in the Running, Queued or Scheduled state. Then once it verified that this task is not running(takes 10 minutes), it clears most of the references but forgets a crucial one, making it such that the scheduler can NEVER start this task anymore. You can still start it via the webserver because that has its own celery_executor class instance.
   
   What we noticed:
   - Many tasks were very slowly to be scheduled even though the workers were almost fully idle.
   - The TaskInstances were stuck on Queued or Scheduled.
   - Restarting the scheduler didn't work.
   - Once restarted (with debug logging enabled) you'd get a logging line like this: `[2021-06-14 14:07:31,932] {base_executor.py:152} DEBUG - -62 open slots`
   
   What you can do to verify whether you have the same issue:
   - Stop the scheduler
   - Clear all TaskInstances that are Queued or Scheduled
   - Start the scheduler
   
   Our fix:
   - Increase the airflow.cfg parallelism -> from 32 to 1000. This is what could easily deadlock your scheduler after a restart. Because it uses this variable to see if it can launch any new task. If you had 50 tasks in Scheduled waiting, it will deadlock your entire scheduler.
   - Increase the default pool size (for a speedup) -> from 128 to 1000
   - For any task that the scheduler can't run anymore. Do the procedure mentioned above or kick-start it yourself by clicking the task instance followed by "Ignore all deps", "Ignore Task states", "Ignore Task Deps" and finally "Run".
   
   Hope this helps anyone and saves you a couple days of debugging :)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lintingbin2009 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lintingbin2009 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-839409350


   We also have this problem. Every few days there will be a task that cannot be scheduled in the queue state forever. After restarting the scheduler, the status task will become up_for_retry and continue to run. So our current solution is to restart the scheduler every hour.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-814068888


   @lukas-at-harren -- Can you check the Airflow Webserver -> Admin -> Pools  and then in the row with your pool (`mssql_dwh`) check the Used slots. And click on the number in Used slots, it should take you to the TaskInstance page that should show the currently "running" taskinstances in that Pool.
   
   It is possible that they are not actually running but somehow got in that state in DB. If you see 3 entries over here, please mark those tasks as success or failed, that should clear your pool.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] WattsInABox commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
WattsInABox commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1048780849


   @danmactough ouch! Thanks for responding to a rando, we appreciate it :) 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jpkoponen edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jpkoponen edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029887270


   Thank you @val2k !
   
   It's interesting that increasing `try_number` and modifying the `STATE` did not work for you. I will first try the automatic method without deleting hoping that it will work for me.
   
   I just wish there was a way to test the DAG other than waiting for the issue to arise again. I think we have had it happen only twice in the past two weeks. 😄 
   
   Edit: Actually, we didn't have it happen in prod but in dev there were two instances last night. I tried to use the utility DAG to increase their `try_number ` but it didn't work. 😞 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] danmactough commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
danmactough commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1014119402


   > Hello @danmactough,
   > 
   > Thanks for your work ! Could you provide the whole code for your utility DAG please ?
   > 
   > But if I understand correctly, when you see that a task within a dag run is blocked, you have to manually trigger your utility DAG passing its dag id / execution date ?
   
   Sure @val2k. I updated my previous comment to include the full DAG as well as a description of how to provide the DAG run config.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] haninp commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
haninp commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1060152029


   Hello, found same issue when i used ver 2.2.4 (latest)
   maybe we have some workaround for this things ?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] zachliu commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
zachliu commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-841265164


   > @zachliu What executor are you using?
   
   Celery


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] caleboverman commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
caleboverman commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-875929266


   We've just migrated to `2.1.1` and are continuing to see tasks stuck in `Scheduled` using the `LocalExecutor`.
   
   In our case, Airflow seems to kick off tasks as expected until a DAG reaches it's concurrency limit (which we typically set at the individual DAG level instead of globally). Once a DAG hits the concurrency limit then tasks in other DAGs don't start executing despite Airflow having open executor slots. We're only making use of the `default_pool` and tasks are moving from `Queued` to `Running` there but aren't entering the queue despite being in a `Scheduled` state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821356514


   @SalmonTimo i have access to the workers, as well as the dag logs folder, which is saved in the file share (PVC), can be viewed via azure storage explorer. 
   The environment is set up new, and migrated a few tables from old environment. How would like me to fetch the log? Just the dag log run? 
   
   update: the one i can view via storage explorer - it is empty, since the task was not executed.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614


   I am also noticing this issue with my Airflow v2.1.2 instance, where the task is forever in queued state, and in the scheduler log I see 
   ```
   airflow-scheduler1_1  | [2021-08-04 12:45:23,286] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mongakshay commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
mongakshay commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-892630614


   I am also noticing this issue with my Airflow v2.1.2 instance, where the task is forever in queued state, and in the scheduler log I see 
   ```
   airflow-scheduler1_1  | [2021-08-04 12:45:23,286] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='example_complex', task_id='create_tag_template_result2', execution_date=datetime.datetime(2021, 8, 3, 22, 39, 34, 577031, tzinfo=Timezone('UTC')), try_number=1)
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] renanleme edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
renanleme edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-803572439






-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861300533


   We were running into a similar issue as we have 100+ dags and around a 1000 tasks.
   
   I figured out there is a bug in the `celery_executor` which I still want to fix myself and contribute.
   
   Summary of that problem:
   At the start of the scheduler, the celery_executor class instance of the scheduler picks up everything from 'dead' schedulers (your previous run). That is (if you run one scheduler) every TaskInstance in the Running, Queued or Scheduled state. Then once it verified that this task is not running(takes 10 minutes), it clears most of the references but forgets a crucial one, making it such that the scheduler can NEVER start this task anymore. You can still start it via the webserver because that has its own celery_executor class instance.
   
   What we noticed:
   - Many tasks were very slowly to be scheduled even though the workers were almost fully idle.
   - The TaskInstances were stuck on Queued or Scheduled.
   - Restarting the scheduler didn't work.
   - Once restarted (with debug logging enabled) you'd get a logging line indicating you have negative open slots: `[2021-06-14 14:07:31,932] {base_executor.py:152} DEBUG - -62 open slots`
   
   What you can do to verify whether you have the same issue:
   - Stop the scheduler
   - Clear all TaskInstances that are Queued or Scheduled
   - Start the scheduler
   
   Our fix:
   - Increase the airflow.cfg parallelism -> from 32 to 1000. This is what could easily deadlock your scheduler after a restart. Because it uses this variable to see if it can launch any new task. If you had 50 tasks in Scheduled waiting, it will deadlock your entire scheduler.
   - Increase the default pool size (for a speedup) -> from 128 to 1000
   - For any task that the scheduler can't run anymore. Do the procedure mentioned above or kick-start it yourself by clicking the task instance followed by "Ignore all deps", "Ignore Task states", "Ignore Task Deps" and finally "Run".
   
   Hope this helps anyone and saves you a couple days of debugging :)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] girishrd2004 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
girishrd2004 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-813778875


   I had the task stuck because I changed the DAG id as part of my code change. After reverting the DAG id change, things worked fine. However the old queued tasks had to be manually marked complete.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-810866892


   Back on this. I am currently observing the behaviour again.
   
   I can confirm:
   
   * The solution description of @renanleme does not apply to my case. Definitely not.
   * Restarting the scheduler is a workaround to the problem
   
   The issue persists with `2.0.1` please update the tag accordingly.
   
   The issue is definitely "critical" as it halts THE ENTIRE airflow operation...!


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812743460


   We have changed the default `min_file_process_interval` to `30` from Airflow 2.0.1
   
   https://github.com/apache/airflow/blob/8cc8d11fb87d0ad5b3b80907874f695a77533bfa/UPDATING.md#default-scheduler-min_file_process_interval-is-changed-to-30
   
   although I wouldn't think that might be the cause to task staying in queued state but worth a try


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] renanleme edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
renanleme edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-803572439


   I had the same problem today and I think I found the problem.
   
   I'm testing with:
   Apache Airflow version: `2.0.1`
   Executor: `Celery`
   Running locally
   
   I was testing one dag and after changing a few parameters in one of the tasks in the dag file and cleaning the tasks, the task got stuck on scheduled state.
   **The problem:** The changes that I made broke the task, something like this:
   ```
   airflow-worker_1     | airflow.exceptions.AirflowException: Invalid arguments were passed to GoogleCloudStorageToBigQueryOperator (task_id: load_dag). Invalid arguments were:
   airflow-worker_1     | **kwargs: {'gcp_conn_id': 'bigquery_default'}
   ```
   So, the worker was refusing to execute because I was passing an invalid argument to the task. **_The problem is that the worker doesn't notify (or update the task status to running) the scheduler/web that the file is wrong_** (no alert of a broken dag was being show in the Airflow home page).
   
   After updating the task parameter and cleaning the task, it ran successfully.
   
   _Ps.: Probably is not the same problem that the OP is having but it's related to task stuck on scheduled_


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-875932438


   > We're only making use of the `default_pool` and tasks are moving from `Queued` to `Running` there but aren't entering the queue despite being in a `Scheduled` state.
   
   
   We had a similar issue and increased the `default_pool` 10x (from 128 to 1000). This fixed it completely for us (but we are using Celery Executor though) 
   Did you already attempt something similar? Wondering if this helps in your case as well


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911307179


   > I'm experiencing a new, similar issue on `2.1.3`, where the DAGruns themselves get stuck in a queued state. The scheduler will accumulate queued runs but they never pop off. I'd be happy to share logs if someone can tell me which file they would like to see or what text to grep for.
   > ![image](https://user-images.githubusercontent.com/7014837/131776954-bffb4ac9-e303-462c-8703-2934178f1a6b.png)
   
   Check the dag details page for the running tasks, it's possible there's a running task but due to the number of running as well as queued dagruns, the running is not shown here in the UI. Check the page ('Details') in dag view


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] minnieshi edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
minnieshi edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-821258217


   **My situation**
   - kubenetes
   - airflow 1.10.14
   - celery executor
   - only 1 dag is 'on', the rest 20 dags are 'off'
   - dag is correct as it works in other environment. 
   - pool (default_pool), 32 slots, 0 used slots, queued slots =1)
   - tasks in the dag can be run manually (by clear it), but it does not automatically run the next task.
   - one situation: after restarting the scheduler manually (to restart configuration is set to never, value schedulerNumRuns is set  -1),  it decided to run 3 out of 4 tasks, and the last one just **stuck at the queued state**
   - after that, tried to load the dag with different name and different id, the 1st task of the dag 
    **stuck at the 'scheduled' state** after clear the task.
   - when checking log on scheduler, it has error like this
   `[2021-04-16 13:06:36,392] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 3497') Celery Task ID: ('XXXXXXXX_YYY_test', 'Task_blahblahblah', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)`
   
   - i reported here: https://github.com/helm/charts/issues/19399 but found the issue is already closed.
   
   **I tried to experiment the ...., which did not help as i expected.**
   - Uploaded the dag with new name/id. enabled, cleared the dag (otherwise the 1st task just stuck at the 'queued' state)
   and 1st task is at the 'scheduled' state and stuck there.
   - check scheduler log:
   - `[2021-04-16 15:58:51,991] {celery_executor.py:282} ERROR - Error fetching Celery task state, ignoring it:AirflowTaskTimeout('Timeout, PID: 1851')
   Celery Task ID: ('XXXXXX_min_test_3', 'Load_XXXX_to_YYYY', datetime.datetime(2021, 4, 15, 3, 0, tzinfo=<TimezoneInfo [UTC, GMT, +00:00:00, STD]>), 1)
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/executors/celery_executor.py", line 117, in fetch_celery_task_state
       res = (celery_task[0], celery_task[1].state)`
   
   
   ps. The environment is set up new, and migrated a few tables listed below from old environment.  during debug of this stuck situation, the table 'dag', 'task_*', 'celery_*' had been truncated.
   ```
   - celeray_taskmeta
   - dag
   - dag_run
   - log
   - task_fail
   - task_instance
   - task_reschedule
   - connections
   ```
   
   the dag log itself is empty, since the task was not executed. 
   the worker has no errors. i will attach the log anyhow.
   [log-workers.txt](https://github.com/apache/airflow/files/6327151/log-workers.txt)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Vbubblery commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Vbubblery commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-869488492


   Hello, may I know this will release in Airflow 2.1.1 or Airflow 2.2?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-907143038


   @gabrielsyapse . Thanks for your comment. Maybe you do not understamd how open source and free software works, so let me explain it.
   
   Many people are working - often on their own time and with volunteered effort - so that you can use the software for free. And yeah - this means sometimes that the best way to repay it and contribute back by those users (at least a minimum effort that the users of the free software can do to thank for the effort) is to verify that what we 'think' is the fix is actually one.
   
   Surely, you can stay in 1.10 but be aware that it has reached end of life already in June and you will not get support of those (often unpaid volunteers) if you decide to stay with it.
   
   But maybe this is what you want. I am sure you can find  paid support for 1.10 and expect the company doing the support makes sure that all your problems are fixed and will not ask you to verify it, it will 'just work'.
   
   Good luck with your decision. There are already a few security fixes that are only fixed in  Airflow 2 that will never make it to Airflow 1.10 so you will either need someone to fix them for you or you need luck so that those are not exploited. You can learn more about it at https://youtu.be/86uqFr3LIc8 - discussion about keeping your Airflow secure that we had at the Airflow Summit.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] timgriffiths edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
timgriffiths edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-854356862


   Environment:
   
   Airflow 2.1.0
   Docker : apache/airflow:2.1.0-python3.8
   Executor : Celery
   
   So I have also hit this today, the way i was able to replicate this 
   
   -> have a py file that generates move than 1 dag
   -> then have a user break 1 of the dag from this resultant chain
   -> all the other still valid dags will be triggered in the scheduler but fail in the Worker but it looks like the exception is not passed back to the Scheduler so it permanently stays in a "queued state"
   
   > [2021-06-04 01:11:22,927: INFO/ForkPoolWorker-40] Executing command in Celery: ['airflow', 'tasks', 'run', 'myworkingjob', 'taskname-abc', '2021-06-04T01:11:21.775097+00:00', '--local', '--pool', 'pool', '--subdir', '/opt/airflow/dags/mm-dynamic-dags.py']
   [2021-06-04 01:11:22,940: INFO/ForkPoolWorker-34] Filling up the DagBag from /opt/airflow/dags/mm-dynamic-dags.py
   [2021-06-04 01:11:27,993: ERROR/ForkPoolWorker-39] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dagbag.py", line 317, in _load_modules_from_file
       loader.exec_module(new_module)
     File "<frozen importlib._bootstrap_external>", line 848, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 187, in <module>
       create_dag("{}/{}".format(global_dags_base_url, dag_file))
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 123, in create_dag
       dag = DAG(
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dag.py", line 277, in __init__
       validate_key(dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/helpers.py", line 44, in validate_key
       raise AirflowException(
   airflow.exceptions.AirflowException: The key (test-test+abc) has to be made of alphanumeric characters, dashes, dots and underscores exclusively
   [2021-06-04 01:11:27,996: ERROR/ForkPoolWorker-39] Failed to execute task dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse..
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 116, in _execute_in_fork
       args.func(args)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 91, in wrapper
       return f(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/commands/task_command.py", line 219, in task_run
       dag = get_dag(args.subdir, args.dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 191, in get_dag
       raise AirflowException(
   airflow.exceptions.AirflowException: dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse.
   [2021-06-04 01:11:28,017: ERROR/ForkPoolWorker-39] Task airflow.executors.celery_executor.execute_command[66453177-4042-4c6a-882b-bd0fcdbda0d8] raised unexpected: AirflowException('Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm')
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 412, in trace_task
       R = retval = fun(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 704, in __protected_call__
       return self.run(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 87, in execute_command
       _execute_in_fork(command_to_exec)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 98, in _execute_in_fork
       raise AirflowException('Celery command failed on host: ' + get_hostname())
   airflow.exceptions.AirflowException: Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm
   [2021-06-04 01:11:28,091: ERROR/ForkPoolWorker-37] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] MatrixManAtYrService commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
MatrixManAtYrService commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849031715


   While trying to recreate this, I wrote a [stress test](https://github.com/MatrixManAtYrService/airflow-git-sync/blob/master/scheduler_stress.py) which I ran overnight on my local microk8s cluster (airflow v2.0.1, release:2.0.1+beb8af5ac6c438c29e2c186145115fb1334a3735 configured like [this](https://github.com/MatrixManAtYrService/airflow-git-sync/blob/master/zsh.stdin)).
   
   I was hoping that it would get fully stuck by the time I woke.  Instead there were only two stuck tasks: 
    
   ![stucktasks](https://user-images.githubusercontent.com/5834582/119712787-34db0f80-be1e-11eb-9a41-875dd88c0566.gif)
   
   Deleting the scheduler pod and letting kubernetes recreate it did cause the stuck tasks to complete.  My current guess is that it won't get stuck in a restarting-scheduler-doesnt-help kind of way until the number of stuck-queued tasks matches `dag_concurrency`.  I'll update here if I'm able to confirm that guess.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] shivanshs9 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
shivanshs9 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861406671


   > `kill -USR2 <pid of scheduler>` -- how you get the pid depends upon how and where you are running it :)
   > 
   > Likely exec in to the container, run `ps auxww` and find the oldest scheduler processs (you'll see some sub processes, possibly named helpfully).
   
   @ashb @Jorricks  I'm facing a similar problem as OP on Airflow v2.0.1 with Celery Executor. If it helps, you can find the debug information from scheduler logs:
   
   ```
   [2021-06-15 07:13:48,623] {{dag_processing.py:1071}} INFO - Finding 'running' jobs without a recent heartbeat
   [2021-06-15 07:13:48,624] {{dag_processing.py:1075}} INFO - Failing jobs without heartbeat after 2021-06-15 07:08:48.624534+00:00
   [2021-06-15 07:13:49,539] {{scheduler_job.py:757}} INFO - --------------------------------------------------------------------------------
   SIGUSR2 received, printing debug
   --------------------------------------------------------------------------------
   [2021-06-15 07:13:49,539] {{base_executor.py:302}} INFO - executor.queued (0)
   	
   [2021-06-15 07:13:49,539] {{base_executor.py:307}} INFO - executor.running (40)
   	TaskInstanceKey(dag_id='ergo_job_collector', task_id='process_job_result', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 10, 753973, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 13, 22, 40, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='ergo_task_queuer', task_id='push_tasks', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 1, 316420, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=32)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_1', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 3, 23, 693482, tzinfo=Timezone('UTC')), try_number=41)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 13, 9, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 13, 9, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=39)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 11, 6, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 11, 6, 30, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 13, 22, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 13, 9, 16, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 11, 6, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 5, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=42)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_2', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797327, tzinfo=Timezone('UTC')), try_number=40)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 32, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 15, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 10, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 3, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 0, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 12, 21, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 30, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=41)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=30)
   	TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 11, 6, 0, tzinfo=Timezone('UTC')), try_number=1)
   	TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=39)
   [2021-06-15 07:13:49,540] {{base_executor.py:308}} INFO - executor.event_buffer (0)
   	
   [2021-06-15 07:13:49,540] {{celery_executor.py:387}} INFO - executor.tasks (27)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=30), <AsyncResult: 6960c6d0-7e21-4ef9-8f04-d95efdd9d706>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_2', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797327, tzinfo=Timezone('UTC')), try_number=40), <AsyncResult: 4a815d59-d824-4373-8f96-34272174cfc0>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=39), <AsyncResult: 5296e550-5efd-474a-9e34-897330504886>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=39), <AsyncResult: cad2a01b-cde3-473e-85bc-9f96eae7da7c>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: cae54db9-56a5-4a27-8226-1dc5867265f7>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 438e5379-ef8d-4bfe-8e55-cae461a3f62f>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 9a953b60-b222-4d4c-aff4-5b831caef71f>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 840e404c-1cee-4c18-8123-b354a63e8d80>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 4410416b-7132-4713-a78d-caa0ce3cacb0>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 91d5060a-ff5c-4997-832a-757c25069ac7>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 10, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ade2d701-f632-41d9-a901-40531619dfba>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: febb3b1e-0f63-41d3-83dc-58f5b11a39ca>)
   	(TaskInstanceKey(dag_id='ergo_job_collector', task_id='process_job_result', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 10, 753973, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 390f1451-7f44-4415-8c57-e2542ed6b6d2>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 30, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 7926f732-e671-4f64-afaf-7c352a85a929>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 32, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ba1edc8e-66b4-4a20-9cc6-5b0c5758d8c4>)
   	(TaskInstanceKey(dag_id='ergo_task_queuer', task_id='push_tasks', execution_date=datetime.datetime(2021, 6, 15, 4, 39, 1, 316420, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 0ef80d4f-1c4c-4e62-b5ed-cebd0b3ba075>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: fea3edfa-477d-4fb0-a580-03ea91faaef1>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_pusher', execution_date=datetime.datetime(2021, 6, 15, 4, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 4749177d-df70-49cf-a2fb-9b0a230d5ff4>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 0, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: bf04f105-efc7-41be-864c-8161c4973f6b>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_task_queued', execution_date=datetime.datetime(2021, 6, 15, 3, 0, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 7c8c6550-cdcc-4405-816b-15da1dbeb79e>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 15, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ee255fe6-21d2-4d3f-8860-83e55265fea9>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 14, 22, 20, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: 46b77d60-43b5-41e6-99eb-e407e0d74791>)
   	(TaskInstanceKey(dag_id='[REDACTED]', task_id='chronos_job_sensor', execution_date=datetime.datetime(2021, 6, 15, 4, 5, tzinfo=Timezone('UTC')), try_number=1), <AsyncResult: ed5115f9-dec6-49da-a5bf-15860a907ab4>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_4', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 10, 54, 17, 679502, tzinfo=Timezone('UTC')), try_number=32), <AsyncResult: 49ee2d3f-eef1-4f7a-a5da-923e8cb8b914>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_1', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 3, 23, 693482, tzinfo=Timezone('UTC')), try_number=41), <AsyncResult: 910a1d67-658d-4fe9-954a-2d093b64c95d>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_0', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 7, 0, 2, 374657, tzinfo=Timezone('UTC')), try_number=42), <AsyncResult: 8061f49f-8050-4b73-84fd-4a0222369040>)
   	(TaskInstanceKey(dag_id='smart_sensor_group_shard_3', task_id='smart_sensor_task', execution_date=datetime.datetime(2021, 6, 14, 6, 58, 43, 797668, tzinfo=Timezone('UTC')), try_number=41), <AsyncResult: 262cf0ac-1132-4176-aba0-77ec4a2bddb7>)
   [2021-06-15 07:13:49,540] {{celery_executor.py:390}} INFO - executor.adopted_task_timeouts (0)
   	
   [2021-06-15 07:13:49,540] {{scheduler_job.py:760}} INFO - --------------------------------------------------------------------------------
   [2021-06-15 07:13:51,028] {{dag_processing.py:838}} INFO -
   ================================================================================
   DAG File Processing Stats
   
   File Path                                                                                           PID  Runtime      # DAGs    # Errors  Last Runtime    Last Run
   ------------------------------------------------------------------------------------------------  -----  ---------  --------  ----------  --------------  -------------------
   /home/airflow/.local/lib/python3.8/site-packages/airflow/smart_sensor_dags/__init__.py                                     0           0  0.09s           2021-06-15T07:13:50
   /opt/airflow/dags/dag_ergo.py                                                                                              2           0  0.47s           2021-06-15T07:13:51
   /home/airflow/.local/lib/python3.8/site-packages/airflow/smart_sensor_dags/smart_sensor_group.py  16087  0.00s             5           0  0.15s           2021-06-15T07:13:50
   /opt/airflow/dags/dag_dagen.py                                                                    16081  1.48s            90           0  4.10s           2021-06-15T07:13:49
   ================================================================================
   ```
   
   The only fix that works is by manually terminating and restarting the scheduler pod.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] suxin1995 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
suxin1995 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817807364


   i have some problem.
   task always keep schedule status and canot  continue work.
   for this,i restart  `airflow schedule ` process by use crontab script.
   It's not good idea.but useful.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ohdearaugustin commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ohdearaugustin commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849435787


   We are facing the same issue, with the kubernetes executer. K8s: 1.19.3. We use the helmchart from https://airflow-helm.github.io/charts (8.0.6), which refers to Airflow Version 2.0.1.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] grepthat commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
grepthat commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-848898802


   I noticed, that **statsd** reports _0 queued task_, whereas the WebUI has **>10 queued tasks**. I'm on Airflow 2.1.0, using k8s Executor. The queued tasks are also not registered in k8s, as _Pending_ pods or so.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849533075


   If anyone has this problem, could you please send a SIGUSR2 to your scheduler process? That will cause it to print out some internal state that might help us track down this problem.
   
   Also if anyone is able to _reliablely_ produce and able to share your dags that would help massively.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812742063


   I ran into this issue due to the scheduler over-utilizing CPU because our `min_file_process_interval` was set to 0 (the default prior to 2.0), which in airflow 2.0 causes 100% CPU utilization by constantly pinging for DAG files. Setting this parameter to 60 fixed the issue.
   The stack I observed this on:
   host: AWS ECS Cluster
   executor: CeleryExecutor
   queue: AWS SQS Queue
   
   The behavior I observed was that the scheduler would mark tasks are "queued", but never actually send them to the queue (I think the scheduler does actual queueing via the executor). My manual workaround until correcting the `min_file_process_interval` param was to stop the scheduler, clear queued tasks, and then start a new scheduler. The new scheduler would temporarily properly send tasks to the queue, before degenerating to marking tasks as queued without sending to the queue.
   
   I suspect the OP may have this same issue because they mentioned having 100% CPU utilization on their scheduler.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853117451


   More data, if it can benefit:
   Airflow version: `2.1.0`
   Kubernetes version: `1.18.9`
   
   I'm using Airflow `2.1.0` (upgraded from `1.10.15` in our development environment) and trying to use the `KubernetesExecutor`. I am running _extremely_ basic DAGs that exist solely to test. In `airflow.cfg` I set `delete_worker_pods = False` so that I could try and examine what's going on. As far as I can tell, my worker pods are being created, immediately considering themselves successful (not sure if they actually receive a task to complete), and terminating. I don't think the pods are "successful but not writing the success state to the DB" because one of my tasks is a simple `CREATE TABLE` statement on my data warehouse, and the table does not appear.
   
   
   Here are the results of various logs:
   
   If I run `kubectl get pods`, I see my `airflow-webserver` and `airflow-scheduler` pods, as well as some "Completed" worker pods.
   
   ```
   NAME                                                           READY   STATUS      RESTARTS   AGE
   airflow-scheduler                                              1/1     Running     0          45m
   airflow-webserver-1                                            2/2     Running     0          56m
   airflow-webserver-2                                            2/2     Running     0          56m
   airflow-webserver-3                                            2/2     Running     0          45m
   airflow-webserver-4                                            2/2     Running     0          45m
   airflow-webserver-5                                            2/2     Running     0          45m
   <GENERATED_WORKER_POD_1>                                       0/1     Completed   0          15m
   <GENERATED_WORKER_POD_2>                                       0/1     Completed   0          56m
   <GENERATED_WORKER_POD_3>                                       0/1     Completed   0          45m
   ```
   
   `kubectl logs <my_worker_pod>` - I can tell that the pod was briefly alive but that did nothing, because the only output is a line that always appears as a side effect of the Docker image config. In contrast, if I get the logs from one of my functional `1.10.15` worker pods, I can see the task start:
   ```
   [2021-06-02 13:33:01,358] {__init__.py:50} INFO - Using executor LocalExecutor
   [2021-06-02 13:33:01,358] {dagbag.py:417} INFO - Filling up the DagBag from /usr/local/airflow/dags/my_dag.py
   ...
   etc
   ```
   
   `kubectl describe pod <my_worker_pod>` - the event log is quite tame:
   ```
   Events:
     Type    Reason     Age    From               Message
     ----    ------     ----   ----               -------
     Normal  Scheduled  3m21s  default-scheduler  Successfully assigned <GENERATED_POD_NAME> to <EC2_INSTANCE>
     Normal  Pulling    3m19s  kubelet            Pulling image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Pulled     3m19s  kubelet            Successfully pulled image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Created    3m19s  kubelet            Created container base
     Normal  Started    3m19s  kubelet            Started container base
   ```
   
   `kubectl logs airflow-scheduler` - I've trimmed the logs significantly, but here are the statements mentioning a stuck task
   ```
   [2021-06-02 14:49:42,742] {kubernetes_executor.py:369} INFO - Attempting to finish pod; pod_id: <GENERATED_POD_NAME>; state: None; annotations: {'dag_id': '<TEST_DAG>', 'task_id': '<TASK_ID>', 'execution_date': '2021-05-30T00:00:00+00:00', 'try_number': '1'}
   [2021-06-02 14:49:42,748] {kubernetes_executor.py:546} INFO - Changing state of (TaskInstanceKey(dag_id='<TEST_DAG>', task_id='<TASK_ID>', execution_date=datetime.datetime(2021, 5, 30, 0, 0, tzinfo=tzlocal()), try_number=1), None, '<GENERATED_POD_NAME>', 'airflow', '1000200742') to None
   [2021-06-02 14:49:42,751] {scheduler_job.py:1212} INFO - Executor reports execution of <TASK_ID> execution_date=2021-05-30 00:00:00+00:00 exited with status None for try_number 1
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-811901182


   Unassigning myself as I can't reproduce the bug again.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nikitarulz commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
nikitarulz commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-797346044


   This is happening with Celery executor as well.
   I'm using Airflow 2.0.0 with Celery executor and mysql, facing similar issue.
   Sorry for the basic question but I'm unable to figure-out the manual way to find all tasks in "queued" state and clearing them.
   Can somebody help here.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812743460


   We have changed the default `min_file_process_interval` to `30` from Airflow 2.0.1
   
   https://github.com/apache/airflow/blob/8cc8d11fb87d0ad5b3b80907874f695a77533bfa/UPDATING.md#default-scheduler-min_file_process_interval-is-changed-to-30
   
   For existing deployment, a user will have to change this manually in their `airflow.cfg`
   
   although I wouldn't think that might be the cause to task staying in the queued state but worth a try


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861304352


   > `kill -USR2 <pid of scheduler>` -- how you get the pid depends upon how and where you are running it :)
   > 
   > Likely exec in to the container, run `ps auxww` and find the oldest scheduler processs (you'll see some sub processes, possibly named helpfully).
   
   Thanks a lot @ashb. This command allowed our team to investigate the issue better and finally helped me figure the problem out.
   Expect a PR from me this week :)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jonathonbattista commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jonathonbattista commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-797514714


   This is probably not helpful because you are on 2.x but our solution was to set [AIRFLOW__SCHEDULER__RUN_DURATION](https://airflow.apache.org/docs/apache-airflow/1.10.14/configurations-ref.html#run-duration) which will restart the scheduler every x hours.
   
   You could probably achieve something similar tho.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-819411427


   @ephraimbuddy I found the root cause for the problem, and a way to reproduce it. Keep in mind my stack (Airflow + KubernetesExecutor), as this issue has been watered down by many different stacks and situations, ending with the same symptoms.
   
   Steps to reproduce:
   
   * Create a DAG and schedule some work for it.
   * While work is scheduled, remove the DAG.
   * Restart the scheduler.
   * Now the DAG does no longer exist, but it still exists in the database. And its scheduled tasks also still exist.
   * The scheduler dutifully schedules work for the non-existent DAG
   * The KubernetesExecutor spawns a new worker pod
   * The worker pod is awfully surprised that there is no DAG for the work he was tasked with
   * The worker pod commits suicide without telling anybody
   * The scheduler faithfully keeps the task in "queued" state, although the worker is no more
   
   Solution:
   
   * The scheduler should not schedule work for tasks that are no longer in the DagBag
   * The worker must fail properly (with its task ending in a "failed" state) when he cannot find the DAG + task he was tasked with


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-822260516


   @ephraimbuddy no, I have to revisit this issue again.
   
   I can now confirm, that tasks are stuck in "queued" even with the afroimentioned issue solved.
   I no longer have any missing DAGs, but the issue persists.
   
   I now have a complete Airflow installation with 100+ DAGs that gets stuck every day or so.
   This is a critical issue. Please adjust your priority accordingly.
   Many other issues pop up with the same symptoms!
   
   * https://github.com/apache/airflow/issues/15192
   * https://github.com/apache/airflow/issues/15305
   * https://github.com/apache/airflow/issues/15369


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861300533


   We were running into a similar issue as we have 100+ dags and around a 1000 tasks.
   
   I figured out there is a bug in the `celery_executor` which I still want to fix myself and contribute.
   
   Summary of that problem:
   At the start of the scheduler, the celery_executor class instance of the scheduler picks up everything from 'dead' schedulers (your previous run). That is (if you run one scheduler) every TaskInstance in the Running, Queued or Scheduled state. Then once it verified that this task is not running(takes 10 minutes), it clears most of the references but forgets a crucial one, making it such that the scheduler can NEVER start this task anymore. You can still start it via the webserver because that has its own celery_executor class instance.
   
   What we noticed:
   - Many tasks were very slowly to be scheduled even though the workers were almost fully idle.
   - The TaskInstances were stuck on Queued or Scheduled.
   - Restarting the scheduler didn't work.
   
   What you can do to verify whether you have the same issue:
   - Stop the scheduler
   - Clear all TaskInstances that are Queued or Scheduled
   - Start the scheduler
   
   Our fix:
   - Increase the airflow.cfg parallelism -> from 32 to 1000. This is what could easily deadlock your scheduler after a restart. Because it uses this variable to see if it can launch any new task. If you had 50 tasks in Scheduled waiting, it will deadlock your entire scheduler.
   - Increase the default pool size (for a speedup) -> from 128 to 1000
   - For any task that the scheduler can't run anymore. Do the procedure mentioned above or kick-start it yourself by clicking the task instance followed by "Ignore all deps", "Ignore Task states", "Ignore Task Deps" and finally "Run".
   
   Hope this helps anyone and saves you a couple days of debugging :)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bensonnd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
bensonnd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-823322869


   Following on what @pelaprat mentioned, we are not running with either the CeleryExecutor or KubernetesExecutor, but the LocalExecutor in a Docker container. We get tasks stuck in scheduled or queued and the dag is marked as running, but is not. It seems like the scheduler falls asleep or misses queued tasks.
   
   Either clearing the queued tasks or restarting the scheduler with `airflow scheduler` inside the container gets it moving again. 
   
   We've observed two different sets of logs over and over again when it does get in this stuck state. One detecting zombie jobs, and the other just checking for the regular heartbeat.
   
   ```
        61 File Path                                       PID  Runtime      # DAGs    # Errors  Last Runtime    Last Run
        62 -------------------------------------------  ------  ---------  --------  ----------  --------------  -------------------
        63 /opt/ingest/batch_ingest/dags/ingest_dag.py  120318  4.02s             1           0  5.43s           2021-04-08T16:37:43
        64 ================================================================================
        65 [2021-04-08 16:37:58,444] {dag_processing.py:1071} INFO - Finding 'running' jobs without a recent heartbeat
        66 [2021-04-08 16:37:58,445] {dag_processing.py:1075} INFO - Failing jobs without heartbeat after 2021-04-08 16:32:58.445055+00:00
        67 [2021-04-08 16:37:58,455] {dag_processing.py:1098} INFO - Detected zombie job: {'full_filepath': '/opt/ingest/batch_ingest/dags/ingest_dag.py', 'msg': 'Detected as zombie', 'simple_task_instance': <airflow.models.taskinstance.Si>
        68 [2021-04-08 16:38:08,595] {dag_processing.py:1071} INFO - Finding 'running' jobs without a recent heartbeat
        69 [2021-04-08 16:38:08,596] {dag_processing.py:1075} INFO - Failing jobs without heartbeat after 2021-04-08 16:33:08.596291+00:00
        70 [2021-04-08 16:38:08,607] {dag_processing.py:1098} INFO - Detected zombie job: {'full_filepath': '/opt/ingest/batch_ingest/dags/ingest_dag.py', 'msg': 'Detected as zombie', 'simple_task_instance': <airflow.models.taskinstance.Si>
        71 [2021-04-08 16:38:18,650] {dag_processing.py:1071} INFO - Finding 'running' jobs without a recent heartbeat
        72 [2021-04-08 16:38:18,651] {dag_processing.py:1075} INFO - Failing jobs without heartbeat after 2021-04-08 16:33:18.651308+00:00
        73 [2021-04-08 16:38:18,661] {dag_processing.py:1098} INFO - Detected zombie job: {'full_filepath': '/opt/ingest/batch_ingest/dags/ingest_dag.py', 'msg': 'Detected as zombie', 'simple_task_instance': <airflow.models.taskinstance.Si>
        74 [2021-04-08 16:38:22,690] {dag_processing.py:838} INFO - 
        75 ================================================================================
        76 DAG File Processing Stats
   ```
   
   or
   
   ```
   File Path                                    PID    Runtime      # DAGs    # Errors  Last Runtime    Last Run
   -------------------------------------------  -----  ---------  --------  ----------  --------------  -------------------
   /opt/ingest/batch_ingest/dags/ingest_dag.py                           1           0  1.52s           2021-04-08T18:29:22
   ================================================================================
   [2021-04-08 18:29:33,015] {dag_processing.py:1071} INFO - Finding 'running' jobs without a recent heartbeat
   [2021-04-08 18:29:33,016] {dag_processing.py:1075} INFO - Failing jobs without heartbeat after 2021-04-08 18:24:33.016077+00:00
   [2021-04-08 18:29:43,036] {dag_processing.py:1071} INFO - Finding 'running' jobs without a recent heartbeat
   [2021-04-08 18:29:43,037] {dag_processing.py:1075} INFO - Failing jobs without heartbeat after 2021-04-08 18:24:43.037136+00:00
   [2021-04-08 18:29:53,072] {dag_processing.py:1071} INFO - Finding 'running' jobs without a recent heartbeat
   [2021-04-08 18:29:53,072] {dag_processing.py:1075} INFO - Failing jobs without heartbeat after 2021-04-08 18:24:53.072257+00:00
   [2021-04-08 18:29:53,080] {dag_processing.py:838} INFO - 
   ================================================================================
   DAG File Processing Stats
   ```
   
   We are in the process of pushing 2.0.2 as @kaxil noted to see if that is the issue. 
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] tienhung2812 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
tienhung2812 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-929988205


   > Hi, just got this issue again on `2.1.3` but the behavior quite different, all of schedulers stuck into a DAG check loop, but restart the scheduler will fix it, all other DAG as stuck in `scheduled`
   > 
   > ```
   >  [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,511] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,512] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.aa.aa_20.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,512] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,513] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.aa.ff_46.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,513] {scheduler_job.py:410} INFO - DAG abc_dag has 16/16 running and queued tasks`
   > airflow_scheduler.2.bj0t349vgttt@ip-172-30-2-178    | [2021-09-17 14:29:07,514] {scheduler_job.py:417} INFO - Not executing <TaskInstance: abc_dag.ff.gg_121.collector_v2 2021-09-17 11:30:00+00:00 [scheduled]> since the number of tasks running or queued from DAG abc_dag is >= to the DAG's task concurrency limit of 16
   > ```
   > 
   > The current system have about >= 150 active dags but we have a`abc_dag` have more than 500 tasks inside I have already increase the scheduler config to handling more task
   > 
   > * max_tis_per_query: 512
   > * max_dagruns_to_create_per_loop: 50
   > * max_dagruns_per_loop_to_schedule: 70
   >   I have also add a scheduler healthcheck but it can not detect this issue
   > 
   > Environment:
   > 
   > * Celery Executor
   > * MySQL 8.0.23-14
   > * Airflow: 2.1.3
   
   I have updated to `2.1.4` and it works properly, this might as same as https://github.com/apache/airflow/issues/18131


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861300533






-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-861521020


   @shivanshs9 did you try what I mentioned? 
   Stopping the scheduler, then clearing the Task Instances and starting the scheduler again?
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ddcatgg edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ddcatgg edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-983261840


   I ran into the same issue, the scheduler's log:
   ```
   [2021-12-01 11:45:11,850] {scheduler_job.py:1206} INFO - Executor reports execution of jira_pull_5_min.jira_pull execution_date=2021-12-01 03:40:00+00:00 exited with status success for try_number 1
   [2021-12-01 11:46:26,870] {scheduler_job.py:941} INFO - 1 tasks up for execution:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,871] {scheduler_job.py:975} INFO - Figuring out tasks to run in Pool(name=data_etl_daily_jobs_pool) with 10 open slots and 1 task instances ready to be queued
   [2021-12-01 11:46:26,871] {scheduler_job.py:1002} INFO - DAG data_etl_daily_jobs has 0/16 running and queued tasks
   [2021-12-01 11:46:26,871] {scheduler_job.py:1063} INFO - Setting the following tasks to queued state:
           <TaskInstance: data_etl_daily_jobs.dwd.dwd_ti_lgc_project 2021-11-29 21:06:00+00:00 [scheduled]>
   [2021-12-01 11:46:26,873] {scheduler_job.py:1105} INFO - Sending TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 2 and queue default
   [2021-12-01 11:46:26,873] {base_executor.py:85} ERROR - could not queue task TaskInstanceKey(dag_id='data_etl_daily_jobs', task_id='dwd.dwd_ti_lgc_project', execution_date=datetime.datetime(2021, 11, 29, 21, 6, tzinfo=Timezone('UTC')), try_number=1)
   ```
   Stucked task: dwd.dwd_ti_lgc_project
   The restart of the scheduler, webserver and executor does not do anything.
   
   Version: v2.0.1
   Git Version:.release:2.0.1+beb8af5ac6c438c29e2c186145115fb1334a3735


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FurcyPin commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
FurcyPin commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-948858685


   Hello, I think I ran into the same issue.
   
   ![2021-10-21-19:05:47](https://user-images.githubusercontent.com/6087587/138324724-d56b19d0-8dda-40e8-a62b-4b95f6155ba5.png)
   
   Here's all that the relevant info I could find, hoping this will help to solve it.
   
   * Stack: Cloud Composer 2 
   * Image version: composer-2.0.0-preview.3-airflow-2.1.2
   * Executor: I'm not sure what Cloud Composer uses, but the airflow.cfg in the bucket says "CeleryExecutor"
   
   - I am 100% sure that my DAG code does not contain any error (other similar tasks work fine).
   - I do not use any pool.
   - I do not have a trigger date in the future.
   
   I tried clearing the queued tasks, they immediately appeared as queued again, and the scheduler logged this message (and nothing else useful):
   ```
   could not queue task TaskInstanceKey(dag_id='my_dag', task_id='my_task', execution_date=datetime.datetime(2021, 10, 20, 0, 0, tzinfo=Timezone('UTC')), try_number=2)
   ```
   
   I tried restarting the scheduler by destroying the pod, it did not change anything.
   I tried destroying the Redis pod, but I did not have the necessary permission.
   
   I was running a large number of dags at the same time (more than 20), so it might be linked to `max_dagruns_per_loop_to_schedule`, so I increased it to a number larger than my number of dags,
   but the 3 tasks are still stuck in the queued state, even when I clear them.
   
   I'm running out of ideas... luckily it's only a POC, so if someone has some suggestions to what I could try next, I would be happy to try them.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hyungryuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hyungryuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-945379278


   i got same issue here. in my case, after i resizing airflow worker pod resouece size. (smaller than before.) it happend!.
   i'm not sure but it might related with your worker pod resource size.. i guess..


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] danmactough edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
danmactough edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1011598836


   Airflow 2.0.2+e494306fb01f3a026e7e2832ca94902e96b526fa (MWAA on AWS)
   
   This happens to us a LOT: a DAG will be running, task instances will be marked as "queued", but nothing gets moved to "running".
   
   When this happened today (the first time today), I was able to track down the following error in the scheduler logs:
   
   ![2022-01-12 at 7 16 PM](https://user-images.githubusercontent.com/357481/149243393-0f0b5b91-d1f7-4a51-8a43-3eab644a49e7.png)
   
   At some point after the scheduler had that exception, I tried to clear the state of the queued task instances to get them to run. That resulting in the following logs:
   
   ![2022-01-12 at 7 18 PM](https://user-images.githubusercontent.com/357481/149243535-3ebfd0b1-31af-43aa-99e2-7ee5aa1dbaff.png)
   
   This corresponds to this [section of code](https://github.com/apache/airflow/blob/2.0.2/airflow/executors/base_executor.py#L73-L85):
   
   ![2022-01-12 at 10 38 AM](https://user-images.githubusercontent.com/357481/149171972-e9824366-6e85-4c2e-a00c-5ee66d466de8.png)
   
   My conclusion is that when the scheduler experienced that error, it entered a pathological state: it was running but had bad state in memory. Specifically, the queued task instances were in the `queued_tasks` or `running` in-memory cache, and thus any attempts to re-queue those tasks would fail as long as that scheduler process was running because the tasks would appear to already have been queued and/or running.
   
   Both caches use the [`TaskInstanceKey`](https://github.com/apache/airflow/blob/10023fdd65fa78033e7125d3d8103b63c127056e/airflow/models/taskinstance.py#L224-L230), which is made up of `dag_id` (which we can't change), `task_id` (which we can't change), `execution_date` (nope, can't change), and `try_number` (🎉 we can change this!!).
   
   So to work around this, I created a utility DAG that will find all task instances in a "queued" or "None" state and increment the `try_number` field.
   
   The DAG runs as a single `PythonOperator`:
   
   ```python
   import os
   from datetime import datetime, timedelta
   from pprint import pprint
   
   from airflow import DAG
   from airflow.models.dagrun import DagRun
   from airflow.models.taskinstance import TaskInstance
   from airflow.operators.python import PythonOperator
   from airflow.utils import timezone
   from airflow.utils.db import provide_session
   from airflow.utils.state import State
   from dateutil.parser import parse
   from sqlalchemy.sql.expression import or_
   
   DAG_NAME = os.path.splitext(os.path.basename(__file__))[0]
   DEFAULT_ARGS = {
       "owner": "airflow",
       "depends_on_past": False,
       "email_on_failure": False,
       "email_on_retry": False,
       "execution_timeout": timedelta(minutes=10),
       "retries": 0,
   }
   
   
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       dag_id = dag_run.conf.get("dag_id")
       if not dag_id:
           raise AssertionError("dag_id was not provided")
       execution_date = dag_run.conf.get("execution_date")
       if not execution_date:
           raise AssertionError("execution_date was not provided")
       execution_date = parse(execution_date)
   
       filter = [
           or_(TaskInstance.state == State.QUEUED, TaskInstance.state == State.NONE),
           TaskInstance.dag_id == dag_id,
           TaskInstance.execution_date == execution_date,
       ]
       print(
           (
               f"DAG id: {dag_id}, Execution Date: {execution_date}, State: "
               f"""{dag_run.conf.get("state", f"{State.QUEUED} or {State.NONE}")}, """
               f"Filter: {[str(f) for f in filter]}"
           )
       )
   
       tis = session.query(TaskInstance).filter(*filter).all()
       dr = (
           session.query(DagRun)
           .filter(DagRun.dag_id == dag_id, DagRun.execution_date == execution_date)
           .first()
       )
       dagrun = (
           dict(
               id=dr.id,
               dag_id=dr.dag_id,
               execution_date=dr.execution_date,
               start_date=dr.start_date,
               end_date=dr.end_date,
               _state=dr._state,
               run_id=dr.run_id,
               creating_job_id=dr.creating_job_id,
               external_trigger=dr.external_trigger,
               run_type=dr.run_type,
               conf=dr.conf,
               last_scheduling_decision=dr.last_scheduling_decision,
               dag_hash=dr.dag_hash,
           )
           if dr
           else {}
       )
   
       print(f"Updating {len(tis)} task instances")
       print("Here are the task instances we're going to update")
       # Print no more than 100 tis so we don't lock up the session too long
       for ti in tis[:100]:
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
       if len(tis) > 100:
           print("Output truncated after 100 task instances")
   
       for ti in tis:
           ti.try_number = ti.next_try_number
           ti.state = State.NONE
           session.merge(ti)
   
       if dag_run.conf.get("activate_dag_runs", True):
           dr.state = State.RUNNING
           dr.start_date = timezone.utcnow()
   
       print("Done")
   
   
   with DAG(
       DAG_NAME,
       description="Utility DAG to fix TaskInstances stuck in queued or None state",
       default_args=DEFAULT_ARGS,
       schedule_interval=None,
       start_date=datetime(year=2021, month=8, day=1),
       max_active_runs=1,
       catchup=False,
       default_view="graph",
       is_paused_upon_creation=False,
   ) as dag:
       PythonOperator(task_id="unstick_dag", python_callable=unstick_dag_callable)
   ```
   
   To use the DAG, trigger a DAG run with a `dag_id` and `execution_date` like:
   
   ```json
   {
       "dag_id": "my_stuck_dag",
       "execution_date": "2022-01-01T00:00:00Z"
   }
   ```
   
   
   Moments after I shipped this DAG, another DAG got stuck, and I had a chance to see if this utility DAG worked -- it did! 😅 
   
   -----
   
   Couple of thoughts:
   
   - I don't think my error is exactly the same as OP, as some very key conditions are not applicable to my case, but this issue appears to have many different and probably not at all related bugs that kind of manifest as "stuck DAGs" and this issue has pretty good Google juice -- I just hope my explanation and/or work-around help someone else.
   - The MWAA product from AWS is using an older version of Airflow, so the combination of factors that leads to this pathological state may no longer be possible in the current version of Airflow.
   - MWAA uses the CeleryExecutor, which I suspect is where the pathological state is coming from, not BaseExecutor directly.
   - All that being said, I'm surprised to see this critical state being kept in memory (`queued_tasks` and `running`), but I don't have a complete mental model of how the executor and the scheduler are distinct or not. My understanding is that this is scheduler code, but with the scheduler being high-availability (we're running 3 schedulers), in-memory state seems like something we should be using very judiciously and be flushing and rehydrating from the database regularly.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1013862830


   Hello @danmactough,
   
   Thanks for your work ! Could you provide the whole code for your utility DAG please ? 
   
   But if I understand correctly, when you see that a task within a dag run is blocked, you have to manually trigger your utility DAG passing its dag id / execution date ?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1030913188


   > @jpkoponen At least in v2.0.2 (which is the only v2.x version available on AWS MWAA), there's no reason to call `session.commit()` when using the `@provide_session` decorator. It [creates the session](https://github.com/apache/airflow/blob/2.0.2/airflow/utils/session.py#L69), and [calls `session.commit()` for you](https://github.com/apache/airflow/blob/2.0.2/airflow/utils/session.py#L32).
   
   Thre is also 2.2.2 as of recently. Can you please upgrade and check it there @danmactough ?
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] DVerzal commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
DVerzal commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1054922919


   Thanks for pointing me in the right direction, @potiuk. We're planning to continue with our investigation when some resources free up to continue the migration.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Overbryd commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Overbryd commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-817718310


   @ephraimbuddy ok. I have now set `AIRFLOW__KUBERNETES__DELETE_WORKER_PODS` to `False`. Let me see if I triggers again.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] rrbarbosa commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
rrbarbosa commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-844789569


   We are also experiencing tasks getting stuck in Airflow 2.0.1 with the Kubernetes Executor. We created a "heartbeat" DAG that runs every 10m, when it doesn't, we get an alert.
   
   In the last occurrence, the scheduler seem to think the heartbeat DAG reached its `max_active_runs (1 of 1)`, so it refused to schedule a new task. As far as I can tell, the previous execution succeeded and the pod correctly was killed. 
   
   That task remained in the `scheduled` state, and I the message `Reset the following X orphaned TaskInstances` for the only task in this DAG every 5min. After ~1h hour, the DAG was finally executed again.
   
   Our current workaround is a cronjob to restart the scheduler pod every few hours.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] maxcountryman commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
maxcountryman commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-876053681


   Chiming in here to say we also noticed a stuck task this morning after updating to 2.1.1 on Tuesday.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Jorricks commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
Jorricks commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-875932438


   > We're only making use of the `default_pool` and tasks are moving from `Queued` to `Running` there but aren't entering the queue despite being in a `Scheduled` state.
   
   
   We had a similar issue and increased the `default_pool` 10x (from 128 to 1000). This fixed it completely for us.
   Did you already attempt something similar? Wondering if this helps in your case as well


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853117451


   More data, if it can benefit:
   Airflow version: `2.1.0`
   Kubernetes version: `1.18.9`
   
   I'm using Airflow `2.1.0` (upgraded from `1.10.15` in our development environment) and trying to use the `KubernetesExecutor`. I am running _extremely_ basic DAGs that exist solely to test. In `airflow.cfg` I set `delete_worker_pods = False` so that I could try and examine what's going on. As far as I can tell, my worker pods are being created, immediately considering themselves successful (not sure if they actually receive a task to complete), and terminating. I don't think the pods are "successful but not writing the success state to the DB" because one of my tasks is a simple `CREATE TABLE` statement on my data warehouse, and the table does not appear.
   
   
   Here are the results of various logs:
   
   If I run `kubectl get pods`, I see my `airflow-webserver` and `airflow-scheduler` pods, as well as some "Completed" worker pods.
   
   ```
   NAME                                                           READY   STATUS      RESTARTS   AGE
   airflow-scheduler                                              1/1     Running     0          45m
   airflow-webserver                                              2/2     Running     0          56m
   <GENERATED_WORKER_POD_1>                                       0/1     Completed   0          15m
   <GENERATED_WORKER_POD_2>                                       0/1     Completed   0          56m
   <GENERATED_WORKER_POD_3>                                       0/1     Completed   0          45m
   ```
   
   `kubectl logs <my_worker_pod>` - I can tell that the pod was briefly alive but that did nothing, because the only output is a line that always appears as a side effect of the Docker image config. In contrast, if I get the logs from one of my functional `1.10.15` worker pods, I can see the task start:
   ```
   [2021-06-02 13:33:01,358] {__init__.py:50} INFO - Using executor LocalExecutor
   [2021-06-02 13:33:01,358] {dagbag.py:417} INFO - Filling up the DagBag from /usr/local/airflow/dags/my_dag.py
   ...
   etc
   ```
   
   `kubectl describe pod <my_worker_pod>` - the event log is quite tame:
   ```
   Events:
     Type    Reason     Age    From               Message
     ----    ------     ----   ----               -------
     Normal  Scheduled  3m21s  default-scheduler  Successfully assigned <GENERATED_POD_NAME> to <EC2_INSTANCE>
     Normal  Pulling    3m19s  kubelet            Pulling image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Pulled     3m19s  kubelet            Successfully pulled image <MY_AIRFLOW_2.1.0_DOCKER_IMAGE>
     Normal  Created    3m19s  kubelet            Created container base
     Normal  Started    3m19s  kubelet            Started container base
   ```
   
   `kubectl logs airflow-scheduler` - I've trimmed the logs significantly, but here are the statements mentioning a stuck task
   ```
   [2021-06-02 14:49:42,742] {kubernetes_executor.py:369} INFO - Attempting to finish pod; pod_id: <GENERATED_POD_NAME>; state: None; annotations: {'dag_id': '<TEST_DAG>', 'task_id': '<TASK_ID>', 'execution_date': '2021-05-30T00:00:00+00:00', 'try_number': '1'}
   [2021-06-02 14:49:42,748] {kubernetes_executor.py:546} INFO - Changing state of (TaskInstanceKey(dag_id='<TEST_DAG>', task_id='<TASK_ID>', execution_date=datetime.datetime(2021, 5, 30, 0, 0, tzinfo=tzlocal()), try_number=1), None, '<GENERATED_POD_NAME>', 'airflow', '1000200742') to None
   [2021-06-02 14:49:42,751] {scheduler_job.py:1212} INFO - Executor reports execution of <TASK_ID> execution_date=2021-05-30 00:00:00+00:00 exited with status None for try_number 1
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849537076


   `kill -USR2 <pid of scheduler>` -- how you get the pid depends upon how and where you are running it :)
   
   Likely exec in to the container, run `ps auxww` and find the oldest scheduler processs (you'll see some sub processes, possibly named helpfully).


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-824886213


   @lukas-at-harren  Ahhhh, right, yes if you have dynamic dags and those dags disappear this is currently and edge case.  I've split that off as a separate issue, as I _think_ it's separate to the OP's issue.
   
   @Overbryd Do you have static dags (where _every_ time you parse a dag file it produces the same DAGs) or dynamic (where there is some loop/external file read where the result of parsing the dag file could produce different results)?
   
   What method do you use to get the DAGs available to the pods created by KubeExecutor? Is it possible that some pods are being created with "stale" dag definitions (if for example if you have DAG files baked in to the image and your image pull policy means that and old image is used on some nodes)?
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-845106707


   > I can observe the same problem with version 2.0.2:
   > 
   > * Tasks fail, because a DAG/task has gone missing (we are using dynamically created DAGs, and they can go missing)
   > * The scheduler keeps those queued
   > * The pool gradually fills up with these queued tasks
   > * The whole operation stops, because of this behaviour
   > 
   > My current remedy:
   > 
   > * Manually remove those queued tasks
   > 
   > My desired solution:
   > 
   > When a DAG/task goes missing while it is queued, it should end up in a failed state.
   
   Can you be able to try this with airflow master? We recently merged this https://github.com/apache/airflow/pull/15929. I'm thinking it will help in 'The scheduler keeps those queued' part


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hafid-d commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hafid-d commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849260716


   Any update on this ? Facing the same issue.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] timgriffiths commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
timgriffiths commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-854356862


   Environment:
   
   Airflow 2.1.0
   Docker : apache/airflow:2.1.0-python3.8
   Executor : Celery
   
   So I have also hit this today, the way i was able to replicate this 
   
   -> have a py file that generates move than 1 dag
   -> then have a user break 1 of the dag from this resultant chain
   -> all the other still valid dags will be triggered in the scheduler but fail in the Worker but it looks like the exception is not passed back to the Scheduler so it permanently stays in a "queued state"
   
   `[2021-06-04 01:11:22,927: INFO/ForkPoolWorker-40] Executing command in Celery: ['airflow', 'tasks', 'run', 'myworkingjob', 'taskname-abc', '2021-06-04T01:11:21.775097+00:00', '--local', '--pool', 'pool', '--subdir', '/opt/airflow/dags/mm-dynamic-dags.py']
   [2021-06-04 01:11:22,940: INFO/ForkPoolWorker-34] Filling up the DagBag from /opt/airflow/dags/mm-dynamic-dags.py
   [2021-06-04 01:11:27,993: ERROR/ForkPoolWorker-39] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dagbag.py", line 317, in _load_modules_from_file
       loader.exec_module(new_module)
     File "<frozen importlib._bootstrap_external>", line 848, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 187, in <module>
       create_dag("{}/{}".format(global_dags_base_url, dag_file))
     File "/opt/airflow/dags/mm-dynamic-dags.py", line 123, in create_dag
       dag = DAG(
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/dag.py", line 277, in __init__
       validate_key(dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/helpers.py", line 44, in validate_key
       raise AirflowException(
   airflow.exceptions.AirflowException: The key (test-test+abc) has to be made of alphanumeric characters, dashes, dots and underscores exclusively
   [2021-06-04 01:11:27,996: ERROR/ForkPoolWorker-39] Failed to execute task dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse..
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 116, in _execute_in_fork
       args.func(args)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 91, in wrapper
       return f(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/commands/task_command.py", line 219, in task_run
       dag = get_dag(args.subdir, args.dag_id)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 191, in get_dag
       raise AirflowException(
   airflow.exceptions.AirflowException: dag_id could not be found: myworkingjob. Either the dag did not exist or it failed to parse.
   [2021-06-04 01:11:28,017: ERROR/ForkPoolWorker-39] Task airflow.executors.celery_executor.execute_command[66453177-4042-4c6a-882b-bd0fcdbda0d8] raised unexpected: AirflowException('Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm')
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 412, in trace_task
       R = retval = fun(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/celery/app/trace.py", line 704, in __protected_call__
       return self.run(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 87, in execute_command
       _execute_in_fork(command_to_exec)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 98, in _execute_in_fork
       raise AirflowException('Celery command failed on host: ' + get_hostname())
   airflow.exceptions.AirflowException: Celery command failed on host: airflow-mm-worker-5464db666b-f8mlm
   [2021-06-04 01:11:28,091: ERROR/ForkPoolWorker-37] Failed to import: /opt/airflow/dags/mm-dynamic-dags.py`
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] alete89 edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
alete89 edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-825688001


   As many others here, we're also experiencing 0 slots, and task scheduled but never being run on Airflow 1.10.14 on kubernetes.
   Restarting the scheduler pod did trigger the run for those stuck tasks. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-824899155


   > @ashb The setup works like this (I am @Overbryd, just a different account.)
   
   :lolsob:
   
   Got it!
   
   Right, I think we have enough case to fix this now, and there are two areas to fix:
   
   1. Dags that vanish should get their queued tasks failed.
   2. The task failing with DAG not found should _also_ get reported back and make the executor/scheduler update the state.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lukas-at-harren commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
lukas-at-harren commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-824407606


   Running on 2.0.2 now, will keep you posted.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-853693757


   I have resolved this issue for my environment! I'm not sure if this is the same "bug" as others, or a different issue with similar symptoms. But here we go
   
   ----
   
   In my airflow Docker image, the entrypoint is just a bootstrap script that accepts `webserver` or `scheduler` as arguments, and accepts the appropriate command.
   ```
   # Installing python libs, jars, etc
   ...
   ENTRYPOINT ["/bootstrap.sh"]
   ```
   
   `bootstrap.sh`:
   ```
   if [ "$1" = "webserver" ]
   then
   	exec airflow webserver
   fi
   
   if [ "$1" = "scheduler" ]
   then
   	exec airflow scheduler
   fi
   ```
   
   Previous to #12766, the KubernetesExecutor fed the `airflow tasks run` (or `airflow run` in older versions) into the `command` section of pod YAML.
   
   ```
   "containers": [
         {
           "args": [],
           "command": [
             "airflow",
             "run",
             "my_dag",
             "my_task",
             "2021-06-03T03:40:00+00:00",
             "--local",
             "--pool",
             "default_pool",
             "-sd",
             "/usr/local/airflow/dags/my_dag.py"
           ],
   ```
   This works fine for my setup -- the `command` just overrides my Docker's `ENTRYPOINT`, the pod executes its given command and terminates on completion. However, [this](https://github.com/apache/airflow/pull/12766/files#diff-681de8974a439f70dfa41705f5c1681ecce615fac6c4c715c1978d28d8f0da84L300) change moved the `airflow tasks run` issuance to the `args` section of the YAML. 
   ```
   'containers': [{'args': ['airflow',
                                      'tasks',
                                      'run',
                                      'my_dag,
                                      'my_task',
                                      '2021-06-02T00:00:00+00:00',
                                      '--local',
                                      '--pool',
                                      'default_pool',
                                      '--subdir',
                                      '/usr/local/airflow/dags/my_dag.py'],
                             'command': None,
   ```
   
   These new args do not match either `webserver` or `scheduler` in `bootstrap.sh`, therefore the script ends cleanly and so does the pod. Here is my solution, added to the bottom of `bootstrap.sh`:
   ```
   if [ "$1" = "airflow" ] && [ "$2" = "tasks" ] && [ "$3" = "run" ]
   then
   	exec "$@"
   fi
   ```
   Rather than just allow the pod to execute _whatever_ it's given in `args` (aka just running `exec "$@"` without a check), I decided to at least make sure the pod is being fed an `airflow run task` command.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hellmean commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hellmean commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-929043044


   Having a similar issue. The dags' status is green, but the underlying tasks are in the queue, and they won't proceed for days. When you clear the dag run, it starts to go fine again. The restart of the executor does not do anything.
   
   airflow 2.1.1, Postgres, 13.3 standalone executor, we use Python operators only
   <img width="1668" alt="Screenshot 2021-09-28 at 12 01 51" src="https://user-images.githubusercontent.com/14246376/135066670-13005640-3b77-4051-ae9d-872313f0bd58.png">
   .
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-819436342


   Thank you @lukas-at-harren, you captured it correctly because the error message in all cases is that the dag did not exist.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] easontm edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
easontm edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-911359911


   Thanks @ephraimbuddy! That wasn't it but I did find something that _may_ be issue-worthy. I added some custom logging to `scheduler_job._start_queued_dagruns()` and noticed that the contents of `dag_runs` was the same 20 DAGruns in every loop (default count from config `max_dagruns_per_loop_to_schedule`) -- the single DAG associated with these DAGruns is _not_ the one pictured. The DAGruns in question show up first when ordering by the following code from `dagrun.next_dagruns_to_examine()`:
   ```
   .order_by(
       nulls_first(cls.last_scheduling_decision, session=session),
       cls.execution_date,
   )
   ```
   
   This DAG is set to `max_active_runs=1` so all 20 examined queued DAGruns do not change state (because there is another running already). The problem arises because the `_start_queued_dagruns()` function (AFAIK) doesn't update `last_scheduling_decision`, so every time the query is run to get next DAGruns, the same ones appear (and continue to not be scheduled if the currently active DAGrun for that DAG takes a long time -- and it continues so long as there are more than 20 DAGruns queued).
   
   I think the `last_scheduling_decision` column needs to be updated somewhere here:
   ```
   if dag.max_active_runs and active_runs >= dag.max_active_runs:
       self.log.debug(
           "DAG %s already has %d active runs, not moving any more runs to RUNNING state %s",
           dag.dag_id,
           active_runs,
           dag_run.execution_date,
       )
   ```
   
   I was able to get around this issue currently by simply increasing the number of DAGruns handled per loop (and telling my users not to queue so many), but perhaps it should still be addressed.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812742063


   I ran into this issue due to the scheduler over-utilizing CPU because our `min_file_process_interval` was set to 0 (the default prior to 2.0), which in airflow 2.0 causes a ton of CPU work from constantly refreshing DAG files. Setting this parameter to 60 fixed the issue.
   The stack I observed this on:
   host: AWS ECS Cluster
   executor: CeleryExecutor
   queue: AWS SQS Queue
   
   The behavior I observed was that the scheduler would mark tasks are "queued", but never actually send them to the queue (I think the scheduler does actual queueing via the executor). My manual workaround until correcting the `min_file_process_interval` param was to stop the scheduler, clear queued tasks, and then start a new scheduler. The new scheduler would temporarily properly send tasks to the queue, before degenerating to marking tasks as queued without sending to the queue.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] MatrixManAtYrService edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
MatrixManAtYrService edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849031715


   While trying to recreate this, I wrote a [stress test](https://github.com/MatrixManAtYrService/airflow-git-sync/blob/master/scheduler_stress.py) which I ran overnight on my local microk8s cluster (release:2.0.1+beb8af5ac6c438c29e2c186145115fb1334a3735 configured like [this](https://github.com/MatrixManAtYrService/airflow-git-sync/blob/master/zsh.stdin)).
   
   I was hoping that it would get fully stuck by the time I woke.  Instead there were only two stuck tasks: 
    
   ![stucktasks](https://user-images.githubusercontent.com/5834582/119712787-34db0f80-be1e-11eb-9a41-875dd88c0566.gif)
   
   Deleting the scheduler pod and letting kubernetes recreate it did cause the stuck tasks to complete.  My current guess is that it won't get stuck in a restarting-scheduler-doesnt-help kind of way until the number of stuck-queued tasks matches `dag_concurrency`.  I'll update here if I'm able to confirm that guess.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hellmean commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
hellmean commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-929043044


   Having a similar issue. The dags' status is green, but the underlying tasks are in the queue, and they won't proceed for days. When you clear the dag run, it starts to go fine again. The restart of the executor does not do anything.
   
   airflow 2.1.1, Postgres, 13.3 standalone executor, we use Python operators only
   <img width="1668" alt="Screenshot 2021-09-28 at 12 01 51" src="https://user-images.githubusercontent.com/14246376/135066670-13005640-3b77-4051-ae9d-872313f0bd58.png">
   .
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nitinpandey-154 commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
nitinpandey-154 commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-849533941


   How can we send SIGUSR2 to the scheduler process @ashb ? 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SalmonTimo edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
SalmonTimo edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-812742063


   I ran into this issue due to the scheduler over-utilizing CPU because our `min_file_process_interval` was set to 0 (the default prior to 2.0), which in airflow 2.0 causes 100% CPU utilization by constantly searching for new DAG files. Setting this parameter to 60 fixed the issue.
   
   The stack I observed this on:
   host: AWS ECS Cluster
   executor: CeleryExecutor
   queue: AWS SQS Queue
   
   The behavior I observed was that the scheduler would mark tasks are "queued", but never actually send them to the queue. I think the scheduler does the actual queueing via the executor, so I suspect that the executor is starved of resources and unable to queue the new task. My manual workaround until correcting the `min_file_process_interval` param was to stop the scheduler, clear queued tasks, and then start a new scheduler. The new scheduler would temporarily properly send tasks to the queue, before degenerating to marking tasks as queued without sending to the queue.
   
   I suspect the OP may have this same issue because they mentioned having 100% CPU utilization on their scheduler.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jonathonbattista commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jonathonbattista commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-770431504


   We are experiencing these symptoms as well. Doesn't seem `git-sync` related.
   
   Does any work have a clue why this is happening?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] val2k edited a comment on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
val2k edited a comment on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029786174


   @jpkoponen I adapted the @danmactough DAG to make it automatic and fit our use case. In filter DAGs that have been stuck for more than 5 minutes, and simply delete them. (In my case, changing the `try_number` and the state has no other effect than queuing the DAG again).
   
   ```python
   import os
   import requests
   import time
   import json
   from datetime import datetime, timedelta
   from pprint import pprint
   
   from airflow import DAG
   from airflow.models.dagrun import DagRun
   from airflow.models.taskinstance import TaskInstance
   from airflow.operators.python import PythonOperator
   from airflow.utils import timezone
   from airflow.utils.db import provide_session
   from airflow.utils.state import State
   from dependencies.utils.var import DATADOG_API_KEY
   
   DAG_NAME = os.path.splitext(os.path.basename(__file__))[0]
   DEFAULT_ARGS = {
       "owner": "airflow",
       "depends_on_past": False,
       "email_on_failure": False,
       "email_on_retry": False,
       "execution_timeout": timedelta(minutes=10),
       "retries": 0,
   }
   
   @provide_session
   def unstick_dag_callable(dag_run, session, **kwargs):
       filter = [
           TaskInstance.state == State.QUEUED,
           TaskInstance.queued_dttm < datetime.now(timezone.utc) - timedelta(minutes=5)
       ]
   
       tis = session.query(TaskInstance).filter(*filter).all()
       print(f"Task instances: {tis}")
       print(f"Updating {len(tis)} task instances")
   
   
       for ti in tis:
           dr = (
               session.query(DagRun)
               .filter(DagRun.run_id == ti.dag_run.run_id)
               .first()
           )
   
           dagrun = (
               dict(
                   id=dr.id,
                   dag_id=dr.dag_id,
                   execution_date=dr.execution_date,
                   start_date=dr.start_date,
                   end_date=dr.end_date,
                   _state=dr._state,
                   run_id=dr.run_id,
                   creating_job_id=dr.creating_job_id,
                   external_trigger=dr.external_trigger,
                   run_type=dr.run_type,
                   conf=dr.conf,
                   last_scheduling_decision=dr.last_scheduling_decision,
                   dag_hash=dr.dag_hash,
               )
               if dr
               else {}
           )
           pprint(
               dict(
                   task_id=ti.task_id,
                   job_id=ti.job_id,
                   key=ti.key,
                   dag_id=ti.dag_id,
                   execution_date=ti.execution_date,
                   state=ti.state,
                   dag_run={**dagrun},
               )
           )
   
   
           dr.state = State.FAILED
           print(f"Deleting {str(ti)}.")
           session.delete(ti)
   
       session.commit()
       print("Done.")
   
   
   with DAG(
       DAG_NAME,
       description="Utility DAG to fix TaskInstances stuck in queued state",
       default_args=DEFAULT_ARGS,
       schedule_interval="*/5 * * * *",
       start_date=datetime(year=2021, month=8, day=1),
       max_active_runs=1,
       catchup=False,
       default_view="graph",
       is_paused_upon_creation=False,
   ) as dag:
       PythonOperator(task_id="unstick_dag", python_callable=unstick_dag_callable)
   ```
      The DAG runs every 5 minutes and I never caught it in a queued state.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jpkoponen commented on issue #13542: Task stuck in "scheduled" or "queued" state, pool has all slots queued, nothing is executing

Posted by GitBox <gi...@apache.org>.
jpkoponen commented on issue #13542:
URL: https://github.com/apache/airflow/issues/13542#issuecomment-1029887270


   Thank you @val2k !
   
   It's interesting that increasing `try_number` and modifying the `STATE` did not work for you. I will first try the automatic method without deleting hoping that it will work for me.
   
   I just wish there was a way to test the DAG other than waiting for the issue to arise again. I think we have had it happen only twice in the past two weeks. 😄 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org