You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "knab-analytics (via GitHub)" <gi...@apache.org> on 2023/02/14 12:47:14 UTC

[GitHub] [airflow] knab-analytics opened a new issue, #29531: Dynamic task mapping does not always create mapped tasks

knab-analytics opened a new issue, #29531:
URL: https://github.com/apache/airflow/issues/29531

   ### Apache Airflow version
   
   2.5.1
   
   ### What happened
   
   Same problem as https://github.com/apache/airflow/issues/28296, but seems to happen nondeterministically, and still happens when ignoring `depends_on_past=True`.
   
   I've got a task that retrieves some filenames, which then creates dynamically mapped tasks to move the files, one per task.
   I'm using a similar task across multiple DAGs. However, task mapping fails on some DAG runs: it inconsistently happens per DAG run, and some DAGs do not seem to be affected at all. These seem to be the DAGs where no task was ever mapped, so that the mapped task instance ended up in a Skipped state.
   
   What happens is that multiple files will be found, but only a single dynamically mapped task will be created. This task never starts and has map_index of -1. It can be found under the "List instances, all runs" menu, but says "No Data found." under the "Mapped Tasks" tab.
   
   ![Screenshot 2023-02-14 at 13 29 15](https://user-images.githubusercontent.com/64646000/218742434-c132d3c1-8013-446f-8fd0-9b485506f43e.png)
   
   ![Screenshot 2023-02-14 at 13 29 25](https://user-images.githubusercontent.com/64646000/218742461-fb0114f6-6366-403b-841e-03b0657e3561.png)
   
   When I press the "Run" button when the mapped task is selected, the following error appears:
   
   ```
   Could not queue task instance for execution, dependencies not met: Previous Dagrun State: depends_on_past is true for this task's DAG, but the previous task instance has not run yet., Task has been mapped: The task has yet to be mapped!
   ```
   
   The previous task _has_ run however. No errors appeared in my Airflow logs.
   
   When I try to run the task with **Ignore All Deps** enabled, I get the error:
   
   ```
   Could not queue task instance for execution, dependencies not met: Previous Dagrun State: depends_on_past is true for this task's DAG, but the previous task instance has not run yet., Task has been mapped: The task has yet to be mapped!
   ```
   
   This last bit is a contradiction, the task cannot be mapped and not mapped simultaneously.
   
   If the amount of mapped tasks is 0 while in this erroneous state, the mapped tasks will not be marked as skipped as expected.
   
   ### What you think should happen instead
   
   The mapped tasks should not get stuck with "no status".
   
   The mapped tasks should be created and ran successfully, or in the case of a 0-length list output of the upstream task they should be skipped.
   
   ### How to reproduce
   
   Run the below DAG, if it runs successfully clear several tasks out of order. This may not immediately reproduce the bug, but after some task clearing, for me it always ends up in the faulty state described above.
   
   ```
   from airflow import DAG
   from airflow.decorators import task
   import datetime as dt
   
   from airflow.operators.python import PythonOperator
   import random
   
   
   @task
   def get_filenames_kwargs():
       return [
           {"file_name": i}
           for i in range(random.randint(0, 2))
       ]
   
   def print_filename(file_name):
       print(file_name)
   
   with DAG(
           dag_id="dtm_test_2",
           start_date=dt.datetime(2023, 2, 10),
           default_args={
               "owner": "airflow",
               "depends_on_past": True,
           },
           schedule="@daily",
   ) as dag:
       get_filenames_task = get_filenames_kwargs.override(task_id="get_filenames_task")()
   
       print_filename_task = PythonOperator.partial(
           task_id="print_filename_task",
           python_callable=print_filename,
       ).expand(op_kwargs=get_filenames_task)
   ```
   
   ### Operating System
   
   Amazon Linux v2
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Docker-Compose
   
   ### Deployment details
   
   _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.apache.org

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


[GitHub] [airflow] uranusjr closed issue #29531: Dynamic task mapping does not always create mapped tasks

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr closed issue #29531: Dynamic task mapping does not always create mapped tasks
URL: https://github.com/apache/airflow/issues/29531


-- 
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] dzhigimont commented on issue #29531: Dynamic task mapping does not always create mapped tasks

Posted by "dzhigimont (via GitHub)" <gi...@apache.org>.
dzhigimont commented on issue #29531:
URL: https://github.com/apache/airflow/issues/29531#issuecomment-1523977469

   Hi @potiuk, I've prepared the MR https://github.com/apache/airflow/pull/30892 with a solution to such a problem.
   The previous fix doesn't work for two reasons:
   1.  dependecies are checked before expanding of mapped tasks
   2. amount of mapped tasks for each dag run can be different so when current dag run have greater amount of tasks then previous dag run the depends_on_past dependecy will never be sutisfied.
   
   I propose a solution in which I check all previously mapped tasks for one task_id as one whole dependency for the current mapped task since we can have different amounts of mapped tasks, and if the current dag run has more mapped tasks and there is no such index in the previous dag run, it is difficult to understand that the previous dag run really doesn't have such an index or tasks haven't been expanded yet. Also, I think if tasks have the same map_index, it aren't guaranteed that they do the same work and it's better to wait until all mapped tasks in the past finish.


-- 
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] knab-analytics commented on issue #29531: Dynamic task mapping does not always create mapped tasks

Posted by "knab-analytics (via GitHub)" <gi...@apache.org>.
knab-analytics commented on issue #29531:
URL: https://github.com/apache/airflow/issues/29531#issuecomment-1603743267

   @spire-mike This is not fixed AFAIK, still happens on 2.6.1 using the code in the [top post](https://github.com/apache/airflow/issues/29531#issue-1584108885).


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

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

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


[GitHub] [airflow] potiuk commented on issue #29531: Dynamic task mapping does not always create mapped tasks

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on issue #29531:
URL: https://github.com/apache/airflow/issues/29531#issuecomment-1437455477

   cc: @uranusjr just dragging your attention - seems that https://github.com/apache/airflow/pull/28379 did not solve the issue and semes also another user also sees the same behaviour in 2.5.1 that supposed to fix it https://github.com/apache/airflow/issues/28296#issuecomment-1403551217


-- 
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] spire-mike commented on issue #29531: Dynamic task mapping does not always create mapped tasks

Posted by "spire-mike (via GitHub)" <gi...@apache.org>.
spire-mike commented on issue #29531:
URL: https://github.com/apache/airflow/issues/29531#issuecomment-1604625983

   Thanks @knab-analytics, that's too bad. I hope it gets resolved. As a workaround I set `depends_on_past=False` and am using `ExternalTaskSensor` to ensure the last task of the previous run was successful. It's not ideal but it works for my specific situation. Just mentioning it in case it helps you.


-- 
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] spire-mike commented on issue #29531: Dynamic task mapping does not always create mapped tasks

Posted by "spire-mike (via GitHub)" <gi...@apache.org>.
spire-mike commented on issue #29531:
URL: https://github.com/apache/airflow/issues/29531#issuecomment-1603589690

   Hi @knab-analytics, I'm trying to follow what the status is of this issue, but getting confused between all of the related issues and milestones. Is it your understanding that this issue is supposed to be fixed? I am still seeing this problem occurring on my Airflow 2.5.1 installation using Cloud Composer.


-- 
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 #29531: Dynamic task mapping does not always create mapped tasks

Posted by "boring-cyborg[bot] (via GitHub)" <gi...@apache.org>.
boring-cyborg[bot] commented on issue #29531:
URL: https://github.com/apache/airflow/issues/29531#issuecomment-1429693430

   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