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 2020/11/05 12:59:30 UTC

[GitHub] [airflow] olchas opened a new issue #12103: Tasks can be stuck in running state indefinitely

olchas opened a new issue #12103:
URL: https://github.com/apache/airflow/issues/12103


   <!--
   
   Welcome to Apache Airflow!  For a smooth issue process, try to answer the following questions.
   Don't worry if they're not all applicable; just try to include what you can :-)
   
   If you need to include code snippets or logs, please put them in fenced code
   blocks.  If they're super-long, please use the details tag like
   <details><summary>super-long log</summary> lots of stuff </details>
   
   Please delete these comment blocks before submitting the issue.
   
   -->
   
   <!--
   
   IMPORTANT!!!
   
   PLEASE CHECK "SIMILAR TO X EXISTING ISSUES" OPTION IF VISIBLE
   NEXT TO "SUBMIT NEW ISSUE" BUTTON!!!
   
   PLEASE CHECK IF THIS ISSUE HAS BEEN REPORTED PREVIOUSLY USING SEARCH!!!
   
   Please complete the next sections or the issue will be closed.
   These questions are the first thing we need to know to understand the context.
   
   -->
   
   **Apache Airflow version**: 1.10.10
   
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`): -
   
   **Environment**: Cloud Composer
   
   - **Composer addon version**: 1.12.4
   - **Python version**: 3 (3.6.10)
   - **Node count**: 3
   - **Machine type**: n1-standard-2
   
   **What happened**:
   
   Sometimes Airflow tasks can be stuck in `running state` indefinitely.
   
   I was testing Cloud Composer with Airflow 1.10.10 using [elastic Dag](https://github.com/apache/airflow/blob/master/tests/test_utils/perf/dags/elastic_dag.py). My setup of its environment variables was as follows:
   - "PERF_DAG_PREFIX": "workflow",
   - "PERF_DAGS_COUNT": "10",
   - "PERF_TASKS_COUNT": "100",
   - "PERF_START_AGO": "1d",
   - "PERF_SCHEDULE_INTERVAL": "@once",
   - "PERF_SHAPE": "no_structure",
   
   So the Composer instance was supposed to execute 1000 tasks total. The test started on October 28th at around 5:30 PM UTC, and while 998 tasks finished within 70 minutes (at around 6:40 PM UTC), the last two are still in `running` state. Scheduler and workers are generally running fine, as `airflow_monitoring` Dag, present on every Composer instance, is regularly executed. Moreover, local task jobs responsible for these two tasks, are also running and updating their heartbeat.
   
   ![image](https://user-images.githubusercontent.com/34898234/98236656-fd7d4200-1f63-11eb-8de4-8169f4ef92be.png)
   
   I have done some digging. Both tasks are running on the same hostname: `airflow-worker-86455b549d-zkjsc`. I have checked the contents of `logs` table and it appears that these two tasks were among the last 6 this worker has started, which happened at around 5:50 PM UTC (so long before the test was finished). After that point, no more tasks (even for `airflow_monitoring`) were executed by this worker.
   - according to `logs` table, for the remaining 4 tasks this worker has only managed to start the `--local` process, but has not changed their status to `running` (no `running` event present). For each of these tasks, after around 10 minutes has passed, a new `--local` process was executed by different worker and the task was executed as normal. Within those 10 minutes the scheduler job has restarted, so I assume the scheduler simply rescheduled the tasks in `queued` state after its restart.
   - for the two "hanging" tasks, however, their state was changed to `running` and `--raw` process has started as well.
   
   For all of these 6 tasks there is also yet another event in `logs` table with executing the `--local` process (on worker other than `airflow-worker-86455b549d-zkjsc`), occurring at around 11:50 PM UTC, so after 6 hours of the first attempt to start these tasks. This time corresponds to the default value of 21600 seconds for celery broker's `visibility_timeout` option. In [airflow default config template](https://github.com/apache/airflow/blob/master/airflow/config_templates/default_airflow.cfg) we can read:
   
   > The visibility timeout defines the number of seconds to wait for the worker
   > to acknowledge the task before the message is redelivered to another worker.
   > Make sure to increase the visibility timeout to match the time of the longest
   > ETA you're planning to use.
   > visibility_timeout is only supported for Redis and SQS celery brokers. 
   
   So it appears the worker `airflow-worker-86455b549d-zkjsc` has not acknowledged getting the message from Redis queue and after 6 hours, the tasks were redelivered to other workers. But at this point, the 4 tasks were already finished and 2 still running, and neither of these two situations is a valid state for task execution. You can even see the attempt to run the hanging task again in its log file:
   
   ```
   *** Reading remote log from gs://us-central1-cmp-latest-af1--6dadbf30-bucket/logs/workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once/tasks__47_of_100/2020-10-27T17:31:07.372135+00:00/1.log.
   [2020-10-28 17:49:46,865] {taskinstance.py:670} INFO - Dependencies all met for <TaskInstance: workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once.tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 [queued]>
   [2020-10-28 17:49:46,932] {taskinstance.py:670} INFO - Dependencies all met for <TaskInstance: workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once.tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 [queued]>
   [2020-10-28 17:49:46,932] {taskinstance.py:880} INFO -
   --------------------------------------------------------------------------------
   [2020-10-28 17:49:46,933] {taskinstance.py:881} INFO - Starting attempt 1 of 1
   [2020-10-28 17:49:46,933] {taskinstance.py:882} INFO -
   --------------------------------------------------------------------------------
   [2020-10-28 17:49:47,028] {taskinstance.py:901} INFO - Executing <Task(PythonOperator): tasks__47_of_100> on 2020-10-27T17:31:07.372135+00:00
   [2020-10-28 17:49:47,029] {base_task_runner.py:131} INFO - Running on host: airflow-worker-86455b549d-zkjsc
   [2020-10-28 17:49:47,029] {base_task_runner.py:132} INFO - Running: ['airflow', 'run', 'workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once', 'tasks__47_of_100', '2020-10-27T17:31:07.372135+00:00', '--job_id', '385', '--pool', 'default_pool', '--raw', '-sd', 'DAGS_FOLDER/elastic_dag.py', '--cfg_path', '/tmp/tmpkgoycqh4']
   [2020-10-28 23:49:55,650] {taskinstance.py:664} INFO - Dependencies not met for <TaskInstance: workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once.tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 [running]>, dependency 'Task Instance State' FAILED: Task is in the 'running' state which is not a valid state for execution. The task must be cleared in order to be run.
   ```
   
   I have checked the GKE cluster running for given Composer instance. However, the pod `airflow-worker-86455b549d-zkjsc` appears to be running fine and I cannot see any issues with it using kubectl. Its logs do not reveal any errors (apart from the last log message being from October 28th at around 5:50 PM UTC). I have also connected to the pod and I was able to see that the aforementioned  `--local` and `--raw` processes are still running here as well.
   
   ```
   USER         PID %CPU %MEM    VSZ   RSS TTY      STAT START   TIME COMMAND
   airflow        1  0.0  0.0  19796  3236 ?        Ss   Oct28   0:00 /bin/bash /var/local/airflow.sh worker
   airflow       15  0.0  0.3 939096 28564 ?        Ssl  Oct28   0:58 /usr/local/go/bin/gcsfuse --foreground --file-mode 755 --implicit-dirs --limit-ops-per-sec -1 us-central1-cmp-latest-af1--6dadbf30-bucket /home/airflow/gcsfuse
   airflow       72  0.1  1.4 408800 107312 ?       S    Oct28  18:45 [celeryd: celery@airflow-worker-86455b549d-zkjsc:MainProcess] -active- (worker)
   airflow       77  0.0  1.2 395484 98192 ?        Sl   Oct28   1:32 /usr/bin/python /usr/local/bin/airflow serve_logs
   airflow       78  0.0  1.1 407548 88796 ?        S    Oct28   0:00 [celeryd: celery@airflow-worker-86455b549d-zkjsc:ForkPoolWorker-1]
   airflow       79  0.0  1.1 407552 88864 ?        S    Oct28   0:00 [celeryd: celery@airflow-worker-86455b549d-zkjsc:ForkPoolWorker-2]
   airflow       80  0.0  1.1 407556 88864 ?        S    Oct28   0:00 [celeryd: celery@airflow-worker-86455b549d-zkjsc:ForkPoolWorker-3]
   airflow       83  0.0  1.1 407560 88868 ?        S    Oct28   0:00 [celeryd: celery@airflow-worker-86455b549d-zkjsc:ForkPoolWorker-4]
   airflow       84  0.0  1.1 407564 88868 ?        S    Oct28   0:00 [celeryd: celery@airflow-worker-86455b549d-zkjsc:ForkPoolWorker-5]
   airflow       85  0.0  1.1 407568 88872 ?        S    Oct28   0:00 [celeryd: celery@airflow-worker-86455b549d-zkjsc:ForkPoolWorker-6]
   airflow     2104  0.3  1.3 470824 106056 ?       Sl   Oct28  35:08 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 --local --pool default_pool -sd /home/airflow/gcs/dags/elastic_dag.py
   airflow     2107  0.0  1.3 404660 105060 ?       Sl   Oct28   0:04 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_6_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 --local --pool default_pool -sd /home/airflow/gcs/dags/elastic_dag.py
   airflow     2108  0.0  1.3 404348 105000 ?       Sl   Oct28   0:04 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_10_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__48_of_100 2020-10-27T17:31:07.372135+00:00 --local --pool default_pool -sd /home/airflow/gcs/dags/elastic_dag.py
   airflow     2109  0.3  1.3 470824 106264 ?       Sl   Oct28  35:12 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_8_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 --local --pool default_pool -sd /home/airflow/gcs/dags/elastic_dag.py
   airflow     2110  0.0  1.3 404356 104972 ?       Sl   Oct28   0:04 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_1_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__48_of_100 2020-10-27T17:31:07.372135+00:00 --local --pool default_pool -sd /home/airflow/gcs/dags/elastic_dag.py
   airflow     2111  0.0  1.3 404348 104992 ?       Sl   Oct28   0:04 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_2_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__48_of_100 2020-10-27T17:31:07.372135+00:00 --local --pool default_pool -sd /home/airflow/gcs/dags/elastic_dag.py
   airflow     2130  0.0  1.3 404628 105060 ?       Ssl  Oct28   0:03 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_3_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 --job_id 385 --pool default_pool --raw -sd DAGS_FOLDER/elastic_dag.py --cfg_path /tmp/tmpkgoycqh4
   airflow     2138  0.0  1.3 404624 105068 ?       Ssl  Oct28   0:03 /usr/bin/python /usr/local/bin/airflow run workflow__SHAPE_no_structure__DAGS_COUNT_8_of_10__TASKS_COUNT_100__START_DATE_1d__SCHEDULE_INTERVAL_once tasks__47_of_100 2020-10-27T17:31:07.372135+00:00 --job_id 387 --pool default_pool --raw -sd DAGS_FOLDER/elastic_dag.py --cfg_path /tmp/tmp7zjbj1qc
   airflow    79624  0.0  0.0  19972  3544 pts/0    Ss   17:06   0:00 /bin/bash
   airflow    79652  0.0  0.0  36128  3164 pts/0    R+   17:08   0:00 ps -aux
   ```
   If have attempted to kill one of the `--raw` processes with the pid 2130. After sending the SIGTERM signal to it, the LocalTaskJob 385 (from screen above) changed state to `success` and the task was marked as `failed`. No new messages in task's log file. Interestingly enough, the process with pid 2104 (the `--local` process for the terminated `--raw` process), has not terminated by itself.
   
   ![image](https://user-images.githubusercontent.com/34898234/98243061-b3995980-1f6d-11eb-9d0e-d326664afc53.png)
   
   <!-- (please include exact error messages if you can) -->
   
   **What you expected to happen**:
   
   The tasks are not stuck in running state forever.
   
   <!-- What do you think went wrong? -->
   
   **How to reproduce it**:
   <!---
   
   As minimally and precisely as possible. Keep in mind we do not have access to your cluster or dags.
   
   If you are using kubernetes, please attempt to recreate the issue using minikube or kind.
   
   ## Install minikube/kind
   
   - Minikube https://minikube.sigs.k8s.io/docs/start/
   - Kind https://kind.sigs.k8s.io/docs/user/quick-start/
   
   If this is a UI bug, please provide a screenshot of the bug or a link to a youtube video of the bug in action
   
   You can include images using the .md style of
   ![alt text](http://url/to/img.png)
   
   To record a screencast, mac users can use QuickTime and then create an unlisted youtube video with the resulting .mov file.
   
   --->
   
   It is hard, as this issue appears randomly. You would have to repeat the test I have done, hoping for the issue to occur. This issue is not related to Cloud Composer specifically, as I have encountered it a few months back using different setup (with Airflow 1.10.3, python 3.7 running on Microsoft Azure virtual machines). From what I can say, it appears to be related to celery/redis, so an instance you use should have CeleryExecutor with Redis as message broker. In the aforementioned previous setup this issue started occurring after switching Airflow instances from Python 2.7 to Python 3.7, so this issue may affect only python 3 instances, though I was not doing tests for python 2.7.
   
   **Anything else we need to know**:
   
   <!--
   
   How often does this problem occur? Once? Every time etc?
   
   Any relevant logs to include? Put them here in side a detail tag:
   <details><summary>x.log</summary> lots of stuff </details>
   
   -->
   


----------------------------------------------------------------
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] subparry commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   In Airflow 2.2.1 this problem still happens


-- 
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 closed issue #12103: Tasks can be stuck in running state indefinitely

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #12103:
URL: https://github.com/apache/airflow/issues/12103


   


-- 
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] ambeshsingh commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   I am facing the same issue, any solution or workaround?


-- 
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] eladkal commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   This issue is reported against old version of Airflow that reached end of life.
   If the issue is reproducible on latest Airflow version please let us know


-- 
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] ambeshsingh commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   > This issue is reported against old version of Airflow that reached end of life. If the issue is reproducible on latest Airflow version please let us know
   
   @eladkal yes, I am on composer-1.17.2-airflow-1.10.15 version and intermittently facing 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] li-alan-owneriq commented on issue #12103: Tasks can be stuck in running state indefinitely

Posted by GitBox <gi...@apache.org>.
li-alan-owneriq commented on issue #12103:
URL: https://github.com/apache/airflow/issues/12103#issuecomment-1070981173


   Having this issue in Airflow 2.2.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] lucidsushi commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   not to resurrect this, but the error still happens in airflow 2.1.1, just logging it so people can see:
   
   `Dependencies not met for <TaskInstance: ___ [running]>, dependency 'Task Instance State' FAILED: Task is in the 'running' state which is not a valid state for execution. The task must be cleared in order to be 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ambeshsingh commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   I am facing the same issue, any solution or workaround?


