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

[GitHub] [airflow] Dundo777 opened a new issue #13529: Scheduler ends up in Deadlock and marks task as failed

Dundo777 opened a new issue #13529:
URL: https://github.com/apache/airflow/issues/13529


   **Apache Airflow version**:
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**:
   - **OS** (e.g. from /etc/os-release): CentOS 7
   - **Kernel** (e.g. `uname -a`): x86_64
   - **Install tools**: yum, pip
   - **Others**: MariaDB 10.2, Python 3.6.8, RabbitMQ 3.3.5
   
   **What happened**:
   
   After upgrading to Airflow 2.0.0, scheduler ends up in a deadlock when running a DAG. 
   When I trigger a DAG (or just run the task) scheduler adds the task to the queue, worker picks up the task and successfully executes it but eventually scheduler ends up in a deadlock and marks run as failed.
   In database there are no deadlocks and everything seems to be working fine.
   
   <details>
   <summary>
   Scheduler logs
   </summary>
   Jan 07 07:43:10 host bash[9738]: [2021-01-07 07:43:10,882] {scheduler_job.py:938} INFO - 1 tasks up for execution:
   Jan 07 07:43:10 host  bash[9738]: <TaskInstance: dbimport.start 2021-01-07 06:43:10.874823+00:00 [scheduled]>
   Jan 07 07:43:10 host  bash[9738]: [2021-01-07 07:43:10,885] {scheduler_job.py:972} INFO - Figuring out tasks to run in Pool(name=dbimport) with 24 open slots and 1 task instances ready to be queued
   Jan 07 07:43:10 host  bash[9738]: [2021-01-07 07:43:10,885] {scheduler_job.py:999} INFO - DAG dbimport has 0/4 running and queued tasks
   Jan 07 07:43:10host  bash[9738]: [2021-01-07 07:43:10,885] {scheduler_job.py:1060} INFO - Setting the following tasks to queued state:
   Jan 07 07:43:10 host  bash[9738]: <TaskInstance: dbimport.start 2021-01-07 06:43:10.874823+00:00 [scheduled]>
   Jan 07 07:43:10 host  bash[9738]: [2021-01-07 07:43:10,888] {scheduler_job.py:1102} INFO - Sending TaskInstanceKey(dag_id='dbimport', task_id='start', execution_date=datetime.datetime(2021, 1, 7, 6, 43, 10, 874823, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 36 and queue default
   Jan 07 07:43:10 host  bash[9738]: [2021-01-07 07:43:10,888] {base_executor.py:79} INFO - Adding to queue: ['airflow', 'tasks', 'run', 'dbimport', 'start', '2021-01-07T06:43:10.874823+00:00', '--local', '--pool', 'dbimport', '--subdir', '/usr/local/airflow/dags/bigdata-internal-poc/dbimport.py']
   Jan 07 07:43:10 host  bash[9738]: [2021-01-07 07:43:10,922] {scheduler_job.py:1200} INFO - Executor reports execution of dbimport.start execution_date=2021-01-07 06:43:10.874823+00:00 exited with status queued for try_number 1
   Jan 07 07:43:10 host  bash[9738]: [2021-01-07 07:43:10,955] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:11 host  bash[9738]: [2021-01-07 07:43:11,994] {scheduler_job.py:1671} INFO - DAG dbimport already has 2 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:13 host  bash[9738]: [2021-01-07 07:43:13,041] {scheduler_job.py:1671} INFO - DAG dbimport already has 2 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:14 host  bash[9738]: [2021-01-07 07:43:14,087] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:14 host  bash[9738]: [2021-01-07 07:43:14,102] {scheduler_job.py:1200} INFO - Executor reports execution of dbimport.start execution_date=2021-01-07 06:43:10.874823+00:00 exited with status success for try_number 1
   Jan 07 07:43:14 host  bash[9738]: [2021-01-07 07:43:14,132] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:15 host  bash[9738]: [2021-01-07 07:43:15,169] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:15 host  bash[9738]: [2021-01-07 07:43:15,956] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:17 host  bash[9738]: [2021-01-07 07:43:17,001] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:18 host  bash[9738]: [2021-01-07 07:43:18,032] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:19 host  bash[9738]: [2021-01-07 07:43:19,062] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:19 host  bash[9738]: [2021-01-07 07:43:19,854] {scheduler_job.py:1671} INFO - DAG dbimport already has 1 active runs, not queuing any tasks for run 2021-01-07 06:43:09.837529+00:00
   Jan 07 07:43:19 host  bash[9738]: [2021-01-07 07:43:19,878] {scheduler_job.py:853} WARNING - Set 1 task instances to state=None as their associated DagRun was not in RUNNING state
   Jan 07 07:43:20 host  bash[9738]: [2021-01-07 07:43:20,920] {dagrun.py:459} ERROR - Deadlock; marking run <DagRun dbimport @ 2021-01-07 06:43:09.837529+00:00: manual__2021-01-07T06:43:09.837529+00:00, externally triggered: True> failed
   </details>
   
   <details>
   <summary>
   Worker logs
   </summary>
   Jan 07 07:43:10 host  bash[8564]: [2021-01-07 07:43:10,935: INFO/MainProcess] Received task: airflow.executors.celery_executor.execute_command[6bc6dc4e-662e-4e4c-89af-50f070e21d9c]
   Jan 07 07:43:11 host  bash[8564]: [2021-01-07 07:43:11,077: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dbimport', 'start', '2021-01-07T06:43:10.874823+00:00', '--local', '--pool', 'dbimport', '--subdir', '/usr/local/airflow/dags/bigdata-internal-poc/dbimport.py']
   Jan 07 07:43:11host  bash[8564]: [2021-01-07 07:43:11,198: INFO/ForkPoolWorker-8] Filling up the DagBag from /usr/local/airflow/dags/bigdata-internal-poc/dbimport.py
   Jan 07 07:43:11 host  bash[8564]: [2021-01-07 07:43:11,543: WARNING/ForkPoolWorker-8] Running <TaskInstance: dbimport.start 2021-01-07T06:43:10.874823+00:00 [None]> on host analytics-dn3.lan.croz.net
   Jan 07 07:43:11 host  sudo[12294]:  airflow : TTY=unknown ; PWD=/tmp/airflowtmp258n1mmv ; USER=dbimport ; COMMAND=/bin/bash -c /usr/local/dbimport/bin/manage --checkAirflowExecution
   Jan 07 07:43:13 host  bash[8564]: [2021-01-07 07:43:13,431: INFO/ForkPoolWorker-8] Task airflow.executors.celery_executor.execute_command[6bc6dc4e-662e-4e4c-89af-50f070e21d9c] succeeded in 2.466554159997031s: None
   </details>
   
   **What you expected to happen**:
   
   Scheduler should mark task as succeeded and not end up in a deadlock.
   
   **How to reproduce it**:
   
   Upgrade from Airflow 1.10.14 to 2.0.0, with MariaDB 10.2, trigger the DAG or task.
   
   **Anything else we need to know**:
   
   Problem occurs every time a DAG or task is triggered. This is critical because now we can't run anything.
   


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   I think I found what might be problem.
   In scheduler_job.py around line number 1210 "#Check state of finished tasks" it queries task_instances table but with wrong filter parameter. Time is one hour behind. In task_instances table I see the dag run with success state and execution date e.g. 14 o'clock and filter in scheduler_job.py is filtering for 13 o'clock. and 14 o'clock is the correct time, so time that is written in task_instances table is correct. The filter time is wrong.
   But I didn't yet figure out where that date is setup. Maybe some of you will know that by heart? :)
   
   In airflow.cfg default_timezone = Europe/Zagreb and that's consistent with the task_instances execution_date time.


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   I think I found what might be problem.
   In scheduler_job.py around line number 1210 "#Check state of finished tasks" it queries task_instances table but with wrong filter parameter. Time is one hour behind. In task_instances table I see the dag run with success state and execution date e.g. 14 o'clock and filter in scheduler_job.py is filtering for 13 o'clock. and 14 o'clock is the correct time, so time that is written in task_instances table is correct. The filter time is wrong.
   But I didn't yet figure out where that date is setup. Maybe some of you will know that by heart? :)
   
   In airflow.cfg default_timezone = Europe/Zagreb and that's consistent with the task_instances execution_date time.


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

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



[GitHub] [airflow] Dundo777 edited a comment on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   The problem is related to my previous comment that timestamps that are being compared are different.
   In task_instance table default value of execution_date is current_timestamp(6) with on update current_timestamp(6).
   Since my timezone in MariaDB is setup to SYSTEM which is set to Europe/Zagreb it will save timestamp in my local time instead of utc.
   When I altered table to default utc_timestamp(6) everything is working fine. Another option is to change the database timezone.
   This means that it is assumed that metadata db timezone will be utc.
   
   Since this is not mentioned anywhere in the documentation, I'm wondering if this is expected behaviour? And it looks like there should be option to change that or insist on utc for database backend.


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

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



[GitHub] [airflow] kaxil commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Are you running just a single Scheduler?
   
   Mariadb is not officially supported, but since you said you were able to replicate it with MySQL 5.7 -- can you give us steps to reproduce with MySQL


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   The problem is related to my previous comment that timestamps that are being compared are different.
   In task_instance table default value of execution_date is current_timestamp(6) with on update current_timestamp(6).
   Since my timezone in MariaDB is setup to SYSTEM which is set to Europe/Zagreb it will save timestamp in my local time instead of utc.
   When I altered table to default utc_timestamp(6) on update utc_timestamp(6) everything is working fine. Another option is to change the database timezone.
   This means that it is assumed that metadata db timezone will be utc.
   
   Since this is not mentioned anywhere in the documentation, I'm wondering if this is expected behaviour? And it looks like there should be option to change that or insist on utc for database backend.


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Now I tried to da a fresh installation of MySQL 5.7 and migrated the data from MariaDB 10.2. Everything is the same as on MariaDB 10.2. All services are up and running but when running the dag the same deadlock error happens


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

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



[GitHub] [airflow] kaxil edited a comment on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Actually, we have mentioned it in a couple of places but we can make it more clearer I think:
   
   https://airflow.apache.org/docs/apache-airflow/stable/timezone.html#time-zones
   https://airflow.apache.org/docs/apache-airflow/stable/dag-run.html#external-triggers
   
   Would you like to make a PR to make it clearer in our docs in https://airflow.apache.org/docs/apache-airflow/stable/installation.html#requirements ?
   
   
   
   >I found this issue that looks the same as this one #1015
   
   We moved to JIRA in 2016/2017 and moved from Jira back to Github Issue last year -- so some of the issues got closed in that migrations.


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Yes, I'm running a single scheduler, multiple workers, CeleryExecutor, setup with RabbitMQ.
   To move from MariaDB 10.2 to MySQL 5.7 I did the following:
   - backup the MariaDB database using mysqldump command
   - made a new installation of MySQL 5.7 and migrated backed up data into it
   - switched Airflow to using MySQL by changing airflow.cfg to MySQL, both sql_alchemy_conn and Celery result_backend
   


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

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



[GitHub] [airflow] kaxil edited a comment on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Actually, we have mentioned it in a couple of places but we can make it more clearer I think:
   
   https://airflow.apache.org/docs/apache-airflow/stable/timezone.html#time-zones
   https://airflow.apache.org/docs/apache-airflow/stable/dag-run.html#external-triggers
   
   Would you like to make a PR to make it clearer in our docs in https://airflow.apache.org/docs/apache-airflow/stable/installation.html#requirements ?
   
   
   
   >I found this issue that looks the same as this one #1015
   
   We moved to JIRA in 2016/2017 and moved from Jira back to Github Issue last year -- so some of the issues got closed in that migrations.
   
   
   ---- 
   
   Separately can you also list steps to reproduce this in a clean environment -- So that we can fix this in source code if there is an issue. I only know few places where we use `utcnow()` -- However we use pendulum for datetime and Scheduler uses timezone aware dates -- so comparing them shouldn't be a problem.
   
   cc @ashb 


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

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



[GitHub] [airflow] Dundo777 edited a comment on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   I think I found what might be problem.
   In scheduler_job.py around line number 1210 "#Check state of finished tasks" it queries task_instances table but with wrong filter parameter. Time is one hour behind. In task_instances table I see the dag run with success state and execution date e.g. 14 o'clock and filter in scheduler_job.py is filtering for 13 o'clock. and 14 o'clock is the correct time, so time that is written in task_instances table is correct. The filter time is wrong.
   But I didn't yet figure out where that date is setup. Maybe some of you will know that by heart? :)
   
   In airflow.cfg default_timezone = Europe/Zagreb and that's consistent with the task_instances execution_date time.
   
   Even though, AFAIK Airflow should store everything in utc internally which means that the time saved in database is not correct, right?


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   I prepared a docker compose with airflow master image and mysql 5.7 configured to Europe/Zagreb timezone. And everything is working fine. 
   
   In this fresh Airflow installation task_instance table doesn't have default value setup for execution_date column.
   So looking at my migration scripts I noticed that in mysql dump sql script I have current_timestamp(6) as default value for execution_date in task_instance table.
   When I started upgrade to Airflow 2.0.0 I started from version 1.10.2. Is it possible that in some previous version of Airflow default timestamp value was setup but was later removed? Because we definitely didn't set that default value :)
   
   It would be good to check if this current_timestamp default value was setup in older versions of Airflow so that anyone that goes through same upgrade path knows what to look out for :) 
   Since it affects couple of tables and columns:
   - dag_code.last_updated
   - rendered_task_instance_fields.execution_date
   - sensor_instance.execution_date
   
   I attached default_timestamp_patch.sql inside of a zip that may be added to some patch or useful to someone that stumbles upon this issue. [default_timestamp_patch.zip](https://github.com/apache/airflow/files/5801813/default_timestamp_patch.zip)
   
   Regarding documentation, from the issue I was facing I got impression that database needs to be setup with UTC as default timezone and now with this fresh installation it's clear that that's not the case. So no need to do any changes in documentation, 
   
   As far as I'm concerned we can close this issue. Thanks for your help.
   
   
   
   


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

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



[GitHub] [airflow] kaxil edited a comment on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Actually, we have mentioned it in a couple of places but we can make it more clearer I think:
   
   https://airflow.apache.org/docs/apache-airflow/stable/timezone.html#time-zones
   https://airflow.apache.org/docs/apache-airflow/stable/dag-run.html#external-triggers
   
   Would you like to make a PR to make it clearer in our docs in https://airflow.apache.org/docs/apache-airflow/stable/installation.html#requirements ?
   
   
   
   >I found this issue that looks the same as this one #1015
   
   We moved to JIRA in 2016/2017 and moved from Jira back to Github Issue last year -- so some of the issues got closed in that migrations.
   
   
   ---- 
   
   Separately can you also list steps to reproduce this in a clean environment -- So that we can fix this in source code if there is an issue. I only know few places where we use `utcnow()` or `func.now` (which might return time based on DB's TZ) -- However we use pendulum for datetime and Scheduler uses timezone aware dates -- so comparing them shouldn't be a problem.
   
   cc @ashb 


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

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



[GitHub] [airflow] Dundo777 closed issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   


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

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



[GitHub] [airflow] kaxil commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   Actually, we have mentioned it in a couple of places but we can make it more clearer I think:
   
   https://airflow.apache.org/docs/apache-airflow/stable/timezone.html#time-zones
   https://airflow.apache.org/docs/apache-airflow/stable/dag-run.html#external-triggers
   
   Would you like to make a PR to make it clearer in our docs in https://airflow.apache.org/docs/apache-airflow/stable/installation.html#requirements


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

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



[GitHub] [airflow] Dundo777 edited a comment on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   I think I found what might be problem.
   In scheduler_job.py around line number 1210 "#Check state of finished tasks" it queries task_instances table but with wrong filter parameter. Time is one hour behind. In task_instances table I see the dag run with success state and execution date e.g. 14 o'clock and filter in scheduler_job.py is filtering for 13 o'clock. and 14 o'clock is the correct time, so time that is written in task_instances table is correct. The filter time is wrong.
   But I didn't yet figure out where that date is setup. Maybe some of you will know that by heart? :)
   
   In airflow.cfg default_timezone = Europe/Zagreb and that's consistent with the task_instances execution_date time.
   
   Even though, AFAIK Airflow should store everything in utc internally which means that the time saved in database is not correct, right?


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

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



[GitHub] [airflow] Dundo777 commented on issue #13529: Scheduler ends up in a deadlock and marks task as failed

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


   I found this issue that looks the same as this one #1015 
   @kaxil you closed that issue last year in March, was there any conclusion about it?


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

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