-
Notifications
You must be signed in to change notification settings - Fork 13.9k
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
Tasks with 'none_failed_min_one_success' trigger_rule skipping before Dynamic Task Group is fully expanded #39801
Comments
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. |
Here's a slightly simpler reproduction without nested task groups or branching: from airflow.decorators import dag, task, task_group
from pendulum import datetime
@dag(
start_date=datetime(2024, 1, 1),
schedule="@daily",
catchup=False,
doc_md=__doc__,
tags=["example"],
)
def repro():
@task
def init():
return ["seize", "the", "day"]
@task_group(group_id="tg")
def tg(message):
@task
def imsleepy(message):
return message
@task(trigger_rule="none_failed_min_one_success")
def imawake(message):
print(message)
imawake(imsleepy(message))
tg.expand(message=init())
repro() |
Hi, could I take this issue? |
Please do. It would be great if you can confirm that you are able to reproduce the issue. |
I have just reproduced the issue using @RNHTTR 's example code. Does anyone have an idea of which parts of the codebase I should take a look first to find this bug? In the meanwhile I will try to figure it out. |
Some candidate starting points: One good idea looking into what's going on would be to review the unit tests for none_failed_min_one_success and see the expected behavior. |
Nice, thank you so much! I will take a look. |
Hello, I am also facing this issue, it will be awesome if someone can find a solution (or at least a workaround). Best regards and thank you for developing Airflow! |
Hi @le-chartreux ! I have already found the issue, and I am working on a solution. Soon I will send an explanation here. |
Awesome, thank you @mateuslatrova! |
@mateuslatrova - Great to hear there's progress with this! |
Hello Airflow Community, While waiting for a resolution to the trigger rule issue, I wanted to share a workaround that I've implemented. Problem ContextIn scenarios involving branching, where only one (or a part) of several tasks is executed and its result is needed downstream, the ideal trigger rule for the downstream task that will get the output would be For example: branching ─┬─> task_a ─┐
OR |─> task_to_get_the_result_of_the_task_that_runned
└─> task b ─┘ Unfortunately, this trigger rule doesn't work as expected within dynamic task groups, leading to the downstream task being erroneously skipped. Workaround StrategyMy workaround involves using the Below is an example of using from airflow.decorators import dag, task, task_group
from pendulum import datetime
from airflow.utils.trigger_rule import TriggerRule
from airflow.exceptions import AirflowSkipException
@dag(start_date=datetime(2024, 1, 1), schedule=None, catchup=False)
def workaround():
directions = get_directions()
get_message_from_direction.expand(direction=directions)
@task
def get_directions() -> list[str]:
return ["left", "right", "bottom"]
@task_group
def get_message_from_direction(direction: str) -> str | None:
message_from_left = left_task()
message_from_right = right_task()
choose_next_task_from_direction(direction) >> [message_from_left, message_from_right]
return get_message(message_from_left, message_from_right)
@task.branch
def choose_next_task_from_direction(direction: str) -> str | None:
this_group = "get_message_from_direction"
if direction == "left":
return f"{this_group}.left_task"
if direction == "right":
return f"{this_group}.right_task"
# return None, nothing follows
@task
def left_task():
return "message from left"
@task
def right_task():
return "message from right"
# NONE_FAILED_MIN_ONE_SUCCESS nor ONE_SUCCESS are used because they aren't working
# inside dynamic task groups: see https://github.com/apache/airflow/issues/39801
# To workaround, NONE_FAILED is used, but it's not a problem:
# - If no previous task failed and at least one of the previous tasks returns a message,
# (i.e., at least one succeed), everything is fine and the message is returned.
# - If at least one of the previous tasks failed, this task will not be triggered.
# - If all the previous tasks are skipped, no message will be provided and this task
# will be skipped with the AirflowSkipException.
@task(trigger_rule=TriggerRule.NONE_FAILED)
def get_message(message_from_left: str | None, message_from_right: str | None) -> str:
# Please note that the output of a successfully task should not be equivalent to False
# for the or to work! Else, use something like 'if message_from_left is not None: ...'
message = message_from_left or message_from_right
if message:
# one succeed
return message
raise AirflowSkipException("Both skipped.")
workaround() The
I hope this workaround proves useful to others facing the same issue. Best regards, |
The issueI will use @RNHTTR 's example DAG to make the explanation simpler. So, we have the following tasks:
When the Then, the scheduler gets the next task instance to be run, which is Next, the scheduler gets the following task instance to be run, which is airflow/airflow/ti_deps/deps/trigger_rule_dep.py Lines 355 to 366 in 1d7ede7
When this code is running for Since airflow/airflow/ti_deps/deps/trigger_rule_dep.py Lines 400 to 404 in 1d7ede7
Now, why did that query return 0? Because the call to So the ROOT CAUSE is: the Database is updated with Possible solution
I invite all of you to review and question this explanation to guarantee it is correct, and also, if anyone has suggestions for solutions, they are very welcome! |
I'd just like to point out that this issue seems related to #34023 which I'm trying to tackle (for now, without much success). |
Apache Airflow version
Other Airflow 2 version (please specify below)
If "Other Airflow 2 version" selected, which one?
2.8.3 and 2.9.1
What happened?
If a DAG is run for the first time, a task with a 'none_failed_min_one_success' trigger rule is almost immediately skipped before the dependant upstream tasks are complete resulting in:
![image](https://private-user-images.githubusercontent.com/39799836/333493523-31e57fae-f516-49e3-bebe-900660facb3f.png?jwt=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJnaXRodWIuY29tIiwiYXVkIjoicmF3LmdpdGh1YnVzZXJjb250ZW50LmNvbSIsImtleSI6ImtleTUiLCJleHAiOjE3MjE5MDY5NTUsIm5iZiI6MTcyMTkwNjY1NSwicGF0aCI6Ii8zOTc5OTgzNi8zMzM0OTM1MjMtMzFlNTdmYWUtZjUxNi00OWUzLWJlYmUtOTAwNjYwZmFjYjNmLnBuZz9YLUFtei1BbGdvcml0aG09QVdTNC1ITUFDLVNIQTI1NiZYLUFtei1DcmVkZW50aWFsPUFLSUFWQ09EWUxTQTUzUFFLNFpBJTJGMjAyNDA3MjUlMkZ1cy1lYXN0LTElMkZzMyUyRmF3czRfcmVxdWVzdCZYLUFtei1EYXRlPTIwMjQwNzI1VDExMjQxNVomWC1BbXotRXhwaXJlcz0zMDAmWC1BbXotU2lnbmF0dXJlPThkN2YxNzcwNjM4MWI5N2E4OGZkNjMwZDIwMjdhNGZiM2M4YjExNDJkNzI3MTNhYjBiY2E2MGQ2NGYyZGQ5NWUmWC1BbXotU2lnbmVkSGVhZGVycz1ob3N0JmFjdG9yX2lkPTAma2V5X2lkPTAmcmVwb19pZD0wIn0.mnHWli752KsEGT8hqfPooDQsAhiWgvkM-VZ2ewB3BjI)
If I clear the same DAG run, with the process_items task group fully expanded, the DAG completes as expected:
![image](https://private-user-images.githubusercontent.com/39799836/333493729-04ad50f0-10ee-4f33-b5ee-72be3b6e8210.png?jwt=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJnaXRodWIuY29tIiwiYXVkIjoicmF3LmdpdGh1YnVzZXJjb250ZW50LmNvbSIsImtleSI6ImtleTUiLCJleHAiOjE3MjE5MDY5NTUsIm5iZiI6MTcyMTkwNjY1NSwicGF0aCI6Ii8zOTc5OTgzNi8zMzM0OTM3MjktMDRhZDUwZjAtMTBlZS00ZjMzLWI1ZWUtNzJiZTNiNmU4MjEwLnBuZz9YLUFtei1BbGdvcml0aG09QVdTNC1ITUFDLVNIQTI1NiZYLUFtei1DcmVkZW50aWFsPUFLSUFWQ09EWUxTQTUzUFFLNFpBJTJGMjAyNDA3MjUlMkZ1cy1lYXN0LTElMkZzMyUyRmF3czRfcmVxdWVzdCZYLUFtei1EYXRlPTIwMjQwNzI1VDExMjQxNVomWC1BbXotRXhwaXJlcz0zMDAmWC1BbXotU2lnbmF0dXJlPTdiNGU3NjM3ZTA2MDk5ZmU2OTZjYzVmMTAzZTk0MjBkNGExMjkzYWM3MjY5MWExYTczZTcyNDI5MmZkYjI1ZmYmWC1BbXotU2lnbmVkSGVhZGVycz1ob3N0JmFjdG9yX2lkPTAma2V5X2lkPTAmcmVwb19pZD0wIn0.BZxb-uY51A6EK6q0RIjnM-VEavh65XilyM7np1nHxlY)
What you think should happen instead?
If the a_end task has a 'none_failed_min_one_success' trigger rule it should only be run when its upstream dependant tasks are complete and not in a failed or upstream_failed state and at least one upstream task has succeeded.
How to reproduce
DAG Code:
I'm pretty sure it is related to the 'none_failed_min_one_success' as if I cause an upstream task within the TaskGroup to fail, the task with the trigger rule in question (and subsequent downstream tasks) are skipped:
![image](https://private-user-images.githubusercontent.com/39799836/333501280-c39d945a-e543-4be8-b6c4-b9e59bb0541e.png?jwt=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJnaXRodWIuY29tIiwiYXVkIjoicmF3LmdpdGh1YnVzZXJjb250ZW50LmNvbSIsImtleSI6ImtleTUiLCJleHAiOjE3MjE5MDY5NTUsIm5iZiI6MTcyMTkwNjY1NSwicGF0aCI6Ii8zOTc5OTgzNi8zMzM1MDEyODAtYzM5ZDk0NWEtZTU0My00YmU4LWI2YzQtYjllNTliYjA1NDFlLnBuZz9YLUFtei1BbGdvcml0aG09QVdTNC1ITUFDLVNIQTI1NiZYLUFtei1DcmVkZW50aWFsPUFLSUFWQ09EWUxTQTUzUFFLNFpBJTJGMjAyNDA3MjUlMkZ1cy1lYXN0LTElMkZzMyUyRmF3czRfcmVxdWVzdCZYLUFtei1EYXRlPTIwMjQwNzI1VDExMjQxNVomWC1BbXotRXhwaXJlcz0zMDAmWC1BbXotU2lnbmF0dXJlPWU0NGVmMmJlZWE0ZmFlZDM0YTcwZDc1OGMyYjQxMDkwZmMxNzczOTAyNmZhODBmZmI0YzM1YmIzY2UyN2FlMzMmWC1BbXotU2lnbmVkSGVhZGVycz1ob3N0JmFjdG9yX2lkPTAma2V5X2lkPTAmcmVwb19pZD0wIn0.GrukSiy5ydhtmSdEufSl02wT6DRikztoSqAuxMg_kiM)
Compared to if the trigger rule for the a_end is default:
![image](https://private-user-images.githubusercontent.com/39799836/333501872-065b2fb8-ae4d-4f53-bb8a-07bef94207eb.png?jwt=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJnaXRodWIuY29tIiwiYXVkIjoicmF3LmdpdGh1YnVzZXJjb250ZW50LmNvbSIsImtleSI6ImtleTUiLCJleHAiOjE3MjE5MDY5NTUsIm5iZiI6MTcyMTkwNjY1NSwicGF0aCI6Ii8zOTc5OTgzNi8zMzM1MDE4NzItMDY1YjJmYjgtYWU0ZC00ZjUzLWJiOGEtMDdiZWY5NDIwN2ViLnBuZz9YLUFtei1BbGdvcml0aG09QVdTNC1ITUFDLVNIQTI1NiZYLUFtei1DcmVkZW50aWFsPUFLSUFWQ09EWUxTQTUzUFFLNFpBJTJGMjAyNDA3MjUlMkZ1cy1lYXN0LTElMkZzMyUyRmF3czRfcmVxdWVzdCZYLUFtei1EYXRlPTIwMjQwNzI1VDExMjQxNVomWC1BbXotRXhwaXJlcz0zMDAmWC1BbXotU2lnbmF0dXJlPWU1YWExMjk5N2IwMTc2ZDBlZDFiNzNlODI0ZTdhZTcyNjBiNjFjYTRiNmRjOTQ5OGZlOGI2YjFjNGMxNGYxYjUmWC1BbXotU2lnbmVkSGVhZGVycz1ob3N0JmFjdG9yX2lkPTAma2V5X2lkPTAmcmVwb19pZD0wIn0.rIT6nO2fzDR6FL356EuPndrUjGYJfqQmUE_cpSBoV4A)
Operating System
Mac OS 13.6 and Ubuntu 22.0.4
Versions of Apache Airflow Providers
No response
Deployment
Official Apache Airflow Helm Chart
Deployment details
Dev:
Docker Desktop, Helm, Kubernetes
Prod:
EKS, Helm
Anything else?
100% reproducible
Are you willing to submit PR?
Code of Conduct
The text was updated successfully, but these errors were encountered: