apache / airflow

Apache Airflow - A platform to programmatically author, schedule, and monitor workflows
https://airflow.apache.org/
Apache License 2.0
36.71k stars 14.21k forks source link

DAG with multiple async tasks leads to MySQL errors... which lead to failed tasks #21078

Closed csm10495 closed 2 years ago

csm10495 commented 2 years ago

Apache Airflow version

2.2.2

What happened

We have a DAG that has multiple (10ish) tasks that run at the same time. We are using the celery executor all on one host. A bunch of tasks (that were supposed to run at the same time) failed... though they didn't have any logs for their runs.

Upon some log digging, found that the celery command had failed:

^[[1;31m[2022-01-24 23:16:18,329: ERROR/ForkPoolWorker-32] Task airflow.executors.celery_executor.execute_command[5ef18d0b-4f6f-425e-bf4c-27604f940361] raised unexpected: AirflowException('Celery command failed on host: test-host.test-host.com')^M
Traceback (most recent call last):^M
  File "/python/site-packages/celery/app/trace.py", line 450, in trace_task^M
    R = retval = fun(*args, **kwargs)^M
  File "/python/site-packages/celery/app/trace.py", line 731, in __protected_call__^M
    return self.run(*args, **kwargs)^M
  File "/python/site-packages/airflow/executors/celery_executor.py", line 90, in execute_command^M
    _execute_in_fork(command_to_exec, celery_task_id)^M
  File "/python/site-packages/airflow/executors/celery_executor.py", line 101, in _execute_in_fork^M
    raise AirflowException('Celery command failed on host: ' + get_hostname())^M
airflow.exceptions.AirflowException: Celery command failed on host: test-host.test-host.com^[[0m

Above that we had something like so:

^[[1;31m[2022-01-24 23:16:18,274: ERROR/ForkPoolWorker-32] Failed to execute task (_mysql_exceptions.OperationalError) (1213, 'Deadlock found when trying to get lock; try restarting transaction')^M
[SQL: SELECT task_instance.try_number AS task_instance_try_number, task_instance.task_id AS task_instance_task_id, task_instance.dag_id AS task_instance_dag_id, task_instance.run_id AS task_instance_run_id, task_instance.start_date AS task_instance_start_date, task_instance.end_date AS task_instance_end_date, task_instance.duration AS task_instance_duration, task_instance.state AS task_instance_state, task_instance.max_tries AS task_instance_max_tries, task_instance.hostname AS task_instance_hostname, task_instance.unixname AS task_instance_unixname, task_instance.job_id AS task_instance_job_id, task_instance.pool AS task_instance_pool, task_instance.pool_slots AS task_instance_pool_slots, task_instance.queue AS task_instance_queue, task_instance.priority_weight AS task_instance_priority_weight, task_instance.operator AS task_instance_operator, task_instance.queued_dttm AS task_instance_queued_dttm, task_instance.queued_by_job_id AS task_instance_queued_by_job_id, task_instance.pid AS task_instance_pid, task_instance.executor_config AS task_instance_executor_config, task_instance.external_executor_id AS task_instance_external_executor_id, task_instance.trigger_id AS task_instance_trigger_id, task_instance.trigger_timeout AS task_instance_trigger_timeout, task_instance.next_method AS task_instance_next_method, task_instance.next_kwargs AS task_instance_next_kwargs, dag_run_1.state AS dag_run_1_state, dag_run_1.id AS dag_run_1_id, dag_run_1.dag_id AS dag_run_1_dag_id, dag_run_1.queued_at AS dag_run_1_queued_at, dag_run_1.execution_date AS dag_run_1_execution_date, dag_run_1.start_date AS dag_run_1_start_date, dag_run_1.end_date AS dag_run_1_end_date, dag_run_1.run_id AS dag_run_1_run_id, dag_run_1.creating_job_id AS dag_run_1_creating_job_id, dag_run_1.external_trigger AS dag_run_1_external_trigger, dag_run_1.run_type AS d
ag_run_1_run_type, dag_run_1.conf AS dag_run_1_conf, dag_run_1.data_interval_start AS dag_run_1_data_interval_start, dag_run_1.data_interval_end AS dag_run_1_data_interval_end, dag_run_1.last_scheduling_decision AS dag_run_1_last_scheduling_decision, dag_run_1.dag_hash AS dag_run_1_dag_hash ^M
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 ^M
WHERE task_instance.dag_id = %s AND task_instance.task_id = %s AND task_instance.run_id = %s ^M
 LIMIT %s FOR UPDATE]^M
[parameters: ('parent_dag_for_creation', 'the-task-name', 'manual__2022-01-24T22:31:28.381676+00:00', 1)]^M
(Background on this error at: http://sqlalche.me/e/13/e3q8).^M
Traceback (most recent call last):^M
  File "/python/site-packages/sqlalchemy/engine/base.py", line 1277, in _execute_context^M
    cursor, statement, parameters, context^M
  File "/python/site-packages/sqlalchemy/engine/default.py", line 608, in do_execute^M
    cursor.execute(statement, parameters)^M
  File "/python/site-packages/MySQLdb/cursors.py", line 250, in execute^M
    self.errorhandler(self, exc, value)^M
  File "/python/site-packages/MySQLdb/connections.py", line 50, in defaulterrorhandler^M
    raise errorvalue^M
  File "/python/site-packages/MySQLdb/cursors.py", line 247, in execute^M
    res = self._query(query)^M
  File "/python/site-packages/MySQLdb/cursors.py", line 412, in _query^M
    rowcount = self._do_query(q)^M
  File "/python/site-packages/MySQLdb/cursors.py", line 375, in _do_query^M
    db.query(q)^M
  File "/python/site-packages/MySQLdb/connections.py", line 276, in query^M
    _mysql.connection.query(self, query)^M
_mysql_exceptions.OperationalError: (1213, 'Deadlock found when trying to get lock; try restarting transaction')^M
^M
The above exception was the direct cause of the following exception:^M
^M
Traceback (most recent call last):^M
  File "/python/site-packages/airflow/executors/celery_executor.py", line 121, in _execute_in_fork^M
    args.func(args)^M
  File "/python/site-packages/airflow/cli/cli_parser.py", line 48, in command^M
    return func(*args, **kwargs)^M
  File "/python/site-packages/airflow/utils/cli.py", line 92, in wrapper^M
    return f(*args, **kwargs)^M
  File "/python/site-packages/airflow/cli/commands/task_command.py", line 292, in task_run^M
    _run_task_by_selected_method(args, dag, ti)^M
  File "/python/site-packages/airflow/cli/commands/task_command.py", line 105, in _run_task_by_selected_method^M
    _run_task_by_local_tas
k_job(args, ti)^M
  File "/python/site-packages/airflow/cli/commands/task_command.py", line 163, in _run_task_by_local_task_job^M
    run_job.run()^M
  File "/python/site-packages/airflow/jobs/base_job.py", line 245, in run^M
    self._execute()^M
  File "/python/site-packages/airflow/jobs/local_task_job.py", line 97, in _execute^M
    external_executor_id=self.external_executor_id,^M
  File "/python/site-packages/airflow/utils/session.py", line 70, in wrapper^M
    return func(*args, session=session, **kwargs)^M
  File "/python/site-packages/airflow/models/taskinstance.py", line 1176, in check_and_change_state_before_execution^M
    self.refresh_from_db(session=session, lock_for_update=True)^M
  File "/python/site-packages/airflow/utils/session.py", line 67, in wrapper^M
    return func(*args, **kwargs)^M
  File "/python/site-packages/airflow/models/taskinstance.py", line 729, in refresh_from_db^M
    ti: Optional[TaskInstance] = qry.with_for_update().first()^M
  File "/python/site-packages/sqlalchemy/orm/query.py", line 3429, in first^M
    ret = list(self[0:1])^M
  File "/python/site-packages/sqlalchemy/orm/query.py", line 3203, in __getitem__^M
    return list(res)^M
  File "/python/site-packages/sqlalchemy/orm/query.py", line 3535, in __iter__^M
    return self._execute_and_instances(context)^M
  File "/python/site-packages/sqlalchemy/orm/query.py", line 3560, in _execute_and_instances^M
    result = conn.execute(querycontext.statement, self._params)^M
  File "/python/site-packages/sqlalchemy/engine/base.py", line 1011, in execute^M
    return meth(self, multiparams, params)^M
  File "/python/site-packages/sqlalchemy/sql/elements.py", line 298, in _execute_on_connection^M
    return connection._execute_clauseelement(self, multiparams, params)^M
  File "/python/site-packages/sqlalchemy/engine/base.py", line 1130, in _execute_clauseelement^M
    distilled_params,^M
  File "/python/site-packages/sqlalchemy/engine/base.py", line 1317, in _execute_context^M
    e, statement, parameters, cursor, context^M
  File "/python/site-packages/sqlalchemy/engine/base.py", line 1511, in _handle_dbapi_exception^M
    sqlalchemy_exception, with_traceback=exc_info[2], from_=e^M
  File "/python/site-packages/sqlalchemy/util/compat.py", line 182, in raise_^M
    raise exception^M
  File "/python/site-packages/sqlalchemy/engine/base.py", line 1277, in _execute_context^M
    cursor, statement, parameters, context^M
  File "/python/site-packages/sqlalchemy/engine/default.py", line 608, in do_execute^M
    cursor.execute(statement, parameters)^M
  File "/python/site-packages/MySQLdb/cursors.py", line 250, in execute^M
    self.errorhandler(self, exc, value)^M
  File "/python/site-packages/MySQLdb/connections.py", line 50, in defaulterrorhandler^M
    raise errorvalue^M
  File "/python/site-packages/MySQLdb/cursors.py", line 247, in execute^M
    res = self._query(query)^M
  File "/python/site-packages/MySQLdb/cursors.py", line 412, in _query^M
    rowcount = self._do_query(q)^M
  File "/python/site-packages/MySQLdb/cursors.py", line 375, in _do_query^M
    db.query(q)^M
  File "/python/site-packages/MySQLdb/connections.py", line 276, in query^M
    _mysql.connection.query(self, query)^M
sqlalchemy.exc.OperationalError: (_mysql_exce
ptions.OperationalError) (1213, 'Deadlock found when trying to get lock; try restarting transaction')^M
[SQL: SELECT task_instance.try_number AS task_instance_try_number, task_instance.task_id AS task_instance_task_id, task_instance.dag_id AS task_instance_dag_id, task_instance.run_id AS task_instance_run_id, task_instance.start_date AS task_instance_start_date, task_instance.end_date AS task_instance_end_date, task_instance.duration AS task_instance_duration, task_instance.state AS task_instance_state, task_instance.max_tries AS task_instance_max_tries, task_instance.hostname AS task_instance_hostname, task_instance.unixname AS task_instance_unixname, task_instance.job_id AS task_instance_job_id, task_instance.pool AS task_instance_pool, task_instance.pool_slots AS task_instance_pool_slots, task_instance.queue AS task_instance_queue, task_instance.priority_weight AS task_instance_priority_weight, task_instance.operator AS task_instance_operator, task_instance.queued_dttm AS task_instance_queued_dttm, task_instance.queued_by_job_id AS task_instance_queued_by_job_id, task_instance.pid AS task_instance_pid, task_instance.executor_config AS task_instance_executor_config, task_instance.external_executor_id AS task_instance_external_executor_id, task_instance.trigger_id AS task_instance_trigger_id, task_instance.trigger_timeout AS task_instance_trigger_timeout, task_instance.next_method AS task_instance_next_method, task_instance.next_kwargs AS task_instance_next_kwargs, dag_run_1.state AS dag_run_1_state, dag_run_1.id AS dag_run_1_id, dag_run_1.dag_id AS dag_run_1_dag_id, dag_run_1.queued_at AS dag_run_1_queued_at, dag_run_1.execution_date AS dag_run_1_execution_date, dag_run_1.start_date AS dag_run_1_start_date, dag_run_1.end_date AS dag_run_1_end_date, dag_run_1.run_id AS dag_run_1_run_id, dag_run_1.creating_job_id AS dag_run_1_creating_job_id, dag_run_1.external_trigger AS dag_run_1_external_trigger, dag_run_1.run_type AS dag_run_1_run_type, dag_run_1.conf AS dag_run_1_conf, dag_run_1.data_interval_start AS dag_run
_1_data_interval_start, dag_run_1.data_interval_end AS dag_run_1_data_interval_end, dag_run_1.last_scheduling_decision AS dag_run_1_last_scheduling_decision, dag_run_1.dag_hash AS dag_run_1_dag_hash ^M
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 ^M
WHERE task_instance.dag_id = %s AND task_instance.task_id = %s AND task_instance.run_id = %s ^M
 LIMIT %s FOR UPDATE]^M
[parameters: ('parent_dag_for_creation', 'the-task-name', 'manual__2022-01-24T22:31:28.381676+00:00', 1)]^M
(Background on this error at: http://sqlalche.me/e/13/e3q8)^[[0m

So it kind of seems like an error working with the db happened and it led to the task never actually running and getting marked as failed.

What you expected to happen

If a mysql error like this happens, it should be retried (if appropriate).

At the worst case if the task doesn't start, it should be rescheduled instead of being marked as failure (with no logs).

Optimally any task failure should have a reason in the webui as to why/how it was marked as failure. For this particular one we had to go into the airflow process logs to find any info.

How to reproduce

No response

Operating System

CentOS 7.9.2009

Versions of Apache Airflow Providers

NA

Deployment

Other

Deployment details

PyMySQL=1.0.2 redis=1.7.1 SQLAlchemy=1.3.24 celery=5.1.2 mysqlclient=1.3.13

mysql> SHOW VARIABLES LIKE "%version%";
+--------------------------+------------------------------+
| Variable_name            | Value                        |
+--------------------------+------------------------------+
| admin_tls_version        | TLSv1,TLSv1.1,TLSv1.2        |
| immediate_server_version | 999999                       |
| innodb_version           | 8.0.21                       |
| original_server_version  | 999999                       |
| protocol_version         | 10                           |
| slave_type_conversions   |                              |
| tls_version              | TLSv1,TLSv1.1,TLSv1.2        |
| version                  | 8.0.21                       |
| version_comment          | MySQL Community Server - GPL |
| version_compile_machine  | x86_64                       |
| version_compile_os       | Linux                        |
| version_compile_zlib     | 1.2.11                       |
+--------------------------+------------------------------+

Anything else

No response

Are you willing to submit PR?

Code of Conduct

boring-cyborg[bot] commented 2 years ago

Thanks for opening your first issue here! Be sure to follow the issue template!

csm10495 commented 2 years ago

So it seems like if we have the task's retries set to something > 0 it'll retry and may/may not pass later.

Should it count as a retry if the task didn't even start?

potiuk commented 2 years ago

I think what might be really helpful here is description about the DAGs/operators you had - your issue indicates "async" but nothing in the description points to that,so I wonder what really this all means.

csm10495 commented 2 years ago

So there were several tasks that

         /- Task1
Start 
         \- Task2

         \- Task...n

Each task was just a simple Python Operator doing something external. (Or for testing it can just be a sleep)

This was more easy to reproduce if we set AIRFLOWCOREPARALLELISM and AIRFLOWCOREMAX_ACTIVE_TASKS_PER_DAG to something high like 128 and then had a dag with 200+ parallel tasks. Though we had seen it in a case with neither set (so the fell to defaults).

csm10495 commented 2 years ago

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

a big problem is if you click into one of the failures, the logs are empty:

image

.. 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)
uranusjr commented 2 years ago

Can you post a few more lines in the scheduler log? Your ... part clips the actual failing commands (both sides).

Also from your DAG it seems this is unrelated to async operators?

csm10495 commented 2 years ago

Yeah I have to redo this again and get a fresh log.. I'll try to do that in the coming days

csm10495 commented 2 years ago

Ok, got it to happen again... task with empty log: image

Here is more of a failing log:

coder@rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt:/ecl/apps/airflow/i001 130 $ cat logs/airflow.out | grep -C 100 'sleep_a_while__16 m'
[2022-08-17 17:53:44,658: WARNING/ForkPoolWorker-179]

[2022-08-17 17:53:44,673: WARNING/ForkPoolWorker-164] Running <TaskInstance: sample.sleep_a_while__65 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,673: WARNING/ForkPoolWorker-164]

[2022-08-17 17:53:44,675: WARNING/ForkPoolWorker-157] Running <TaskInstance: sample.sleep_a_while__50 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,675: WARNING/ForkPoolWorker-157]

[2022-08-17 17:53:44,681: WARNING/ForkPoolWorker-182] Running <TaskInstance: sample.sleep_a_while__51 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,682: WARNING/ForkPoolWorker-182]

[2022-08-17 17:53:44,686: WARNING/ForkPoolWorker-163] Running <TaskInstance: sample.sleep_a_while__49 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,686: WARNING/ForkPoolWorker-163]

[2022-08-17 17:53:44,695: DEBUG/ForkPoolWorker-168] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,696: DEBUG/ForkPoolWorker-156] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,700: WARNING/ForkPoolWorker-190] Running <TaskInstance: sample.sleep_a_while__15 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,700: WARNING/ForkPoolWorker-190]