-- 
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] li-alan-owneriq commented on issue #12103: Tasks can be stuck in running state indefinitely

Posted by GitBox <gi...@apache.org>.
li-alan-owneriq commented on issue #12103:
URL: https://github.com/apache/airflow/issues/12103#issuecomment-1070981173


   Having this issue in Airflow 2.2.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] olchas commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   Approximately 10 minutes after sending SIGTERM to the pid 2130, the entire pod got restarted, which effectively cleared the remaining hanging processes. However, there is no error message in the log of pod `airflow-worker-86455b549d-zkjsc` apart from the info on the terminated task ending with cod `-15`:
   
   ```
   [2020-11-05 12:45:14,735] {local_task_job.py:103} INFO - Task exited with return code -15
   ```
   
   Around the time of worker pod restart, I found the following error message in scheduler pod
   
   ```
   [2020-11-05 12:54:24,015] {dagbag.py:397} INFO - Filling up the DagBag from /home/airflow/gcs/dags/elastic_dag.py
   [2020-11-05 12:54:35,050] {dagbag.py:397} INFO - Filling up the DagBag from /home/airflow/gcs/dags/airflow_monitoring.py
   Process DagFileProcessor502-Process:
   Traceback (most recent call last):
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 2285, in _wrap_pool_connect
       return fn()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 363, in connect
       return _ConnectionFairy._checkout(self)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 773, in _checkout
       fairy = _ConnectionRecord.checkout(pool)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 492, in checkout
       rec = pool._do_get()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/impl.py", line 139, in _do_get
       self._dec_overflow()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/langhelpers.py", line 69, in __exit__
       exc_value, with_traceback=exc_tb,
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 178, in raise_
       raise exception
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/impl.py", line 136, in _do_get
       return self._create_connection()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 308, in _create_connection
       return _ConnectionRecord(self)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 437, in __init__
       self.__connect(first_connect_check=True)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 657, in __connect
       pool.logger.debug("Error on connect(): %s", e)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/langhelpers.py", line 69, in __exit__
       exc_value, with_traceback=exc_tb,
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 178, in raise_
       raise exception
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 652, in __connect
       connection = pool._invoke_creator(self)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/strategies.py", line 114, in connect
       return dialect.connect(*cargs, **cparams)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/default.py", line 488, in connect
       return self.dbapi.connect(*cargs, **cparams)
     File "/opt/python3.6/lib/python3.6/site-packages/MySQLdb/__init__.py", line 85, in Connect
       return Connection(*args, **kwargs)
     File "/opt/python3.6/lib/python3.6/site-packages/MySQLdb/connections.py", line 208, in __init__
       super(Connection, self).__init__(*args, **kwargs2)
   _mysql_exceptions.OperationalError: (2006, "Unknown MySQL server host 'airflow-sqlproxy-service.default.svc.cluster.local' (110)")
   
   The above exception was the direct cause of the following exception:
   
   Traceback (most recent call last):
     File "/opt/python3.6/lib/python3.6/multiprocessing/process.py", line 258, in _bootstrap
       self.run()
     File "/opt/python3.6/lib/python3.6/multiprocessing/process.py", line 93, in run
       self._target(*self._args, **self._kwargs)
     File "/usr/local/lib/airflow/airflow/jobs/scheduler_job.py", line 164, in _run_file_processor
       pickle_dags)
     File "/usr/local/lib/airflow/airflow/utils/db.py", line 74, in wrapper
       return func(*args, **kwargs)
     File "/usr/local/lib/airflow/airflow/jobs/scheduler_job.py", line 1571, in process_file
       dag.sync_to_db()
     File "/usr/local/lib/airflow/airflow/utils/db.py", line 74, in wrapper
       return func(*args, **kwargs)
     File "/usr/local/lib/airflow/airflow/models/dag.py", line 1506, in sync_to_db
       DagModel).filter(DagModel.dag_id == self.dag_id).first()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3298, in first
       ret = list(self[0:1])
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3076, in __getitem__
       return list(res)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3403, in __iter__
       return self._execute_and_instances(context)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3425, in _execute_and_instances
       querycontext, self._connection_from_session, close_with_result=True
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3440, in _get_bind_args
       mapper=self._bind_mapper(), clause=querycontext.statement, **kw
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3418, in _connection_from_session
       conn = self.session.connection(**kw)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/session.py", line 1133, in connection
       execution_options=execution_options,
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/session.py", line 1139, in _connection_for_bind
       engine, execution_options
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/orm/session.py", line 432, in _connection_for_bind
       conn = bind._contextual_connect()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 2251, in _contextual_connect
       self._wrap_pool_connect(self.pool.connect, None),
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 2289, in _wrap_pool_connect
       e, dialect, self
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 1555, in _handle_dbapi_exception_noconnection
       sqlalchemy_exception, with_traceback=exc_info[2], from_=e
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 178, in raise_
       raise exception
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 2285, in _wrap_pool_connect
       return fn()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 363, in connect
       return _ConnectionFairy._checkout(self)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 773, in _checkout
       fairy = _ConnectionRecord.checkout(pool)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 492, in checkout
       rec = pool._do_get()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/impl.py", line 139, in _do_get
       self._dec_overflow()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/langhelpers.py", line 69, in __exit__
       exc_value, with_traceback=exc_tb,
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 178, in raise_
       raise exception
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/impl.py", line 136, in _do_get
       return self._create_connection()
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 308, in _create_connection
       return _ConnectionRecord(self)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 437, in __init__
       self.__connect(first_connect_check=True)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 657, in __connect
       pool.logger.debug("Error on connect(): %s", e)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/langhelpers.py", line 69, in __exit__
       exc_value, with_traceback=exc_tb,
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 178, in raise_
       raise exception
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/pool/base.py", line 652, in __connect
       connection = pool._invoke_creator(self)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/strategies.py", line 114, in connect
       return dialect.connect(*cargs, **cparams)
     File "/opt/python3.6/lib/python3.6/site-packages/sqlalchemy/engine/default.py", line 488, in connect
       return self.dbapi.connect(*cargs, **cparams)
     File "/opt/python3.6/lib/python3.6/site-packages/MySQLdb/__init__.py", line 85, in Connect
       return Connection(*args, **kwargs)
     File "/opt/python3.6/lib/python3.6/site-packages/MySQLdb/connections.py", line 208, in __init__
       super(Connection, self).__init__(*args, **kwargs2)
   sqlalchemy.exc.OperationalError: (_mysql_exceptions.OperationalError) (2006, "Unknown MySQL server host 'airflow-sqlproxy-service.default.svc.cluster.local' (110)")
   (Background on this error at: http://sqlalche.me/e/e3q8)
   [2020-11-05 12:55:00,149] {dagbag.py:397} INFO - Filling up the DagBag from /home/airflow/gcs/dags/airflow_monitoring.py
   [2020-11-05 12:55:00,153] {dagbag.py:397} INFO - Filling up the DagBag from /home/airflow/gcs/dags/elastic_dag.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] potiuk commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   > > This issue is reported against old version of Airflow that reached end of life. If the issue is reproducible on latest Airflow version please let us know
   > 
   > @eladkal yes, I am on composer-1.17.2-[airflow-1](https://issues.apache.org/jira/browse/AIRFLOW-1).10.15 version and intermittently facing this issue
   
   Yep. 1.10 is already End-Of-Life: https://github.com/apache/airflow/blob/main/README.md#version-life-cycle  for more than 4 months.
   
   Since Airflow 2 is already GA in Composer, I heartily recommend you to migrate to Airflow 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] gabriel19913 commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   Well, the most recent version is 2.2.2. Is this problem solved?


-- 
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] rjribeiro commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   In Airflow 2.1.3 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] rjribeiro commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   In Airflow 2.1.3 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] kevinlacire commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   @olchas Did you found some workaround by the time ?
   I've the same issue and spend multiple days to try to figure it out unsuccessfully...


-- 
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] andrewrjones commented on issue #12103: Tasks can be stuck in running state indefinitely

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


   We're also having this issue, and it looks like it may be solved by https://github.com/apache/airflow/pull/19375, released in Airflow 2.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