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/08/17 14:04:19 UTC

[GitHub] [airflow] anthonyp97 opened a new issue, #25765: Deadlock in Scheduler Loop when Updating Dag Run

anthonyp97 opened a new issue, #25765:
URL: https://github.com/apache/airflow/issues/25765

   ### Apache Airflow version
   
   2.3.3
   
   ### What happened
   
   We have been getting occasional deadlock errors in our main scheduler loop that is causing the scheduler to error out of the main scheduler loop and terminate. The full stack trace of the error is below:
   
   ```
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: [2022-08-13 00:01:17,377] {{scheduler_job.py:768}} ERROR - Exception when executing SchedulerJob._run_scheduler_loop
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: Traceback (most recent call last):
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1800, in _execute_context
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     cursor, statement, parameters, context
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/dialects/mysql/mysqldb.py", line 193, in do_executemany
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     rowcount = cursor.executemany(statement, parameters)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 239, in executemany
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self.rowcount = sum(self.execute(query, arg) for arg in args)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 239, in <genexpr>
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self.rowcount = sum(self.execute(query, arg) for arg in args)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 206, in execute
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     res = self._query(query)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 319, in _query
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     db.query(q)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/connections.py", line 259, in query
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     _mysql.connection.query(self, query)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: MySQLdb._exceptions.OperationalError: (1213, 'Deadlock found when trying to get lock; try restarting transaction')
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: The above exception was the direct cause of the following exception:
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: Traceback (most recent call last):
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 751, in _execute
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self._run_scheduler_loop()
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 839, in _run_scheduler_loop
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     num_queued_tis = self._do_scheduling(session)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 924, in _do_scheduling
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     guard.commit()
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/airflow/utils/sqlalchemy.py", line 296, in commit
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self.session.commit()
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1451, in commit
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self._transaction.commit(_to_root=self.future)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 829, in commit
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self._prepare_impl()
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 808, in _prepare_impl
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self.session.flush()
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 3383, in flush
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self._flush(objects)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 3523, in _flush
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     transaction.rollback(_capture_exception=True)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/util/langhelpers.py", line 72, in __exit__
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     with_traceback=exc_tb,
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/util/compat.py", line 208, in raise_
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     raise exception
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 3483, in _flush
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     flush_context.execute()
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/unitofwork.py", line 456, in execute
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     rec.execute(self)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/unitofwork.py", line 633, in execute
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     uow,
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/persistence.py", line 242, in save_obj
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     update,
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/orm/persistence.py", line 1002, in _emit_update_statements
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     statement, multiparams, execution_options=execution_options
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1631, in _execute_20
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     return meth(self, args_10style, kwargs_10style, execution_options)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/sql/elements.py", line 333, in _execute_on_connection
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self, multiparams, params, execution_options
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1508, in _execute_clauseelement
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     cache_hit=cache_hit,
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1863, in _execute_context
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     e, statement, parameters, cursor, context
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2044, in _handle_dbapi_exception
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     sqlalchemy_exception, with_traceback=exc_info[2], from_=e
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/util/compat.py", line 208, in raise_
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     raise exception
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1800, in _execute_context
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     cursor, statement, parameters, context
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/sqlalchemy/dialects/mysql/mysqldb.py", line 193, in do_executemany
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     rowcount = cursor.executemany(statement, parameters)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 239, in executemany
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self.rowcount = sum(self.execute(query, arg) for arg in args)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 239, in <genexpr>
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     self.rowcount = sum(self.execute(query, arg) for arg in args)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 206, in execute
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     res = self._query(query)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/cursors.py", line 319, in _query
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     db.query(q)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:   File "/home/ubuntu/.virtualenvs/ycharts/lib/python3.7/site-packages/MySQLdb/connections.py", line 259, in query
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]:     _mysql.connection.query(self, query)
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: sqlalchemy.exc.OperationalError: (MySQLdb._exceptions.OperationalError) (1213, 'Deadlock found when trying to get lock; try restarting transaction')
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: [SQL: UPDATE dag_run SET last_scheduling_decision=%s WHERE dag_run.id = %s]
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: [parameters: ((datetime.datetime(2022, 8, 13, 0, 1, 17, 280720), 9), (datetime.datetime(2022, 8, 13, 0, 1, 17, 213661), 11), (datetime.datetime(2022, 8, 13, 0, 1, 17, 40686), 12))]
   Aug 13 00:01:17 ip-10-0-2-218 bash[26063]: (Background on this error at: https://sqlalche.me/e/14/e3q8)
   ```
   
   It appears the issue occurs when attempting to update the `last_scheduling_decision` field of the `dag_run` table, but we are unsure why this would cause a deadlock. This issue has only been occurring when we upgrade to version 2.3.3, this was not an issue with version 2.2.4.
   
   ### What you think should happen instead
   
   The scheduler loop should not have any deadlocks that cause it to exit out of its main loop and terminate. I would expect the scheduler loop to always be running constantly, which is not the case if a deadlock occurs in this loop.
   
   ### How to reproduce
   
   This is occurring for us when we run a `LocalExecutor` with smart sensors enabled (2 shards). We only have 3 other daily DAGs which run at different times, and the error seems to occur right when the start time comes for one DAG to start running. After we restart the scheduler after that first deadlock, it seems to run fine the rest of the day, but the next day when it comes time to start the DAG again, another deadlock occurs. 
   
   ### Operating System
   
   Ubuntu 18.04.6 LTS
   
   ### Versions of Apache Airflow Providers
   
   apache-airflow-providers-amazon==4.1.0
   apache-airflow-providers-common-sql==1.0.0
   apache-airflow-providers-ftp==3.1.0
   apache-airflow-providers-http==4.0.0
   apache-airflow-providers-imap==3.0.0
   apache-airflow-providers-mysql==3.1.0
   apache-airflow-providers-sftp==4.0.0
   apache-airflow-providers-sqlite==3.1.0
   apache-airflow-providers-ssh==3.1.0
   
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   We deploy airflow to 2 different ec2 instances. The scheduler lives on one ec2 instances and the webserver lives on a separate ec2 instance. We only run a single scheduler.
   
   ### Anything else
   
   This issue occurs once a day when the first of our daily DAGs gets triggered. When we restart the scheduler after the deadlock, it works fine for the rest of the day typically.
   
   We use a `LocalExecutor` with a `PARALLELISM` of 32, smart sensors enabled using 2 shards.
   
   ### Are you willing to submit PR?
   
   - [ ] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [airflow] eladkal commented on issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   Fixed in https://github.com/apache/airflow/pull/26347