[2022-08-17 17:53:44,700: DEBUG/ForkPoolWorker-167] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,709: DEBUG/ForkPoolWorker-184] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,710: DEBUG/ForkPoolWorker-158] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,716: DEBUG/ForkPoolWorker-175] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,718: DEBUG/ForkPoolWorker-183] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,727: DEBUG/ForkPoolWorker-170] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,733: DEBUG/ForkPoolWorker-185] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,734: WARNING/ForkPoolWorker-156] Running <TaskInstance: sample.sleep_a_while__5 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,735: WARNING/ForkPoolWorker-156]

[2022-08-17 17:53:44,735: WARNING/ForkPoolWorker-168] Running <TaskInstance: sample.sleep_a_while__72 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,735: WARNING/ForkPoolWorker-168]

[2022-08-17 17:53:44,736: WARNING/ForkPoolWorker-167] Running <TaskInstance: sample.sleep_a_while__80 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,736: WARNING/ForkPoolWorker-167]

[2022-08-17 17:53:44,739: WARNING/ForkPoolWorker-158] Running <TaskInstance: sample.sleep_a_while__70 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,739: WARNING/ForkPoolWorker-158]

[2022-08-17 17:53:44,745: WARNING/ForkPoolWorker-184] Running <TaskInstance: sample.sleep_a_while__44 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,745: WARNING/ForkPoolWorker-184]

