You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/03/30 17:08:31 UTC

[GitHub] [airflow] tronlightracer opened a new issue #22626: The command 'airflow dags test' is deadlocking tasks using AIP-42 features with the PythonVirtualenvOperator

tronlightracer opened a new issue #22626:
URL: https://github.com/apache/airflow/issues/22626


   ### Apache Airflow version
   
   main (development)
   
   ### What happened
   
   When using the command airflow dags test tasks are put in the deadlock state.
   
   ### What you think should happen instead
   
   I think the airflow dags test shouldn't deadlock tasks.
   
   ### How to reproduce
   
   from airflow.models import DAG
   from airflow.decorators import task
   from airflow.operators.python import PythonOperator, PythonVirtualenvOperator
   from datetime import datetime, timedelta
   
   
   def dynamic_importer(module1, module2, module3):
       import importlib
       for module in [module1, module2, module3]:
           print(module)
           importlib.import_module(module)
           
   
   def dynamic_imports(imports: list):
       from random import choices
       return [choices(imports, k=3) for _ in range(10)]
   
   the_imports = ["flask", "flask_restful", "wheel", "click", "numpy", "cryptography", "pandas"]
   
   with DAG(
       dag_id="test_mapped_virtualenv",
       start_date=datetime(1970, 1, 1),
       schedule_interval=timedelta(days=30 * 365),
   ) as dag:
       PythonVirtualenvOperator.partial(
           task_id="venv_experiments",
           python_callable=dynamic_importer,
           requirements=the_imports,
           system_site_packages=False,
       ).expand(op_args=dynamic_imports(the_imports))
   
   
   [2022-03-30 09:15:32,872] {taskinstance.py:850} DEBUG - Setting task state for <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [None]> to scheduled
   [2022-03-30 09:15:32,885] {backfill_job.py:405} DEBUG - *** Clearing out not_ready list ***
   [2022-03-30 09:15:32,890] {taskinstance.py:759} DEBUG - Refreshing TaskInstance <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> from DB
   [2022-03-30 09:15:32,895] {backfill_job.py:418} DEBUG - Task instance to run <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> state scheduled
   [2022-03-30 09:15:32,896] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Task Instance State' PASSED: True, Task state scheduled was valid.
   [2022-03-30 09:15:32,896] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Trigger Rule' PASSED: True, The task instance did not have any upstream tasks.
   [2022-03-30 09:15:32,896] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Task Instance Not Running' PASSED: True, Task is not in running state.
   [2022-03-30 09:15:32,896] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
   [2022-03-30 09:15:32,897] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Previous Dagrun State' PASSED: True, The context specified that the state of past DAGs could be ignored.
   [2022-03-30 09:15:32,897] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Task has been mapped' PASSED: False, The task has yet to be mapped!
   [2022-03-30 09:15:32,897] {taskinstance.py:1041} DEBUG - Dependencies not met for <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]>, dependency 'Task has been mapped' FAILED: The task has yet to be mapped!
   [2022-03-30 09:15:32,897] {backfill_job.py:536} DEBUG - Adding <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> to not_ready
   [2022-03-30 09:15:37,847] {base_job.py:226} DEBUG - [heartbeat]
   [2022-03-30 09:15:37,847] {base_executor.py:156} DEBUG - 0 running task instances
   [2022-03-30 09:15:37,848] {base_executor.py:157} DEBUG - 0 in queue
   [2022-03-30 09:15:37,848] {base_executor.py:158} DEBUG - 32 open slots
   [2022-03-30 09:15:37,848] {base_executor.py:167} DEBUG - Calling the <class 'airflow.executors.debug_executor.DebugExecutor'> sync method
   [2022-03-30 09:15:37,848] {backfill_job.py:596} WARNING - Deadlock discovered for ti_status.to_run=dict_values([<TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]>])
   [2022-03-30 09:15:37,853] {dagrun.py:628} DEBUG - number of tis tasks for <DagRun test_mapped_virtualenv @ 2022-03-30T09:15:31+00:00: backfill__2022-03-30T09:15:31+00:00, externally triggered: False>: 1 task(s)
   [2022-03-30 09:15:37,853] {dagrun.py:644} DEBUG - number of scheduleable tasks for <DagRun test_mapped_virtualenv @ 2022-03-30T09:15:31+00:00: backfill__2022-03-30T09:15:31+00:00, externally triggered: False>: 0 task(s)
   [2022-03-30 09:15:37,853] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Task has been mapped' PASSED: False, The task has yet to be mapped!
   [2022-03-30 09:15:37,854] {taskinstance.py:1041} DEBUG - Dependencies not met for <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]>, dependency 'Task has been mapped' FAILED: The task has yet to be mapped!
   [2022-03-30 09:15:37,854] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Not In Retry Period' PASSED: True, The context specified that being in a retry period was permitted.
   [2022-03-30 09:15:37,854] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Previous Dagrun State' PASSED: True, The task did not have depends_on_past set.
   [2022-03-30 09:15:37,854] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Trigger Rule' PASSED: True, The task instance did not have any upstream tasks.
   [2022-03-30 09:15:37,854] {dagrun.py:570} ERROR - Deadlock; marking run <DagRun test_mapped_virtualenv @ 2022-03-30T09:15:31+00:00: backfill__2022-03-30T09:15:31+00:00, externally triggered: False> failed
   [2022-03-30 09:15:37,854] {dagrun.py:594} INFO - DagRun Finished: dag_id=test_mapped_virtualenv, execution_date=2022-03-30T09:15:31+00:00, run_id=backfill__2022-03-30T09:15:31+00:00, run_start_date=2022-03-30 15:15:32.853718+00:00, run_end_date=2022-03-30 15:15:37.854580+00:00, run_duration=5.000862, state=failed, external_trigger=False, run_type=backfill, data_interval_start=2022-03-30T09:15:31+00:00, data_interval_end=2052-03-22T09:15:31+00:00, dag_hash=None
   [2022-03-30 09:15:37,854] {backfill_job.py:362} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 0 | succeeded: 0 | running: 0 | failed: 0 | skipped: 0 | deadlocked: 1 | not ready: 1
   [2022-03-30 09:15:37,855] {backfill_job.py:376} DEBUG - Finished dag run loop iteration. Remaining tasks dict_values([])
   [2022-03-30 09:15:37,858] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Task has been mapped' PASSED: False, The task has yet to be mapped!
   [2022-03-30 09:15:37,858] {taskinstance.py:1041} DEBUG - Dependencies not met for <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]>, dependency 'Task has been mapped' FAILED: The task has yet to be mapped!
   [2022-03-30 09:15:37,861] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
   [2022-03-30 09:15:37,861] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Previous Dagrun State' PASSED: True, The task did not have depends_on_past set.
   [2022-03-30 09:15:37,861] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Trigger Rule' PASSED: True, The task instance did not have any upstream tasks.
   [2022-03-30 09:15:37,861] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Task has been mapped' PASSED: False, The task has yet to be mapped!
   [2022-03-30 09:15:37,862] {taskinstance.py:1041} DEBUG - Dependencies not met for <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]>, dependency 'Task has been mapped' FAILED: The task has yet to be mapped!
   [2022-03-30 09:15:37,864] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
   [2022-03-30 09:15:37,864] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Previous Dagrun State' PASSED: True, The context specified that the state of past DAGs could be ignored.
   [2022-03-30 09:15:37,864] {taskinstance.py:1061} DEBUG - <TaskInstance: test_mapped_virtualenv.venv_expirements backfill__2022-03-30T09:15:31+00:00 [scheduled]> dependency 'Trigger Rule' PASSED: True, The task instance did not have any upstream tasks.
   BackfillJob is deadlocked.
   These tasks have succeeded:
   DAG ID    Task ID    Run ID    Try number
   --------  ---------  --------  ------------
   
   These tasks are running:
   DAG ID    Task ID    Run ID    Try number
   --------  ---------  --------  ------------
   
   These tasks have failed:
   DAG ID    Task ID    Run ID    Try number
   --------  ---------  --------  ------------
   
   These tasks are skipped:
   DAG ID    Task ID    Run ID    Try number
   --------  ---------  --------  ------------
   
   These tasks are deadlocked:
   DAG ID                  Task ID           Run ID                                 Try number
   ----------------------  ----------------  -----------------------------------  ------------
   test_mapped_virtualenv  venv_expirements  backfill__2022-03-30T09:15:31+00:00             1
   [2022-03-30 09:15:37,870] {cli_action_loggers.py:84} DEBUG - Calling callbacks: []
   [2022-03-30 09:15:37,870] {settings.py:383} DEBUG - Disposing DB connection pool (PID 221785)
   
   
   ### Operating System
   
   Ubuntu Linux
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Docker-Compose
   
   ### Deployment details
   
   tested with airflow breeze and a local virtualenv with airflow 2.3.0 installed.
   
   ### 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] boring-cyborg[bot] commented on issue #22626: The command 'airflow dags test' is deadlocking tasks using AIP-42 features with the PythonVirtualenvOperator

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


   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