-- 
This is an automated message from the 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] Jakeway commented on issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   I'm not 100% certain, but perhaps this is coming from this PR? https://github.com/apache/airflow/pull/22004/files Looks like the deadlock is happening with the query that's trying to delete old rendered TI fields for mysql (specifically, the `_remove_old_rendered_ti_fields_mysql` method here: https://github.com/apache/airflow/pull/22004/files#diff-85cabe5faa5d3048719e11872b916751e1d0e39b535014be9432d7fd0d101c6dR236)
   
   Apparently it's known this queries in that method could deadlock, but for whatever reason the decorator is not properly catching+retrying in a way that keeps the scheduler marked as healthy.


-- 
This is an automated message from the 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 #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   > This is occurring for us when we run a LocalExecutor with smart sensors enabled (2 shards).
   
   Is this reproducible without Smart Sensors?
   Smart Sensor is removed in 2.4.0 which will be released in few 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


[GitHub] [airflow] anthonyp97 commented on issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   @potiuk @eladkal we have opened a PR with the resolution to this here: https://github.com/apache/airflow/pull/26347


-- 
This is an automated message from the 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 #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   Please search for and provide extract of a detailed log of what happens at the server at the time when deadlock happens. You should be able to find it in your Mysql log - unfortunately, client deadlock does not give enough information on what deadlocks with what. The log will show the detailed query and what query held deadlock 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.

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

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


[GitHub] [airflow] Jakeway commented on issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   @potiuk Hello, here's the latest detected deadlock info:
   
   ```
   ------------------------
   LATEST DETECTED DEADLOCK
   ------------------------
   2022-08-27 00:01:20 0x14a4bc091700
   *** (1) TRANSACTION:
   TRANSACTION 546446888, ACTIVE 0 sec starting index read
   mysql tables in use 1, locked 1
   LOCK WAIT 3 lock struct(s), heap size 1136, 3 row lock(s), undo log entries 2
   MySQL thread id 28342642, OS thread handle 22697787754240, query id 655321967 10.0.9.225 airflow updating
   UPDATE dag_run SET last_scheduling_decision='2022-08-27 00:01:20.418340' WHERE dag_run.id = 68
   *** (1) WAITING FOR THIS LOCK TO BE GRANTED:
   RECORD LOCKS space id 918 page no 3 n bits 144 index PRIMARY of table `airflow`.`dag_run` trx id 546446888 lock_mode X locks rec but not gap waiting
   Record lock, heap no 71 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000044; asc    D;;
    1: len 6; hex 000020921d4d; asc      M;;
    2: len 7; hex 78000002942abb; asc x    * ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 63080d00000000; asc c      ;;
    5: len 7; hex 72756e6e696e67; asc running;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32365430303a30303a3030; asc scheduled__2022-08-26T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: SQL NULL;
    10: len 7; hex 63095e8503519a; asc c ^  Q ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 63095ec402ffea; asc c ^    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 800017b5; asc     ;;
    15: len 7; hex 63095e800dd9e8; asc c ^    ;;
    16: len 7; hex 63080d00000000; asc c      ;;
    17: len 7; hex 63095e80000000; asc c ^    ;;
    18: len 4; hex 80000003; asc     ;;
   
   *** (2) TRANSACTION:
   TRANSACTION 546446886, ACTIVE 0 sec fetching rows
   mysql tables in use 3, locked 3
   7 lock struct(s), heap size 1136, 53 row lock(s)
   MySQL thread id 28349386, OS thread handle 22697761904384, query id 655321958 10.0.9.225 airflow Sending data
   DELETE FROM rendered_task_instance_fields WHERE rendered_task_instance_fields.dag_id = 'pre_systems_execute_daily_dag' AND rendered_task_instance_fields.task_id = 'companies_morn_import_daily_price_and_nav_data_ftp_time_sensor' AND ((rendered_task_instance_fields.dag_id, rendered_task_instance_fields.task_id, rendered_task_instance_fields.run_id) NOT IN (SELECT subq2.dag_id, subq2.task_id, subq2.run_id
   FROM (SELECT subq1.dag_id AS dag_id, subq1.task_id AS task_id, subq1.run_id AS run_id
   FROM (SELECT DISTINCT rendered_task_instance_fields.dag_id AS dag_id, rendered_task_instance_fields.task_id AS task_id, rendered_task_instance_fields.run_id AS run_id, dag_run.execution_date AS execution_date
   FROM rendered_task_instance_fields INNER JOIN dag_run ON rendered_task_instance_fields.dag_id = dag_run.dag_id AND rendered_task_instance_fields.run_id = dag_run.run_id
   WHERE rendered_task_instance_fields.dag
   *** (2) HOLDS THE LOCK(S):
   RECORD LOCKS space id 918 page no 3 n bits 144 index PRIMARY of table `airflow`.`dag_run` trx id 546446886 lock mode S locks rec but not gap
   Record lock, heap no 6 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000005; asc     ;;
    1: len 6; hex 000020049e5a; asc      Z;;
    2: len 7; hex 7a0000021f19b7; asc z      ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f2f500000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32365430303a30303a3030; asc scheduled__2022-08-26T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: SQL NULL;
    10: len 7; hex 63095e8503519a; asc c ^  Q ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 63095ec402ffea; asc c ^    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 800017b5; asc     ;;
    15: len 7; hex 63095e800dd9e8; asc c ^    ;;
    16: len 7; hex 63080d00000000; asc c      ;;
    17: len 7; hex 63095e80000000; asc c ^    ;;
    18: len 4; hex 80000003; asc     ;;
   
   *** (2) TRANSACTION:
   TRANSACTION 546446886, ACTIVE 0 sec fetching rows
   mysql tables in use 3, locked 3
   7 lock struct(s), heap size 1136, 53 row lock(s)
   MySQL thread id 28349386, OS thread handle 22697761904384, query id 655321958 10.0.9.225 airflow Sending data
   DELETE FROM rendered_task_instance_fields WHERE rendered_task_instance_fields.dag_id = 'pre_systems_execute_daily_dag' AND rendered_task_instance_fields.task_id = 'companies_morn_import_daily_price_and_nav_data_ftp_time_sensor' AND ((rendered_task_instance_fields.dag_id, rendered_task_instance_fields.task_id, rendered_task_instance_fields.run_id) NOT IN (SELECT subq2.dag_id, subq2.task_id, subq2.run_id
   FROM (SELECT subq1.dag_id AS dag_id, subq1.task_id AS task_id, subq1.run_id AS run_id
   FROM (SELECT DISTINCT rendered_task_instance_fields.dag_id AS dag_id, rendered_task_instance_fields.task_id AS task_id, rendered_task_instance_fields.run_id AS run_id, dag_run.execution_date AS execution_date
   FROM rendered_task_instance_fields INNER JOIN dag_run ON rendered_task_instance_fields.dag_id = dag_run.dag_id AND rendered_task_instance_fields.run_id = dag_run.run_id
   WHERE rendered_task_instance_fields.dag
   *** (2) HOLDS THE LOCK(S):
   RECORD LOCKS space id 918 page no 3 n bits 144 index PRIMARY of table `airflow`.`dag_run` trx id 546446886 lock mode S locks rec but not gap
   Record lock, heap no 6 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000005; asc     ;;
    1: len 6; hex 000020049e5a; asc      Z;;
    2: len 7; hex 7a0000021f19b7; asc z      ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f2f500000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31305430303a30303a3030; asc scheduled__2022-08-10T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62f4a16007d45d; asc b  `  ];;
    10: len 7; hex 62f46a190803f2; asc b j    ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62f4a16007ba20; asc b  `   ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000001; asc     ;;
    15: len 7; hex 62f46a19075802; asc b j  X ;;
    16: len 7; hex 62f2f500000000; asc b      ;;
    17: len 7; hex 62f44680000000; asc b F    ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 9 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000008; asc     ;;
    1: len 6; hex 0000200e1690; asc       ;;
    2: len 7; hex 35000002a12ff1; asc 5    / ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f44680000000; asc b F    ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31315430303a30303a3030; asc scheduled__2022-08-11T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62f5f3840b4270; asc b    Bp;;
    10: len 7; hex 62f598000a2089; asc b      ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62f5f3840af2b3; asc b      ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000001; asc     ;;
    15: len 7; hex 62f5980008aa7c; asc b     |;;
    16: len 7; hex 62f44680000000; asc b F    ;;
    17: len 7; hex 62f59800000000; asc b      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 13 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 8000000c; asc     ;;
    1: len 6; hex 000020173685; asc     6 ;;
    2: len 7; hex 7f000040131799; asc    @   ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f59800000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31325430303a30303a3030; asc scheduled__2022-08-12T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62f7445a0e9648; asc b DZ  H;;
    10: len 7; hex 62f6e982075128; asc b    Q(;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62f7445a0e34d9; asc b DZ 4 ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000491; asc     ;;
   15: len 7; hex 62f6e9800c27ca; asc b    ' ;;
    16: len 7; hex 62f59800000000; asc b      ;;
    17: len 7; hex 62f6e980000000; asc b      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 17 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000010; asc     ;;
    1: len 6; hex 00002020604f; asc     `O;;
    2: len 7; hex 39000002172658; asc 9    &X;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f6e980000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31335430303a30303a3030; asc scheduled__2022-08-13T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62f88fd3073b81; asc b    ; ;;
    10: len 7; hex 62f83b0007adcb; asc b ;    ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62f88fd3070a41; asc b     A;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 800005a3; asc     ;;
    15: len 7; hex 62f83b00064673; asc b ;  Fs;;
    16: len 7; hex 62f6e980000000; asc b      ;;
    17: len 7; hex 62f83b00000000; asc b ;    ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 21 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000014; asc     ;;
    1: len 6; hex 000020293045; asc    )0E;;
    2: len 7; hex 250000019b0598; asc %      ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f83b00000000; asc b ;    ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31345430303a30303a3030; asc scheduled__2022-08-14T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62f9e155012b50; asc b  U +P;;
    10: len 7; hex 62f98c800b3655; asc b    6U;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62f9e15500d72c; asc b  U  ,;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 8000076c; asc    l;;
    15: len 7; hex 62f98c800832d3; asc b    2 ;;
    16: len 7; hex 62f83b00000000; asc b ;    ;;
    17: len 7; hex 62f98c80000000; asc b      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 25 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000018; asc     ;;
    1: len 6; hex 0000203232c0; asc    22 ;;
    2: len 7; hex 6a000002b71149; asc j     I;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62f98c80000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31355430303a30303a3030; asc scheduled__2022-08-15T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62fb399a013146; asc b 9  1F;;
    10: len 7; hex 62fade00074a75; asc b    Ju;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62fb399a00d1d8; asc b 9    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000852; asc    R;;
    15: len 7; hex 62fade000608a0; asc b      ;;
    16: len 7; hex 62f98c80000000; asc b      ;;
    17: len 7; hex 62fade00000000; asc b      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 29 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 8000001c; asc     ;;
    1: len 6; hex 0000203b4eb4; asc    ;N ;;
    2: len 7; hex 3f0000002d08fe; asc ?   -  ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62fade00000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31365430303a30303a3030; asc scheduled__2022-08-16T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62fc8b1501d93e; asc b     >;;
    10: len 7; hex 62fc2f8008a54d; asc b /   M;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62fc8b1501837b; asc b     {;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 8000094c; asc    L;;
    15: len 7; hex 62fc2f8007f68d; asc b /    ;;
    16: len 7; hex 62fade00000000; asc b      ;;
    17: len 7; hex 62fc2f80000000; asc b /    ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 33 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000020; asc     ;;
    1: len 6; hex 000020446c6f; asc    Dlo;;
    2: len 7; hex 2c000003051592; asc ,      ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
   4: len 7; hex 62fc2f80000000; asc b /    ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31375430303a30303a3030; asc scheduled__2022-08-17T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62fddc9e02e060; asc b     `;;
    10: len 7; hex 62fd810008a510; asc b      ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62fddc9e02a9f3; asc b      ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000ace; asc     ;;
    15: len 7; hex 62fd810001a954; asc b     T;;
    16: len 7; hex 62fc2f80000000; asc b /    ;;
    17: len 7; hex 62fd8100000000; asc b      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 37 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000024; asc    $;;
    1: len 6; hex 0000204d609e; asc    M` ;;
    2: len 7; hex 2a0000401a0976; asc *  @  v;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62fd8100000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31385430303a30303a3030; asc scheduled__2022-08-18T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 62ff2e20084ca2; asc b .  L ;;
    10: len 7; hex 62fed280071b06; asc b      ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 62ff2e200802af; asc b .    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000c7b; asc    {;;
    15: len 7; hex 62fed28005c4d9; asc b      ;;
    16: len 7; hex 62fd8100000000; asc b      ;;
    17: len 7; hex 62fed280000000; asc b      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 41 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000028; asc    (;;
    1: len 6; hex 00002056425d; asc    VB];;
    2: len 7; hex 350000033f0110; asc 5   ?  ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 62fed280000000; asc b      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d31395430303a30303a3030; asc scheduled__2022-08-19T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 63007ed90e0268; asc c ~   h;;
    10: len 7; hex 630024040e7ca2; asc c $  | ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 63007ed90dbac8; asc c ~    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000ded; asc     ;;
    15: len 7; hex 63002401014419; asc c $  D ;;
    16: len 7; hex 62fed280000000; asc b      ;;
    17: len 7; hex 63002400000000; asc c $    ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 45 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 8000002c; asc    ,;;
    1: len 6; hex 0000205f2d59; asc    _-Y;;
    2: len 7; hex 660000025b0cef; asc f   [  ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 63002400000000; asc c $    ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32305430303a30303a3030; asc scheduled__2022-08-20T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 6301ca620d5f6b; asc c  b _k;;
    10: len 7; hex 6301758202efef; asc c u    ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 6301ca620d03cc; asc c  b   ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80000f75; asc    u;;
    15: len 7; hex 630175810685af; asc c u    ;;
    16: len 7; hex 63002400000000; asc c $    ;;
    17: len 7; hex 63017580000000; asc c u    ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 49 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000030; asc    0;;
    1: len 6; hex 00002067cd80; asc    g  ;;
    2: len 7; hex 7f0000030c11a4; asc        ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 63017580000000; asc c u    ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32315430303a30303a3030; asc scheduled__2022-08-21T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 63031be30eb393; asc c      ;;
    10: len 7; hex 6302c70109a0f4; asc c      ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 63031be30e740e; asc c    t ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 8000113a; asc    :;;
    15: len 7; hex 6302c701066dd0; asc c    m ;;
    16: len 7; hex 63017580000000; asc c u    ;;
    17: len 7; hex 6302c700000000; asc c      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 53 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000034; asc    4;;
    1: len 6; hex 00002070a94c; asc    p L;;
    2: len 7; hex 2e0000401a0f88; asc .  @   ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 6302c700000000; asc c      ;;
   5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32325430303a30303a3030; asc scheduled__2022-08-22T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 630473920b3633; asc c s  63;;
    10: len 7; hex 6304188004cc6b; asc c     k;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 630473920adecd; asc c s    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 80001233; asc    3;;
    15: len 7; hex 6304188003dbca; asc c      ;;
    16: len 7; hex 6302c700000000; asc c      ;;
    17: len 7; hex 63041880000000; asc c      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 57 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000038; asc    8;;
    1: len 6; hex 0000207999d2; asc    y  ;;
    2: len 7; hex 30000001a114eb; asc 0      ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 63041880000000; asc c      ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32335430303a30303a3030; asc scheduled__2022-08-23T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 6305c4e80e1283; asc c      ;;
    10: len 7; hex 63056a010096ff; asc c j    ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 6305c4e80dc9eb; asc c      ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 8000139d; asc     ;;
    15: len 7; hex 63056a000daaae; asc c j    ;;
    16: len 7; hex 63041880000000; asc c      ;;
    17: len 7; hex 63056a00000000; asc c j    ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 61 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 8000003c; asc    <;;
    1: len 6; hex 0000208261d6; asc     a ;;
    2: len 7; hex 25000001a1083a; asc %     :;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 63056a00000000; asc c j    ;;
    5: len 7; hex 73756363657373; asc success;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32345430303a30303a3030; asc scheduled__2022-08-24T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: len 7; hex 630716a40b8d47; asc c     G;;
    10: len 7; hex 6306bb800cc8d8; asc c      ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 630716a40b4c13; asc c    L ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 800014c2; asc     ;;
    15: len 7; hex 6306bb800bbc12; asc c      ;;
    16: len 7; hex 6306bb80000000; asc c      ;;
    17: len 7; hex 63080d00000000; asc c      ;;
    18: len 4; hex 80000003; asc     ;;
   
   Record lock, heap no 71 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000044; asc    D;;
    1: len 6; hex 000020921d4d; asc      M;;
    2: len 7; hex 78000002942abb; asc x    * ;;
    3: len 29; hex 7072655f73797374656d735f657865637574655f6461696c795f646167; asc pre_systems_execute_daily_dag;;
    4: len 7; hex 63080d00000000; asc c      ;;
    5: len 7; hex 72756e6e696e67; asc running;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32365430303a30303a3030; asc scheduled__2022-08-26T00:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: SQL NULL;
    10: len 7; hex 63095e8503519a; asc c ^  Q ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 63095ec402ffea; asc c ^    ;;
    13: len 30; hex 326631633932343536356338636261623431663839623137336539386462; asc 2f1c924565c8cbab41f89b173e98db; (total 32 bytes);
    14: len 4; hex 800017b5; asc     ;;
    15: len 7; hex 63095e800dd9e8; asc c ^    ;;
    16: len 7; hex 63080d00000000; asc c      ;;
    17: len 7; hex 63095e80000000; asc c ^    ;;
    18: len 4; hex 80000003; asc     ;;
   
   *** (2) WAITING FOR THIS LOCK TO BE GRANTED:
   RECORD LOCKS space id 918 page no 3 n bits 144 index PRIMARY of table `airflow`.`dag_run` trx id 546446886 lock mode S locks rec but not gap waiting
   Record lock, heap no 67 PHYSICAL RECORD: n_fields 19; compact format; info bits 0
    0: len 4; hex 80000043; asc    C;;
    1: len 6; hex 000020921e28; asc      (;;
    2: len 7; hex 26000002a003db; asc &      ;;
    3: len 30; hex 706f73745f73797374656d735f657865637574655f6461696c795f646167; asc post_systems_execute_daily_dag;;
    4: len 7; hex 63072c00000000; asc c ,    ;;
    5: len 7; hex 72756e6e696e67; asc running;;
    6: len 30; hex 7363686564756c65645f5f323032322d30382d32355430383a30303a3030; asc scheduled__2022-08-25T08:00:00; (total 36 bytes);
    7: len 1; hex 80; asc  ;;
    8: len 5; hex 80047d942e; asc   } .;;
    9: SQL NULL;
    10: len 7; hex 63087d8005db17; asc c }    ;;
    11: len 9; hex 7363686564756c6564; asc scheduled;;
    12: len 7; hex 63095ed00758f8; asc c ^  X ;;
    13: len 30; hex 343733383036626236666630353639313862303331313763323435613166; asc 473806bb6ff056918b03117c245a1f; (total 32 bytes);
    14: len 4; hex 800017b5; asc     ;;
    15: len 7; hex 63087d8005002e; asc c }   .;;
    16: len 7; hex 63072c00000000; asc c ,    ;;
    17: len 7; hex 63087d80000000; asc c }    ;;
    18: len 4; hex 80000003; asc     ;;
   
   *** WE ROLL BACK TRANSACTION (1)
   ------------
   TRANSACTIONS
   ------------
   Trx id counter 546861435
   Purge done for trx's n:o < 546861431 undo n:o < 0 state: running but idle
   History list length 16
   LIST OF TRANSACTIONS FOR EACH SESSION:
   ---TRANSACTION 304173199398320, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ---TRANSACTION 304173199394640, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ---TRANSACTION 304173199395560, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ---TRANSACTION 304173199399240, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ---TRANSACTION 304173199397400, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ---TRANSACTION 304173199400160, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ---TRANSACTION 304173199396480, not started
   0 lock struct(s), heap size 1136, 0 row lock(s)
   ```


-- 
This is an automated message from the 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] anthonyp97 commented on issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   @eladkal we tried turning off smart sensors and the deadlocks actually occurred much more frequently. The traceback for the deadlock error was exactly the same as when we had smart sensors enabled 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] eladkal closed issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

Posted by GitBox <gi...@apache.org>.
eladkal closed issue #25765: Deadlock in Scheduler Loop when Updating Dag Run
URL: https://github.com/apache/airflow/issues/25765


-- 
This is an automated message from the 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] KFoxder commented on issue #25765: Deadlock in Scheduler Loop when Updating Dag Run

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

   @eladkal when you get a chance, do you mind letting us know if you agree that this PR (https://github.com/apache/airflow/pull/22004) is likely the culprit 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