[2022-08-17 17:53:44,746: DEBUG/ForkPoolWorker-166] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,751: WARNING/ForkPoolWorker-175] Running <TaskInstance: sample.sleep_a_while__81 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,751: WARNING/ForkPoolWorker-175]

[2022-08-17 17:53:44,757: WARNING/ForkPoolWorker-170] Running <TaskInstance: sample.sleep_a_while__30 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,757: WARNING/ForkPoolWorker-170]

[2022-08-17 17:53:44,767: WARNING/ForkPoolWorker-183] Running <TaskInstance: sample.sleep_a_while__96 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,768: WARNING/ForkPoolWorker-183]

[2022-08-17 17:53:44,769: WARNING/ForkPoolWorker-185] Running <TaskInstance: sample.sleep_a_while__3 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,769: WARNING/ForkPoolWorker-185]

[2022-08-17 17:53:44,775: WARNING/ForkPoolWorker-166] Running <TaskInstance: sample.sleep_a_while__56 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,775: WARNING/ForkPoolWorker-166]

[2022-08-17 17:53:44,777: DEBUG/ForkPoolWorker-194] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,785: DEBUG/ForkPoolWorker-193] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,792: DEBUG/ForkPoolWorker-174] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,796: DEBUG/ForkPoolWorker-187] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,811: WARNING/ForkPoolWorker-194] Running <TaskInstance: sample.sleep_a_while__59 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,811: WARNING/ForkPoolWorker-194]

