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 2021/04/21 08:13:43 UTC

[GitHub] [airflow] pateash opened a new issue #15468: DAG state is Success even in case of intermediate Task Failure

pateash opened a new issue #15468:
URL: https://github.com/apache/airflow/issues/15468


   **Description**
   I am not sure if this is BUG, that's why creating as a Feature Request
   
   
   **Minimal code to reproduce**
   
       args = {
       'owner': 'airflow',
       }
   
       with DAG(
           dag_id='dag_state_test',
           default_args=args,
           schedule_interval='0 0 * * *',
           start_date=days_ago(2),
       ) as dag:
   
       run_this = BashOperator(
           task_id='middle',
           bash_command='echo "middle" && exit 1',
       )
   
       startOperator = BashOperator(
           task_id='start',
           bash_command='echo "start"',
       )
   
       endOperator = BashOperator(
           task_id='end',
           bash_command='echo "end"',
           trigger_rule=TriggerRule.ALL_DONE  # so, this runs always no matter what happens in middle.
       )
       startOperator >> run_this >> endOperator
   
   
   **Description** 
   In our case we have following pipelines
   
       DataProcCreateCluster() -> Spark Jobs -> DataProcDeleteCluster()
   
   **DataProcDeleteCluster()** has to run always no matter what happens in between ( TriggerRule.ALL_DONE used), 
   as it looks like the DAG state is inherited only from state of last task.
   
   ![image](https://user-images.githubusercontent.com/16856802/115519682-0ca04400-a2a7-11eb-9c3c-9611317e141b.png)
   
   **Use case / motivation**
   We should be able to set DAG state using rules like ALL_SUCCESS ( only set success if all tasks are success)
   
   **Are you willing to submit a PR?**
   
   <!--- We accept contributions! -->
   
   **Related Issues**
   NONE
   
   <!-- Is there currently another issue associated with 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.

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



[GitHub] [airflow] kaxil commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-958356476


   Like @eladkal mentioned - I have had similar requirements like him and I would have `on_failure_callbacks` notifying that my task failed.
   
   For your case, you can write a CustomOperator which you have at the end that checks if your EMR task failed or not and fail this task itself, thereby the DagRun will also be failed as it is the last task.
   
   
   ---
   
   Your feature request while legit need more thinking through:
   - What happens when there are more than one terminal tasks
   - Do we want to allow users to set which task_id need to succeed for DagRun status


-- 
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] eladkal commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
eladkal commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-859793238






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

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



[GitHub] [airflow] pateash edited a comment on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
pateash edited a comment on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-860211121






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

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



[GitHub] [airflow] kaxil commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-958356476


   Like @eladkal mentioned - I have had similar requirements like him and I would have `on_failure_callbacks` notifying that my task failed.
   
   For your case, you can write a CustomOperator which you have at the end that checks if your EMR task failed or not and fail this task itself, thereby the DagRun will also be failed as it is the last task.
   
   
   ---
   
   Your feature request while legit need more thinking through:
   - What happens when there are more than one terminal tasks
   - Do we want to allow users to set which task_id need to succeed for DagRun status


-- 
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] pateash commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
pateash commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-952550267


   > I have similar use case but I don't cause the DAG itself to fail if EMR failed. Like you I run a task task to terminate the machine. Am I correct that what you are asking is to change the way that Airflow consider DAG runs statuses (success/failure) by giving the user the option to set it according to his own logic?
   
   In your case, if you consider dag to be successful even though intermediate jobs have failed,
   How are you monitoring your jobs and Isn't the dag should fail ideally in your case? 


-- 
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] pateash commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
pateash commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-952549412


   @eladkal @potiuk @ashb @kaxil @mik-laj 
   
   Does this requirement looks legit?
   I would like to work on it, if this looks ok.


-- 
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] pateash edited a comment on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
pateash edited a comment on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-952550267


   > I have similar use case but I don't cause the DAG itself to fail if EMR failed. Like you I run a task task to terminate the machine. Am I correct that what you are asking is to change the way that Airflow consider DAG runs statuses (success/failure) by giving the user the option to set it according to his own logic?
   
   @eladkal  In your case, if you consider dag to be successful even though intermediate jobs have failed,
   How are you monitoring your jobs and Isn't the dag should fail ideally in your case? 


-- 
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] pateash commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
pateash commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-823880427


   ### Workaround
   As the task only look into parents for the state, we can create a **DummyOperator** which has dependency overall 
   
   
   
       ......
   
       dummy_end = DummyOperator(
           task_id='dummy_end',
           trigger_rule=TriggerRule.NONE_FAILED_OR_SKIPPED
   
       )
   
       .........
   
   
       startOperator >> dummy_end
       run_this >> dummy_end
       endOperator >> dummy_end
   
   ![image](https://user-images.githubusercontent.com/16856802/115522012-58ec8380-a2a9-11eb-9fc2-0cd94cb54b0e.png)
   


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

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



[GitHub] [airflow] pateash commented on issue #15468: DAG state is Success even in case of intermediate Task Failure for TriggerRule.ALL_DONE

Posted by GitBox <gi...@apache.org>.
pateash commented on issue #15468:
URL: https://github.com/apache/airflow/issues/15468#issuecomment-860211121






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

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