You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "RachitSharma2001 (via GitHub)" <gi...@apache.org> on 2023/02/07 19:06:30 UTC

[GitHub] [airflow] RachitSharma2001 opened a new pull request, #29406: Add Fail Fast feature for DAGs

RachitSharma2001 opened a new pull request, #29406:
URL: https://github.com/apache/airflow/pull/29406

   Addresses issue #26854


-- 
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] RachitSharma2001 commented on pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1430250319

   Hi everyone, I was wondering if I could get some feedback on the approach I used for adding this feature. Does the approach look good, or do you have any suggestions for changes?


-- 
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 a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1159328895


##########
airflow/exceptions.py:
##########
@@ -207,6 +207,22 @@ def __init__(self, *args, **kwargs):
         warnings.warn("DagFileExists is deprecated and will be removed.", DeprecationWarning, stacklevel=2)
 
 
+class DagInvalidTriggerRule(AirflowException):
+    """Raise when a dag has 'fail_stop' enabled yet has a non-default trigger rule"""
+
+    @staticmethod
+    def check(dag: DAG | None, trigger_rule: str):
+        from airflow.models.abstractoperator import DEFAULT_TRIGGER_RULE
+
+        if dag is not None and dag.fail_stop and trigger_rule != DEFAULT_TRIGGER_RULE:
+            raise DagInvalidTriggerRule()

Review Comment:
   Seems simpler if this is a classmethod and use `raise cls`?



-- 
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 a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1156433853


##########
airflow/models/baseoperator.py:
##########
@@ -767,6 +767,9 @@ def __init__(
         dag = dag or DagContext.get_current_dag()
         task_group = task_group or TaskGroupContext.get_current_task_group(dag)
 
+        if dag is not None and dag.fail_stop and trigger_rule != DEFAULT_TRIGGER_RULE:
+            raise DagInvalidTriggerRule()

Review Comment:
   It seems awkward to me `fail_stop` and `DEFAULT_TRIGGER_RULE` are repeated when we check to raise the exception, and when the exception is rendered.
   
   Would something like a `check` function on `DagInvalidTriggerRule` encapsulating the logic be a good idea, so the logic exists only in one class?



-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1519107818

   Nice one :)


-- 
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] RachitSharma2001 commented on a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1159193888


##########
airflow/models/baseoperator.py:
##########
@@ -767,6 +767,9 @@ def __init__(
         dag = dag or DagContext.get_current_dag()
         task_group = task_group or TaskGroupContext.get_current_task_group(dag)
 
+        if dag is not None and dag.fail_stop and trigger_rule != DEFAULT_TRIGGER_RULE:
+            raise DagInvalidTriggerRule()

Review Comment:
   That makes sense. I have added a static `check` method to the `DagInvalidTriggerRule` class, which checks the necessary condition and throws an exception if needed. Let me know if this was what you had in mind or if there is a better possible implementation for this.



-- 
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] RachitSharma2001 commented on pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1518231147

   Hi everyone, I was wondering if these changes look good, or if there is anything else I should add?


-- 
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] dstandish commented on pull request #29406: Add fail stop feature for DAGs

Posted by "dstandish (via GitHub)" <gi...@apache.org>.
dstandish commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1658842100

   > One suggestion of how to approach this was to modify base_executor to kill all tasks within a DAG once one task fails. The problem that I found within this approach was that, first of all there are several different executors which all store which tasks are running, queued, or completed differently, making it hard to make a unified function that kills all running tasks on all types of executors. More importantly, through my tests I have found that the executor often doesn't have full information of all the tasks in a dag run when one task fails. What this means is that, if a task fails, the executor may not yet have full information about all the tasks currently running within that DAG, and all tasks currently queued, and thus cannot properly fail the DAG.
   
   Yeah it does seem that the current approach does allow for races between scheduler and task.
   
   What if tis are expanded after `tis = self.get_dagrun(session).get_task_instances()` is called and before anything is killed?
   
   Separately, what if here we skip a TI and the scheduler sets it to queued?
   
   I'm not sure the frequency with which this kind of thing would manifest but the conditions aren't hard to imagine.


-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1463340925

   Yeah both sounds right to me.


-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1442267128

   How should this interact with trigger rules?


-- 
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] RachitSharma2001 commented on a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1161329261


##########
airflow/models/dag.py:
##########
@@ -357,6 +358,9 @@ class DAG(LoggingMixin):
         Can be used as an HTTP link (for example the link to your Slack channel), or a mailto link.
         e.g: {"dag_owner": "https://airflow.apache.org/"}
     :param auto_register: Automatically register this DAG when it is used in a ``with`` block
+    :param fail_stop: Fails currently running tasks when task in DAG fails
+        **Warning**: A fail stop dag can only have tasks with the default trigger rule ("all_success").
+        An exception will be thrown if any task in a fail stop dag has a non default trigger rule.