[2022-08-17 17:53:44,811: WARNING/ForkPoolWorker-193] Running <TaskInstance: sample.sleep_a_while__23 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,811: WARNING/ForkPoolWorker-193]

[2022-08-17 17:53:44,826: WARNING/ForkPoolWorker-174] Running <TaskInstance: sample.sleep_a_while__8 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,826: WARNING/ForkPoolWorker-174]

[2022-08-17 17:53:44,826: DEBUG/ForkPoolWorker-181] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,830: WARNING/ForkPoolWorker-187] Running <TaskInstance: sample.sleep_a_while__38 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,830: WARNING/ForkPoolWorker-187]

[2022-08-17 17:53:44,851: DEBUG/ForkPoolWorker-180] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,861: WARNING/ForkPoolWorker-181] Running <TaskInstance: sample.sleep_a_while__95 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,862: WARNING/ForkPoolWorker-181]

[2022-08-17 17:53:44,878: DEBUG/ForkPoolWorker-177] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,879: WARNING/ForkPoolWorker-180] Running <TaskInstance: sample.sleep_a_while__88 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,879: WARNING/ForkPoolWorker-180]

[2022-08-17 17:53:44,886: DEBUG/ForkPoolWorker-176] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,904: DEBUG/ForkPoolWorker-178] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,904: DEBUG/ForkPoolWorker-186] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,908: DEBUG/ForkPoolWorker-188] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,911: WARNING/ForkPoolWorker-177] Running <TaskInstance: sample.sleep_a_while__45 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,911: WARNING/ForkPoolWorker-177]

