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/09/01 09:31:59 UTC

[GitHub] [airflow] crazyproger opened a new issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

crazyproger opened a new issue #17943:
URL: https://github.com/apache/airflow/issues/17943


   ### Apache Airflow version
   
   2.1.3 (latest released)
   
   ### Operating System
   
   ubuntu 20.04
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Other Docker-based deployment
   
   ### Deployment details
   
   We use mysql 8.
   
   ### What happened
   
   We have 5000 dags. Every day we have 34549 dagruns and 138774 task instances running.
   At the morning we have _hot_ hours when major part of dags starts.
   Yesterday we've updated to 2.1.3 from 1.10.15, and all was worked fine. But today at morning we found that scheduler do not send tasks to celery. After some investigation we've found that scheduler freezes on query from method `_change_state_for_tis_without_dagrun`, query is slow due to fullscan of table. 
   Query was:
   ```
   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.execution_date       AS task_instance_execution_date,
          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
   FROM task_instance
            LEFT OUTER JOIN dag_run ON task_instance.dag_id = dag_run.dag_id AND
                                       task_instance.execution_date = dag_run.execution_date
   WHERE task_instance.dag_id IN
         ('<dag_id>',
          ...
         )
     AND task_instance.state IN ('queued', 'scheduled', 'up_for_reschedule', 'sensing')
     AND (dag_run.state NOT IN ('running', 'queued') OR dag_run.state IS NULL) FOR
   UPDATE OF task_instance SKIP LOCKED
   ```
   
   and `IN` section contained 1812 dag ids.
   Seems that mysql optimizer rollbacks to fullscan with this huge count of dags.
   Since we preserve history for about a month - there is lots of data in our tables.
   
   After we increased `clean_tis_without_dagrun_interval=3600` - problem has disappeared. 
   
   ### What you expected to happen
   
   I think it is possible to process such tasks by smaller chunks to avoid full-scans.
   Also please answer - which impact would we get by increasing clean_tis_without_dagrun_interval to such high value?
   
   ### How to reproduce
   
   _No response_
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [ ] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


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

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

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



[GitHub] [airflow] ephraimbuddy closed issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

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


   


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

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

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #17943:
URL: https://github.com/apache/airflow/issues/17943#issuecomment-909239058


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


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

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

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #17943:
URL: https://github.com/apache/airflow/issues/17943#issuecomment-909239058


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


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

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

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



[GitHub] [airflow] uranusjr commented on issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

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


   I think we need indexes on `DagRun.state` and `TaskInstance.state`.


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

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

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



[GitHub] [airflow] ephraimbuddy commented on issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

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


   We no longer have TIs without dagruns and this method was removed in https://github.com/apache/airflow/pull/17719


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

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

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #17943: _change_state_for_tis_without_dagrun query triggers mysql full scan

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #17943:
URL: https://github.com/apache/airflow/issues/17943#issuecomment-909239058


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


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

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

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