Review Comment:
   That makes sense. I have updated the documentation so that the `param fail_stop` sentence has a period at the end of it. See [here](https://github.com/apache/airflow/pull/29406/files#diff-62c8e300ee91e0d59f81e0ea5d30834f04db71ae74f2e155a10b51056b00b59b)



-- 
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] RachitSharma2001 commented on a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1161329214


##########
airflow/exceptions.py:
##########
@@ -207,6 +207,22 @@ def __init__(self, *args, **kwargs):
         warnings.warn("DagFileExists is deprecated and will be removed.", DeprecationWarning, stacklevel=2)
 
 
+class DagInvalidTriggerRule(AirflowException):
+    """Raise when a dag has 'fail_stop' enabled yet has a non-default trigger rule"""
+
+    @staticmethod
+    def check(dag: DAG | None, trigger_rule: str):
+        from airflow.models.abstractoperator import DEFAULT_TRIGGER_RULE
+
+        if dag is not None and dag.fail_stop and trigger_rule != DEFAULT_TRIGGER_RULE:
+            raise DagInvalidTriggerRule()

Review Comment:
   Good point. I have updated the code to be a class method rather than a static method.



-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1437591152

   @ashb @uranusjr - can you think of some hidden side-effects. Seems like this implementation of "fail-fast" is simple and might just work - unless I missed something.


-- 
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] RachitSharma2001 commented on a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1116018115


##########
airflow/models/taskinstance.py:
##########
@@ -172,6 +172,19 @@ def set_current_context(context: Context) -> Generator[Context, None, None]:
             )
 
 
+def stop_all_tasks_in_dag(tis: list[TaskInstance], session: Session, task_id_to_ignore: int):
+    for ti in tis:
+        if ti.task_id == task_id_to_ignore or ti.state in (
+            TaskInstanceState.SUCCESS,
+            TaskInstanceState.FAILED,
+        ):
+            continue
+        if ti.state == TaskInstanceState.RUNNING:
+            ti.error(session)

Review Comment:
   Sounds good! I have added a logging statement for when a running task is being force failed, and when a task is being set to the skipped state. Let me know if these log statements are good or if there are anything to change.



-- 
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 a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1112342504


##########
airflow/models/taskinstance.py:
##########
@@ -172,6 +172,19 @@ def set_current_context(context: Context) -> Generator[Context, None, None]:
             )
 
 
+def stop_all_tasks_in_dag(tis: list[TaskInstance], session: Session, task_id_to_ignore: int):
+    for ti in tis:
+        if ti.task_id == task_id_to_ignore or ti.state in (
+            TaskInstanceState.SUCCESS,
+            TaskInstanceState.FAILED,
+        ):
+            continue
+        if ti.state == TaskInstanceState.RUNNING:
+            ti.error(session)

Review Comment:
   We shoudl add some logging telling that we are doing it. 



-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1461703484

   > If we define “fail” on the DAG level as “a task will not triggered when all its upstreams finish” I think it can work logically; the question is whether this is expected by users (and we probably should invent a term other than “fail” to avoid confusion).
   
   Maybe "cancel" or "stop" ? Yes I think this is a useful cases in a number of scenarios especially when users are cost or timing (or both) conscious. There might be a number of cases, especially when you have dynamic cloud infrastructure, and long running tasks, where you know that failure of any task will generally make any other task results useless - so running them would be a waste of money (or time waiting for things to complete).