[2022-08-17 17:53:44,914: WARNING/ForkPoolWorker-176] Running <TaskInstance: sample.sleep_a_while__94 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,915: WARNING/ForkPoolWorker-176]

[2022-08-17 17:53:44,922: DEBUG/ForkPoolWorker-189] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,933: WARNING/ForkPoolWorker-188] Running <TaskInstance: sample.sleep_a_while__87 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,934: WARNING/ForkPoolWorker-188]

[2022-08-17 17:53:44,940: WARNING/ForkPoolWorker-186] Running <TaskInstance: sample.sleep_a_while__73 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,940: WARNING/ForkPoolWorker-186]

[2022-08-17 17:53:44,940: DEBUG/ForkPoolWorker-191] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,941: WARNING/ForkPoolWorker-178] Running <TaskInstance: sample.sleep_a_while__16 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,941: WARNING/ForkPoolWorker-178]

[2022-08-17 17:53:44,946: WARNING/ForkPoolWorker-189] Running <TaskInstance: sample.sleep_a_while__52 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,946: WARNING/ForkPoolWorker-189]

[2022-08-17 17:53:44,949: DEBUG/ForkPoolWorker-192] Loaded DAG <DAG: sample>
[2022-08-17 17:53:44,974: WARNING/ForkPoolWorker-191] Running <TaskInstance: sample.sleep_a_while manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,974: WARNING/ForkPoolWorker-191]

[2022-08-17 17:53:44,980: WARNING/ForkPoolWorker-192] Running <TaskInstance: sample.sleep_a_while__67 manual__2022-08-17T17:52:59.570105+00:00 [queued]> on host rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:53:44,980: WARNING/ForkPoolWorker-192]

