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 2022/06/16 18:44:01 UTC

[GitHub] [airflow] csm10495 commented on issue #21078: DAG with multiple async tasks leads to MySQL errors... which lead to failed tasks

csm10495 commented on issue #21078:
URL: https://github.com/apache/airflow/issues/21078#issuecomment-1158016200

   I was able to reproduce this again just now with 2.2.3.
   
   I've kind of contrived configs, etc to hit this... but it has happened in non contrived cases as well. 
   
   Here is the dag that hits it for me every time:
   
   ```
   import random
   import time
   
   from airflow.decorators import dag, task
   from airflow.utils.dates import days_ago
   
   @dag(start_date=days_ago(1), schedule_interval=None, tags=["sample"])
   def sample():
       @task()
       def start():
           return time.sleep(3)
   
       the_start_task = start()
   
       sleepers = []
       for i in range(100):
   
           @task()
           def sleep_10_second():
               return time.sleep(10)
   
           s = sleep_10_second()
           the_start_task >> s
           sleepers.append(s)
   
       sleepers_2 = []
       for i in range(100):
   
           @task()
           def sleep_a_while():
               return time.sleep(random.randint(0, 10))
   
           r = sleep_a_while()
           for s in sleepers:
               s >> r
           sleepers_2.append(r)
   
       sleepers_3 = []
       for i in range(100):
   
           @task()
           def sleep_a_long_time():
               return time.sleep(600)
   
           r = sleep_a_long_time()
           for s in sleepers_2:
               s >> r
           sleepers_3.append(r)
   
   
   sample_dag = sample()
   ```
   
   I don't even actually get out of the first set of 100 tasks without failures. 
   
   Example failures:
   
   ![image](https://user-images.githubusercontent.com/5749838/174125365-fb619f53-12d4-494e-b732-032ed4bf2f6f.png)
   
   a big problem is if you click into one of the failures, the logs are empty:
   
   ![image](https://user-images.githubusercontent.com/5749838/174125463-d4cce229-0927-410c-aa9f-64b342fe38a6.png)
   
   .. the only way to know about the error is to then dive into the scheduler logs... in there you'll find the original error:
   
   ```
   ...
   FROM task_instance INNER JOIN dag_run AS dag_run_1 ON dag_run_1.dag_id = task_instance.dag_id AND dag_run_1.run_id = task_instance.run_id
   WHERE task_instance.dag_id = %s AND task_instance.task_id = %s AND task_instance.run_id = %s
    LIMIT %s FOR UPDATE]
   [parameters: ('sample', 'sleep_10_second__45', 'manual__2022-06-16T16:51:33.917678+00:00', 1)]
   (Background on this error at: http://sqlalche.me/e/13/e3q8)
   [2022-06-16 16:53:04,087: ERROR/ForkPoolWorker-73] Failed to execute task (_mysql_exceptions.OperationalError) (1205, 'Lock wait timeout exceeded; try restarting transaction')
   [SQL: SELECT task_instance.try_number AS task_instance_
   ...
   ERROR [airflow.jobs.scheduler_job.SchedulerJob] Executor reports task instance <TaskInstance: sample.sleep_10_second__45 manual__2022-06-16T16:51:33.917678+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
   ERROR [airflow.models.taskinstance.TaskInstance] Executor reports task instance <TaskInstance: sample.sleep_10_second__45 manual__2022-06-16T16:51:33.917678+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
   ```
   
   Now i've contrived up configs too so here are those special values:
   
   ```
   parallelism = 1000
   max_active_tasks_per_dag  = 1000
   max_active_runs_per_dag = 128
   worker_concurrency = 256
   sql_alchemy_pool_enabled = True
   sql_alchemy_pool_size = 32
   sql_alchemy_pool_recycle = 1800
   sql_alchemy_reconnect_timeout = 300
   ```
   
   The sql alchemy connection is using: `mysql+mysqldb`
   
   Then some stuff from the mysql db:
   
   ```
   mysql> show variables like "max_connections";
   +-----------------+-------+
   | Variable_name   | Value |
   +-----------------+-------+
   | max_connections | 10000 |
   +-----------------+-------+
   1 row in set (0.00 sec)
   
   mysql> SHOW VARIABLES LIKE "%version%";
   +--------------------------+------------------------------+
   | Variable_name            | Value                        |
   +--------------------------+------------------------------+
   | immediate_server_version | 999999                       |
   | innodb_version           | 8.0.18                       |
   | original_server_version  | 999999                       |
   | protocol_version         | 10                           |
   | slave_type_conversions   |                              |
   | tls_version              | TLSv1,TLSv1.1,TLSv1.2        |
   | version                  | 8.0.18                       |
   | version_comment          | MySQL Community Server - GPL |
   | version_compile_machine  | x86_64                       |
   | version_compile_os       | Linux                        |
   | version_compile_zlib     | 1.2.11                       |
   +--------------------------+------------------------------+
   11 rows in set (0.00 sec)
   
   ```
   
   


-- 
This is an automated message from the 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