-- 
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] RachitSharma2001 commented on pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1483893796

   Hi everyone, I have added the following changes:
   1. Changed DAG 'fail_fast' to 'fail_stop' ([here](https://github.com/RachitSharma2001/airflow/blob/fail-fast-pr/airflow/models/dag.py#L426))
   2. dag.add_task throws an exception if the task has a trigger rule that is not 'all_success' ([here](https://github.com/RachitSharma2001/airflow/blob/fail-fast-pr/airflow/models/dag.py#L2365))
         a. I have also added a test for this [here](https://github.com/apache/airflow/pull/29406/files#diff-020d9b727c01b132cc07f0e2f35e93508e26f700bae5350018269a0ce487b4b7).
   3. BaseOperator throws an exception if the task has a trigger rule that is not 'all_success' and the given dag is a 'fail_stop" dag. ([here](https://github.com/apache/airflow/pull/29406/files#diff-848f325ace55b3504e8052fecdb53c0f295c891b67a6d90e9341cbe79cc545fb))
         a. I have added a test for this [here](https://github.com/apache/airflow/pull/29406/files#diff-8c9a41e08abb1a5298aad12a2bd686633b706112a8a15e371e0ec9c4aedc77e0)
   4. Documented what the fail stop parameter does, with a warning about trigger rules. ([here](https://github.com/RachitSharma2001/airflow/blob/fail-fast-pr/airflow/models/dag.py#L361)) 


-- 
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] RachitSharma2001 commented on pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1449226926

   Hi everyone, does anyone have an opinion of how the fail fast dag should interact with the trigger rules? Should there be some tests for this? Or maybe documentation to outline how to use a fail fast DAG with trigger rules?


-- 
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] RachitSharma2001 commented on pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1464637162

   That makes sense. I was thinking of making the following changes, let me know if this sounds good:
   1. Change `fail_fast` flag to `fast_stop`
   2. Throw an error if a user tries to add a trigger rule other than `all_success` to a `fast_stop` DAG
   3. Add documentation explaining what `fast_stop` does


-- 
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 a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1112342672


##########
airflow/models/taskinstance.py:
##########
@@ -172,6 +172,19 @@ def set_current_context(context: Context) -> Generator[Context, None, None]:
             )
 
 
+def stop_all_tasks_in_dag(tis: list[TaskInstance], session: Session, task_id_to_ignore: int):
+    for ti in tis:
+        if ti.task_id == task_id_to_ignore or ti.state in (
+            TaskInstanceState.SUCCESS,
+            TaskInstanceState.FAILED,
+        ):
+            continue
+        if ti.state == TaskInstanceState.RUNNING:
+            ti.error(session)

Review Comment:
   Otherwise it will be quite magical



-- 
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 a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1159329708


##########
airflow/models/dag.py:
##########
@@ -357,6 +358,9 @@ class DAG(LoggingMixin):
         Can be used as an HTTP link (for example the link to your Slack channel), or a mailto link.
         e.g: {"dag_owner": "https://airflow.apache.org/"}
     :param auto_register: Automatically register this DAG when it is used in a ``with`` block
+    :param fail_stop: Fails currently running tasks when task in DAG fails
+        **Warning**: A fail stop dag can only have tasks with the default trigger rule ("all_success").
+        An exception will be thrown if any task in a fail stop dag has a non default trigger rule.

Review Comment:
   How does this render in documentation? I don’t think the newline would be rendered, and the end result would read weird without a period in the previous sentense.



-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1485844092

   Static checks?


-- 
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] RachitSharma2001 commented on a diff in pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on code in PR #29406:
URL: https://github.com/apache/airflow/pull/29406#discussion_r1161329261


##########
airflow/models/dag.py:
##########
@@ -357,6 +358,9 @@ class DAG(LoggingMixin):
         Can be used as an HTTP link (for example the link to your Slack channel), or a mailto link.
         e.g: {"dag_owner": "https://airflow.apache.org/"}
     :param auto_register: Automatically register this DAG when it is used in a ``with`` block
+    :param fail_stop: Fails currently running tasks when task in DAG fails
+        **Warning**: A fail stop dag can only have tasks with the default trigger rule ("all_success").
+        An exception will be thrown if any task in a fail stop dag has a non default trigger rule.

Review Comment:
   That makes sense. I have updated the documentation so that the `param fail_stop` sentence has a period at the end of it. 



-- 
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 merged pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk merged PR #29406:
URL: https://github.com/apache/airflow/pull/29406


-- 
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] RachitSharma2001 commented on pull request #29406: Add Fail Fast feature for DAGs

Posted by "RachitSharma2001 (via GitHub)" <gi...@apache.org>.
RachitSharma2001 commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1436047140

   Hi everybody, it looks like all the tests pass now. I was wondering if I could get some feedback on the approach I used for adding this feature. Does the approach look good, or do you have any suggestions for changes?


-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1449441326

   I think it needs a bit of anylysis and design. It's hard to get the logic around it without viualising it, but @uranusjr is right that some thinking should be done around it. I guess we have two options: 
   * fail everything regardless from the rules (this is the current solution)
   * be "smart" about it and include triggering rules in it
   
   How the "smart" should look like - hard to say. probably each rule should be analysed how it should behave and good strategy for each rule shoudl be worked out (if that's even possible) - trying to logically attempt to follow the rules and choose for example a strategy of failing currently run tasks and propagating it furhter. That would be rather complex and there are some non-obvious traps we can fall in (what happens with the tasks that succeed when one failed?). And what about the tasks that follow them?
   
   I think it very much depends on the definition of "fail fast". If we define it as "fail all the tasks if any of those tasks fail", then current solution is good. If we try to follow triggering rules, then well, it's complex and likely "fali fast" is hard to define in general ("fail fast all the currently eligible tasks and propagagate it across the DAG"). 
   
   But maybe there is a middle-ground. Maybe we can only make fail-fast work if the only triggering rules we have are "all_success" (i.e. default). And if you add another triggering rule to a "fail-fast" DAG, this could cause an error. 
   
   I think there is a big class of DAGs which fall into that category and those are precisely the DAGs where "fail fast" defined as "fail all the tasks if any of them fail" makes perfect sense.
   
   @uranusjr ? WDYT? I guess that would only require to add the error in case any non-default triggering rule is added for "fail-fast" dag (and properly documenting it).
   


-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1461407289

   If we define “fail” on the DAG level as “a task will not triggered when all its upstreams finish” I think it can work logically; the question is whether this is expected by users (and we probably should invent a term other than “fail” to avoid confusion).


-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1464839107

   Sounds great!


-- 
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 pull request #29406: Add Fail Fast feature for DAGs

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29406:
URL: https://github.com/apache/airflow/pull/29406#issuecomment-1463360285

   `fast_stop` then instead of `fail_fast` ?


-- 
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