Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Task group gets marked as upstream_failed when dynamically mapped with expand_kwargs even though all upstream tasks were skipped or successfully finished. #33446

Closed
1 of 2 tasks
danielcham opened this issue Aug 16, 2023 · 1 comment · Fixed by #33732
Assignees
Labels
affected_version:2.6 Issues Reported for 2.6 area:dynamic-task-mapping AIP-42 kind:bug This is a clearly a bug

Comments

@danielcham
Copy link

danielcham commented Aug 16, 2023

Apache Airflow version

2.6.3

What happened

I am writing a DAG that transfers data from MSSQL to BigQuery, The part of the ETL process that actually fetches the data from MSSQL and moves it to BQ needs to parallelized.
I am trying to write it as a task group where the first task moves data from MSSQL to GCS, and the 2nd task loads the file into BQ.
for some odd reason when I expand the task group it is automatically marked as upstream_failed , at the very first moment the DAG is triggered.

I have tested this with a simple dag (provided below) as well and the bug was reproduced.

I found a similar issue here but the bug seems to persist even after configuring AIRFLOW__SCHEDULER__SCHEDULE_AFTER_TASK_EXECUTION=False

What you think should happen instead

The task group should be dynamically expanded after all upstream tasks have finished since expand_kwargs needs the previous task's output.

How to reproduce


from airflow.decorators import dag, task, task_group
from airflow.operators.bash import BashOperator
from pendulum import datetime


@dag(
    dag_id="example_task_group_expansion",
    schedule="@once",
    default_args={
        "depends_on_past": False,
        "email": ["[email protected]"],
        "email_on_failure": True,
        "email_on_retry": True,
        "retries": 0,
        "retry_delay": timedelta(minutes=5),
    },
    start_date=datetime(2023, 8, 1),
    catchup=False,
)
def example_dag():
    @task(task_id="TaskDistributer")
    def task_distributer():
        step = 10_000
        return [dict(interval_start=i, interval_end=i + step) for i in range(0, 100_000, step)]

    @task_group(group_id="tg1")
    def tg(interval_start, interval_end):
        task1 = BashOperator(
            task_id="task1",
            bash_command="echo $interval_start -- $interval_end",
            env={"interval_start": str(interval_start), "interval_end": str(interval_end)},
        )

        task2 = BashOperator(
            task_id="task2",
            bash_command="echo $interval_start -- $interval_end",
            env={"interval_start": str(interval_start), "interval_end": str(interval_end)},
        )

        task1 >> task2

        return task2

    tg.expand_kwargs(task_distributer())


example_dag()

Operating System

MacOS 13.4.1

Versions of Apache Airflow Providers

No providers needed to reproduce

Deployment

Docker-Compose

Deployment details

Docker-compose

Airflow image: apache/airflow:2.6.3-python3.9
Executor: Celery
Messaging queue: redis
Metadata DB: MySQL 5.7

Anything else

The problem occurs every time.

Here are some of the scheduler logs that may be relevant.