[2022-08-17 17:53:48,942: DEBUG/ForkPoolWorker-6] Calling callbacks: []
[2022-08-17 17:53:48,956: INFO/ForkPoolWorker-6] Task airflow.executors.celery_executor.execute_command[5dce92cc-4424-43f7-af01-d7a97fde373f] succeeded in 37.233045257627964s: None
[2022-08-17 17:53:54,709: DEBUG/ForkPoolWorker-10] Calling callbacks: []
[2022-08-17 17:53:54,726: INFO/ForkPoolWorker-10] Task airflow.executors.celery_executor.execute_command[ff329e88-af79-4f1c-9c2d-611560c11ca6] succeeded in 42.97503150999546s: None
2022-08-17T17:54:00.002 [reporter.py:30] [25680] [DEBUG] Counters:
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.dag_processing.processes = 0.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.localtaskjob_end = 6.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.localtaskjob_start = 201.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.operator_successes__PythonDecoratedOperator = 101.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler.orphaned_tasks.adopted = 0.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler.orphaned_tasks.cleared = 0.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler_heartbeat = 161.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.task_instance_created-_PythonDecoratedOperator = 301.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__1.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__10.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__11.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__12.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__13.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__14.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__15.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__16.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__17.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__18.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__19.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__2.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__20.success = 1.0
2022-08-17T17:54:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__21.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__22.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__23.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__24.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__25.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__26.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__27.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__28.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__29.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__3.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__30.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__31.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__32.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__33.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__34.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__35.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__36.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__37.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__38.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__39.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__4.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__40.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__41.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__42.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__43.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__44.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__45.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__46.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__47.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__48.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__49.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__5.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__50.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__51.success = 1.0
2022-08-17T17:54:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__52.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__53.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__54.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__55.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__56.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__57.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__58.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__59.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__6.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__60.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__61.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__62.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__63.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__64.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__65.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__66.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__67.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__68.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__69.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__7.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__70.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__71.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__72.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__73.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__74.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__75.success = 1.0
2022-08-17T17:54:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__76.success = 1.0
--
    return func(*args, session=session, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/models/taskinstance.py", line 1173, in check_and_change_state_before_execution
    self.refresh_from_db(session=session, lock_for_update=True)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/utils/session.py", line 67, in wrapper
    return func(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/models/taskinstance.py", line 726, in refresh_from_db
    ti: Optional[TaskInstance] = qry.with_for_update().first()
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/orm/query.py", line 3429, in first
    ret = list(self[0:1])
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy
/orm/query.py", line 3203, in __getitem__
    return list(res)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/orm/query.py", line 3535, in __iter__
    return self._execute_and_instances(context)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/orm/query.py", line 3560, in _execute_and_instances
    result = conn.execute(querycontext.statement, self._params)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/engine/base.py", line 1011, in execute
    return meth(self, multiparams, params)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/sql/elements.py", line 298, in _execute_on_connection
    return connection._execute_clauseelement(self, multiparams, params)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/engine/base.py", line 1130, in _execute_clauseelement
    distilled_params,
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/engine/base.py", line 1317, in _execute_context
    e, statement, parameters, cursor, context
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/engine/base.py", line 1511, in _handle_dbapi_exception
    sqlalchemy_exception, with_traceback=exc_info[2], from_=e
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/util/com
pat.py", line 182, in raise_
    raise exception
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/engine/base.py", line 1277, in _execute_context
    cursor, statement, parameters, context
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/sqlalchemy/engine/default.py", line 608, in do_execute
    cursor.execute(statement, parameters)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/MySQLdb/cursors.py", line 250, in execute
    self.errorhandler(self, exc, value)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/MySQLdb/connections.py", line 50, in defaulterrorhandler
    raise errorvalue
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/MySQLdb/cursors.py", line 247, in execute
    res = self._query(query)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/MySQLdb/cursors.py", line 412, in _query
    rowcount = self._do_query(q)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/MySQLdb/cursors.py", line 375, in _do_query
    db.query(q)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/MySQLdb/connections.py", line 276, in query
    _mysql.connection.query(self, query)
sqlalchemy.exc.OperationalError: (_mysql_exceptions.OperationalError) (1205, 'Lock wait timeout exceeded; try restarting transaction')
[SQL: SELECT task_in
stance.try_number AS task_instance_try_number, task_instance.task_id AS task_instance_task_id, task_instance.dag_id AS task_instance_dag_id, task_instance.run_id AS task_instance_run_id, task_instance.start_date AS task_instance_start_date, task_instance.end_date AS task_instance_end_date, task_instance.duration AS task_instance_duration, task_instance.state AS task_instance_state, task_instance.max_tries AS task_instance_max_tries, task_instance.hostname AS task_instance_hostname, task_instance.unixname AS task_instance_unixname, task_instance.job_id AS task_instance_job_id, task_instance.pool AS task_instance_pool, task_instance.pool_slots AS task_instance_pool_slots, task_instance.queue AS task_instance_queue, task_instance.priority_weight AS task_instance_priority_weight, task_instance.operator AS task_instance_operator, task_instance.queued_dttm AS task_instance_queued_dttm, task_instance.queued_by_job_id AS task_instance_queued_by_job_id, task_instance.pid AS task_instance_pid, task_instance.executor_config AS task_instance_executor_config, task_instance.external_executor_id AS task_instance_external_executor_id, task_instance.trigger_id AS task_instance_trigger_id, task_instance.trigger_timeout AS task_instance_trigger_timeout, task_instance.next_method AS task_instance_next_method, task_instance.next_kwargs AS task_instance_next_kwargs, dag_run_1.state AS dag_run_1_state, dag_run_1.id AS dag_run_1_id, dag_run_1.dag_id AS dag_run_1_dag_id, dag_run_1.queued_at AS dag_run_1_queued_at, dag_run_1.execution_date AS dag_run_1_execution_date, dag_run_1.start_date AS dag_run_1_start_date, dag_run_1.end_date AS dag_run_1_end_date, dag_run_1.run_id AS dag_run_1_run_id, dag_run_1.creating_job_id AS dag_run_1_creating_job_id, dag_run_1.external_trigger AS dag_run_1_external_trigger, dag_run_1.run_type AS dag_run_1_run_type, dag_run_1.conf AS dag_run_1_conf, dag_run_1.data_interval_start AS dag_run_1_data_interval_start, dag_run_1.data_interval_end AS dag_run_1_data_interval_end, dag_run_1.last_scheduling_decision AS d
ag_run_1_last_scheduling_decision, dag_run_1.dag_hash AS dag_run_1_dag_hash
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_a_while', 'manual__2022-08-17T17:52:59.570105+00:00', 1)]
(Background on this error at: http://sqlalche.me/e/13/e3q8)
[2022-08-17 17:56:46,095: ERROR/ForkPoolWorker-188] Task airflow.executors.celery_executor.execute_command[17c83c4d-01cf-46dc-9c12-8b91d09cabd1] raised unexpected: AirflowException('Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com')
Traceback (most recent call last):
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 450, in trace_task
    R = retval = fun(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 731, in __protected_call__
    return self.run(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 90, in execute_command
    _execute_in_fork(command_to_exec, celery_task_id)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 101, in _execute_in_fork
    raise AirflowException('Celery command failed on host: ' + get_hostname())
airflow.exceptions.AirflowException: Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:56:46,098: ERROR/ForkPoolWorker-178] Task airflow.executors.celery_executor.execute_command[54be8c04-1558-4ba8-937a-a16ab76c58b6] raised unexpected: AirflowException('Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com')
Traceback (most recent call last):
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 450, in trace_task
    R = retval = fun(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 731, in __protected_call__
    return self.run(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 90, in execute_command
    _execute_in_fork(command_to_exec, celery_task_id)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 101, in _execute_in_fork
    raise AirflowException('Celery command failed on host: ' + get_hostname())
airflow.exceptions.AirflowException: Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:56:46,098: ERROR/ForkPoolWorker-186] Task airflow.executors.celery_executor.execute_command[97422978-eeb2-4637-9090-dc10e75b3046] raised unexpected: AirflowException('Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com')
Traceback (most recent call last):
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 450, in trace_task
    R = retval = fun(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 731, in __protected_call__
    return self.run(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 90, in execute_command
    _execute_in_fork(command_to_exec, celery_task_id)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 101, in _execute_in_fork
    raise AirflowException('Celery command failed on host: ' + get_hostname())
airflow.exceptions.AirflowException: Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
[2022-08-17 17:56:46,098: ERROR/ForkPoolWorker-191] Task airflow.executors.celery_executor.execute_command[6ca015ad-a19b-4969-8ae5-e8f8e3f74452] raised unexpected: AirflowException('Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com')
Traceback (most recent call last):
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 450, in trace_task
    R = retval = fun(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/celery/app/trace.py", line 731, in __protected_call__
    return self.run(*args, **kwargs)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 90, in execute_command
    _execute_in_fork(command_to_exec, celery_task_id)
  File "/ecl/apps/airflow/i001/libexec/airflow_38b777dd4dddec01fb240ceb99d1ac5e3c8f64f56c17beb1caa9e864be4b7802/site-packages/airflow/executors/celery_executor.py", line 101, in _execute_in_fork
    raise AirflowException('Celery command failed on host: ' + get_hostname())
airflow.exceptions.AirflowException: Celery command failed on host: rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt.corp.rdev.svc.kube.corp.company.com
ERROR [airflow.jobs.scheduler_job.SchedulerJob] Executor reports task instance <TaskInstance: sample.sleep_a_while manual__2022-08-17T17:52:59.570105+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_a_while manual__2022-08-17T17:52:59.570105+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
ERROR [airflow.jobs.scheduler_job.SchedulerJob] Executor reports task instance <TaskInstance: sample.sleep_a_while__16 manual__2022-08-17T17:52:59.570105+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_a_while__16 manual__2022-08-17T17:52:59.570105+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
ERROR [airflow.jobs.scheduler_job.SchedulerJob] Executor reports task instance <TaskInstance: sample.sleep_a_while__73 manual__2022-08-17T17:52:59.570105+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_a_while__73 manual__2022-08-17T17:52:59.570105+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
ERROR [airflow.jobs.scheduler_job.SchedulerJob] Executor reports task instance <TaskInstance: sample.sleep_a_while__87 manual__2022-08-17T17:52:59.570105+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_a_while__87 manual__2022-08-17T17:52:59.570105+00:00 [queued]> finished (failed) although the task says its queued. (Info: None) Was the task killed externally?
[2022-08-17 17:56:50,091: DEBUG/ForkPoolWorker-118] Calling callbacks: []
[2022-08-17 17:56:50,105: INFO/ForkPoolWorker-118] Task airflow.executors.celery_executor.execute_command[825e740c-7da6-42d2-b5f4-f374871fb12a] succeeded in 187.26130820810795s: None
[2022-08-17 17:56:53,703: DEBUG/ForkPoolWorker-5] Calling callbacks: []
[2022-08-17 17:56:53,716: INFO/ForkPoolWorker-5] Task airflow.executors.celery_executor.execute_command[e9d5ca5f-6c57-4585-92b1-99845e256956] succeeded in 190.98830308765173s: None
[2022-08-17 17:56:57,324: DEBUG/ForkPoolWorker-144] Calling callbacks: []
[2022-08-17 17:56:57,338: INFO/ForkPoolWorker-144] Task airflow.executors.celery_executor.execute_command[55861144-b295-4278-aeac-a7062d2451fe] succeeded in 194.33598389476538s: None
2022-08-17T17:57:00.002 [reporter.py:30] [25680] [DEBUG] Counters:
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.dag_processing.processes = 0.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.localtaskjob_end = 104.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.localtaskjob_start = 201.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.operator_failures__PythonDecoratedOperator = 4.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.operator_successes__PythonDecoratedOperator = 197.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler.orphaned_tasks.adopted = 0.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler.orphaned_tasks.cleared = 0.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler.tasks.killed_externally = 4.0
2022-08-17T17:57:00.002 [reporter.py:32] [25680] [DEBUG]   airflow.scheduler_heartbeat = 197.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.task_instance_created-_PythonDecoratedOperator = 301.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__1.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__10.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__11.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__12.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__13.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__14.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__15.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__16.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__17.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__18.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__19.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__2.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__20.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__21.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__22.success = 1.0
2022-08-17T17:57:00.003 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__23.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__24.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__25.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__26.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__27.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__28.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__29.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__3.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__30.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__31.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__32.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__33.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__34.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__35.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__36.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__37.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__38.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__39.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__4.success = 1.0
2022-08-17T17:57:00.004 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__40.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__41.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__42.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__43.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__44.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__45.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__46.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__47.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__48.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__49.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__5.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__50.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__51.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__52.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__53.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__54.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__55.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__56.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__57.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__58.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__59.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__6.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__60.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__61.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__62.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__63.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__64.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__65.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__66.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__67.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__68.success = 1.0
2022-08-17T17:57:00.005 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__69.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__7.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__70.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__71.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__72.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__73.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__74.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__75.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__76.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__77.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__78.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__79.success = 1.0
2022-08-17T17:57:00.006 [reporter.py:32] [25680] [DEBUG]   airflow.ti.finish.sample.sleep_10_second__8.success = 1.0
coder@rdev-fff1b45c-8295-48b4-bcbc-adbe279ded98-zn4xt:/ecl/apps/airflow/i001 $
potiuk commented 2 years ago

We need detailed log from the mysql server from around that time deadlock happens. without it it is next to impossible to find the root cause - the server log will contain detailed information what deadlocs with what.

csm10495 commented 2 years ago

Attaching (big) logs for a similar failure: airflow_sql_log_8_29_2022.log.gz

airflow.out_8_29_2022.txt

This is both the airflow output and the mysql server's general log (with queries).

No logs in webui: image

I tend to grep around for __63 to find corresponding logs for the failing task

potiuk commented 2 years ago

Umfortunately there is no clear indication from the logs and it's also hard to correlate them - if you could help with narrowing down and selecting much smaller set of queries corresponding to the actual problem, that could probably help - otherwise siimply looking at your log and trying to find out and spend time (in the free time we have where we try to help people here) is just too much of an investmen (you have to remember it is a free software and any help you get here is done in the time we can spare outside of the normal time of hours). If you could help to narrow it down even more then trying to help you wil be more efficitent.

From the look I had in the time I could spare, I think the problem is that you might be hitting the limits of query size to be issued by the server - simply your server is not able to cope with the queries that Airlfow is sending to it.

What I could recommend you to look at some of the configuration parameters of scheduler and tune down some of the values there and observe if they help: https://airflow.apache.org/docs/apache-airflow/stable/concepts/scheduler.html#scheduler-configuration-options

The chapter explains in general the fine tuning options you can choose and how they impact scheduler: https://airflow.apache.org/docs/apache-airflow/stable/concepts/scheduler.html#fine-tuning-your-scheduler-performance

Unfortunately there are no "ready recipes" what to do and what to configure - because depending on your MySQL (multitude) of settings there are various limits you can start hitting and there is no clear error that shows up in the logs to indicate what it is. However decreasing down some of the "max values" that scheduler is going to process in single loop for example might improve that a lot - and it would be great to hear if it does help when you try it. Just make sure to experiment one setting at a time.

Another general advice I'd have - if you only can, switch to Postgres. We have > 80% of users using Postrges https://airflow.apache.org/blog/airflow-survey-2022/ and possbly 90% of problems (similar to yours) with stability are coming from MySQL (so you can see that stability of Postgres is likely a few orders of magnitude better than that of mySQL). I think if you are looking for a quick solution to your problems, this might be simply most pragmatic and quickest approach.

csm10495 commented 2 years ago

Part of the problem here is the original problem is pretty difficult to reproduce... we didn't hit it continually, just was a weird once in a while thing.

Changing these settings to the super high values are just to reproduce the issue more consistently. I'm not really worried about the failure as much as no logs coming up in the web ui for certain task failures.

From the code logic, if the db is in limbo, I can see how it would be tough to still ensure logs come out.

Maybe a middle ground would be to have the UI tell the user to ask their administrator to check the scheduler logs around the task start time in the event of a completed task with no logs at all... since any info is likely there. That would be much better than just a blank page.

In terms of Postgres, I'll ask but I don't think its really swap-able in our current production environment.

I'll just close this out.