docker logs 3d4e47791238 | grep example_task_group_expansion
INFO  [alembic.runtime.migration] Context impl PostgresqlImpl.
INFO  [alembic.runtime.migration] Will assume transactional DDL.
/usr/local/lib/python3.10/site-packages/airflow/jobs/scheduler_job_runner.py:189 DeprecationWarning: The '[celery] stalled_task_timeout' config option is deprecated. Please update your config to use '[scheduler] task_queued_timeout' instead.
[2023-08-16 14:09:33 +0000] [15] [INFO] Starting gunicorn 20.1.0
[2023-08-16 14:09:33 +0000] [15] [INFO] Listening at: http://[::]:8793 (15)
[2023-08-16 14:09:33 +0000] [15] [INFO] Using worker: sync
[2023-08-16 14:09:33 +0000] [16] [INFO] Booting worker with pid: 16
[2023-08-16 14:09:33 +0000] [17] [INFO] Booting worker with pid: 17
[2023-08-16T14:10:04.870+0000] {dag.py:3504} INFO - Setting next_dagrun for example_task_group_expansion to None, run_after=None
[2023-08-16T14:10:04.881+0000] {scheduler_job_runner.py:1449} DEBUG - DAG example_task_group_expansion not changed structure, skipping dagrun.verify_integrity
[2023-08-16T14:10:04.883+0000] {dagrun.py:711} DEBUG - number of tis tasks for <DagRun example_task_group_expansion @ 2023-08-01 00:00:00+00:00: scheduled__2023-08-01T00:00:00+00:00, state:running, queued_at: 2023-08-16 14:10:04.858967+00:00. externally triggered: False>: 3 task(s)
[2023-08-16T14:10:04.883+0000] {dagrun.py:732} DEBUG - number of scheduleable tasks for <DagRun example_task_group_expansion @ 2023-08-01 00:00:00+00:00: scheduled__2023-08-01T00:00:00+00:00, state:running, queued_at: 2023-08-16 14:10:04.858967+00:00. externally triggered: False>: 3 task(s)
[2023-08-16T14:10:04.883+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Previous Dagrun State' PASSED: True, The task did not have depends_on_past set.
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Trigger Rule' PASSED: True, The task instance did not have any upstream tasks.
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1103} DEBUG - Dependencies all met for dep_context=None ti=<TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [None]>
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task1 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task1 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Previous Dagrun State' PASSED: True, The task did not have depends_on_past set.
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task1 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Trigger Rule' PASSED: True, The task instance did not have any upstream tasks.
[2023-08-16T14:10:04.884+0000] {taskinstance.py:1103} DEBUG - Dependencies all met for dep_context=None ti=<TaskInstance: example_task_group_expansion.tg1.task1 scheduled__2023-08-01T00:00:00+00:00 [None]>
[2023-08-16T14:10:04.895+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
[2023-08-16T14:10:04.895+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Previous Dagrun State' PASSED: True, The task did not have depends_on_past set.
[2023-08-16T14:10:04.897+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Trigger Rule' PASSED: False, Task's trigger rule 'all_success' requires all upstream tasks to have succeeded, but found 1 non-success(es). upstream_states=_UpstreamTIStates(success=0, skipped=0, failed=0, upstream_failed=0, removed=0, done=0), upstream_task_ids={'tg1.task1'}
[2023-08-16T14:10:04.897+0000] {taskinstance.py:1093} DEBUG - Dependencies not met for <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]>, dependency 'Trigger Rule' FAILED: Task's trigger rule 'all_success' requires all upstream tasks to have succeeded, but found 1 non-success(es). upstream_states=_UpstreamTIStates(success=0, skipped=0, failed=0, upstream_failed=0, removed=0, done=0), upstream_task_ids={'tg1.task1'}
[2023-08-16T14:10:04.902+0000] {scheduler_job_runner.py:1476} DEBUG - Skipping SLA check for <DAG: example_task_group_expansion> because no tasks in DAG have SLAs
    <TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [scheduled]>
[2023-08-16T14:10:04.910+0000] {scheduler_job_runner.py:476} INFO - DAG example_task_group_expansion has 0/16 running and queued tasks
    <TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [scheduled]>
[2023-08-16T14:10:04.911+0000] {scheduler_job_runner.py:625} INFO - Sending TaskInstanceKey(dag_id='example_task_group_expansion', task_id='TaskDistributer', run_id='scheduled__2023-08-01T00:00:00+00:00', try_number=1, map_index=-1) to executor with priority 1 and queue default
[2023-08-16T14:10:04.911+0000] {base_executor.py:147} INFO - Adding to queue: ['airflow', 'tasks', 'run', 'example_task_group_expansion', 'TaskDistributer', 'scheduled__2023-08-01T00:00:00+00:00', '--local', '--subdir', 'DAGS_FOLDER/example.py']
[2023-08-16T14:10:04.915+0000] {local_executor.py:86} INFO - QueuedLocalWorker running ['airflow', 'tasks', 'run', 'example_task_group_expansion', 'TaskDistributer', 'scheduled__2023-08-01T00:00:00+00:00', '--local', '--subdir', 'DAGS_FOLDER/example.py']
[2023-08-16T14:10:04.948+0000] {scheduler_job_runner.py:1449} DEBUG - DAG example_task_group_expansion not changed structure, skipping dagrun.verify_integrity
[2023-08-16T14:10:04.954+0000] {dagrun.py:711} DEBUG - number of tis tasks for <DagRun example_task_group_expansion @ 2023-08-01 00:00:00+00:00: scheduled__2023-08-01T00:00:00+00:00, state:running, queued_at: 2023-08-16 14:10:04.858967+00:00. externally triggered: False>: 3 task(s)
[2023-08-16T14:10:04.954+0000] {dagrun.py:732} DEBUG - number of scheduleable tasks for <DagRun example_task_group_expansion @ 2023-08-01 00:00:00+00:00: scheduled__2023-08-01T00:00:00+00:00, state:running, queued_at: 2023-08-16 14:10:04.858967+00:00. externally triggered: False>: 1 task(s)
[2023-08-16T14:10:04.954+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Not In Retry Period' PASSED: True, The task instance was not marked for retrying.
[2023-08-16T14:10:04.954+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]> dependency 'Previous Dagrun State' PASSED: True, The task did not have depends_on_past set.
[2023-08-16T14:10:04.958+0000] {taskinstance.py:899} DEBUG - Setting task state for <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [None]> to upstream_failed
[2023-08-16T14:10:04.958+0000] {taskinstance.py:1112} DEBUG - <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [upstream_failed]> dependency 'Trigger Rule' PASSED: False, Task's trigger rule 'all_success' requires all upstream tasks to have succeeded, but found 1 non-success(es). upstream_states=_UpstreamTIStates(success=0, skipped=0, failed=0, upstream_failed=1, removed=0, done=1), upstream_task_ids={'tg1.task1'}
[2023-08-16T14:10:04.958+0000] {taskinstance.py:1093} DEBUG - Dependencies not met for <TaskInstance: example_task_group_expansion.tg1.task2 scheduled__2023-08-01T00:00:00+00:00 [upstream_failed]>, dependency 'Trigger Rule' FAILED: Task's trigger rule 'all_success' requires all upstream tasks to have succeeded, but found 1 non-success(es). upstream_states=_UpstreamTIStates(success=0, skipped=0, failed=0, upstream_failed=1, removed=0, done=1), upstream_task_ids={'tg1.task1'}
[2023-08-16T14:10:04.963+0000] {scheduler_job_runner.py:1476} DEBUG - Skipping SLA check for <DAG: example_task_group_expansion> because no tasks in DAG have SLAs
[2023-08-16T14:10:05.236+0000] {dagbag.py:506} DEBUG - Loaded DAG <DAG: example_task_group_expansion>
Changing /usr/local/airflow/logs/dag_id=example_task_group_expansion/run_id=scheduled__2023-08-01T00:00:00+00:00/task_id=TaskDistributer permission to 509
[2023-08-16T14:10:05.265+0000] {task_command.py:410} INFO - Running <TaskInstance: example_task_group_expansion.TaskDistributer scheduled__2023-08-01T00:00:00+00:00 [queued]> on host 3d4e47791238
[2023-08-16T14:10:05.453+0000] {listener.py:32} INFO - TaskInstance Details: dag_id=example_task_group_expansion, task_id=TaskDistributer, dagrun_id=scheduled__2023-08-01T00:00:00+00:00, map_index=-1, run_start_date=2023-08-16 14:10:05.346669+00:00, try_number=1, job_id=302, op_classpath=airflow.decorators.python._PythonDecoratedOperator, airflow.decorators.base.DecoratedOperator, airflow.operators.python.PythonOperator
[2023-08-16T14:10:06.001+0000] {scheduler_job_runner.py:1449} DEBUG - DAG example_task_group_expansion not changed structure, skipping dagrun.verify_integrity
[2023-08-16T14:10:06.002+0000] {dagrun.py:711} DEBUG - number of tis tasks for <DagRun example_task_group_expansion @ 2023-08-01 00:00:00+00:00: scheduled__2023-08-01T00:00:00+00:00, state:running, queued_at: 2023-08-16 14:10:04.858967+00:00. externally triggered: False>: 3 task(s)
[2023-08-16T14:10:06.002+0000] {dagrun.py:609} ERROR - Marking run <DagRun example_task_group_expansion @ 2023-08-01 00:00:00+00:00: scheduled__2023-08-01T00:00:00+00:00, state:running, queued_at: 2023-08-16 14:10:04.858967+00:00. externally triggered: False> failed
[2023-08-16T14:10:06.002+0000] {dagrun.py:681} INFO - DagRun Finished: dag_id=example_task_group_expansion, execution_date=2023-08-01 00:00:00+00:00, run_id=scheduled__2023-08-01T00:00:00+00:00, run_start_date=2023-08-16 14:10:04.875813+00:00, run_end_date=2023-08-16 14:10:06.002810+00:00, run_duration=1.126997, state=failed, external_trigger=False, run_type=scheduled, data_interval_start=2023-08-01 00:00:00+00:00, data_interval_end=2023-08-01 00:00:00+00:00, dag_hash=a89f91f4d5dab071c49b1d98a4bd5c13
[2023-08-16T14:10:06.004+0000] {dag.py:3504} INFO - Setting next_dagrun for example_task_group_expansion to None, run_after=None
[2023-08-16T14:10:06.005+0000] {scheduler_job_runner.py:1476} DEBUG - Skipping SLA check for <DAG: example_task_group_expansion> because no tasks in DAG have SLAs
[2023-08-16T14:10:06.010+0000] {base_executor.py:299} DEBUG - Changing state: TaskInstanceKey(dag_id='example_task_group_expansion', task_id='TaskDistributer', run_id='scheduled__2023-08-01T00:00:00+00:00', try_number=1, map_index=-1)
[2023-08-16T14:10:06.011+0000] {scheduler_job_runner.py:677} INFO - Received executor event with state success for task instance TaskInstanceKey(dag_id='example_task_group_expansion', task_id='TaskDistributer', run_id='scheduled__2023-08-01T00:00:00+00:00', try_number=1, map_index=-1)
[2023-08-16T14:10:06.012+0000] {scheduler_job_runner.py:713} INFO - TaskInstance Finished: dag_id=example_task_group_expansion, task_id=TaskDistributer, run_id=scheduled__2023-08-01T00:00:00+00:00, map_index=-1, run_start_date=2023-08-16 14:10:05.346669+00:00, run_end_date=2023-08-16 14:10:05.518275+00:00, run_duration=0.171606, state=success, executor_state=success, try_number=1, max_tries=0, job_id=302, pool=default_pool, queue=default, priority_weight=1, operator=_PythonDecoratedOperator, queued_dttm=2023-08-16 14:10:04.910449+00:00, queued_by_job_id=289, pid=232

As can be seen from the logs, no upstream tasks are in done state yet the expanded task is set as upstream_failed.

slack discussion

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@danielcham danielcham added area:core kind:bug This is a clearly a bug needs-triage label for new issues that we didn't triage yet labels Aug 16, 2023
@boring-cyborg
Copy link

boring-cyborg bot commented Aug 16, 2023

Thanks for opening your first issue here! Be sure to follow the issue template! If you are willing to raise PR to address this issue please do so, no need to wait for approval.

@RNHTTR RNHTTR added area:dynamic-task-mapping and removed area:core needs-triage label for new issues that we didn't triage yet labels Aug 16, 2023
@ephraimbuddy ephraimbuddy self-assigned this Aug 24, 2023
ephraimbuddy added a commit to astronomer/airflow that referenced this issue Aug 29, 2023
When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: apache#33446
ephraimbuddy added a commit that referenced this issue Aug 29, 2023
* Fix MappedTaskGroup tasks not respecting upstream dependency

When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: #33446

* set the relationship in __exit__
ephraimbuddy added a commit that referenced this issue Sep 1, 2023
* Fix MappedTaskGroup tasks not respecting upstream dependency

When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: #33446

* set the relationship in __exit__

(cherry picked from commit fe27031)
ahidalgob pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue May 15, 2024
* Fix MappedTaskGroup tasks not respecting upstream dependency

When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: apache/airflow#33446

* set the relationship in __exit__

(cherry picked from commit fe27031382e2034b59a23db1c6b9bdbfef259137)

GitOrigin-RevId: 22df7b111261c78fbeeb38191226f9694986bd05
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jul 18, 2024
* Fix MappedTaskGroup tasks not respecting upstream dependency

When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: apache/airflow#33446

* set the relationship in __exit__

GitOrigin-RevId: fe27031382e2034b59a23db1c6b9bdbfef259137
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 20, 2024
* Fix MappedTaskGroup tasks not respecting upstream dependency

When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: apache/airflow#33446

* set the relationship in __exit__

GitOrigin-RevId: fe27031382e2034b59a23db1c6b9bdbfef259137
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 8, 2024
* Fix MappedTaskGroup tasks not respecting upstream dependency

When a MappedTaskGroup has upstream dependencies, the tasks in the group don't wait for the upstream tasks
before they start running, this causes the tasks to fail.
From my investigation, the tasks inside the MappedTaskGroup don't have upstream tasks while the
MappedTaskGroup has the upstream tasks properly set. Due to this, the task's dependencies are met even though the Group has
upstreams that haven't finished.
The Fix was to set upstreams after creating the task group with the factory
Closes: apache/airflow#33446

* set the relationship in __exit__

GitOrigin-RevId: fe27031382e2034b59a23db1c6b9bdbfef259137
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.6 Issues Reported for 2.6 area:dynamic-task-mapping AIP-42 kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants