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 2019/12/29 20:05:35 UTC

[GitHub] [airflow] tooptoop4 opened a new pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

tooptoop4 opened a new pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954
 
 
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Airflow Jira](https://issues.apache.org/jira/browse/AIRFLOW/) issues and references them in the PR title. For example, "\[AIRFLOW-XXX\] My Airflow PR"
     - https://issues.apache.org/jira/browse/AIRFLOW-XXX
     - In case you are fixing a typo in the documentation you can prepend your commit with \[AIRFLOW-XXX\], code changes always need a Jira issue.
     - In case you are proposing a fundamental code change, you need to create an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)).
     - In case you are adding a dependency, check if the license complies with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Description
   
   - [ ] Here are some details about my PR, including screenshots of any UI changes:
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes how to use it.
     - All the public functions and the classes in the PR contain docstrings that explain what it does
     - If you implement backwards incompatible changes, please leave a note in the [Updating.md](https://github.com/apache/airflow/blob/master/UPDATING.md) so we can assign it to a appropriate release
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r364338932
 
 

 ##########
 File path: airflow/config_templates/default_airflow.cfg
 ##########
 @@ -565,6 +565,9 @@ dag_dir_list_interval = 300
 # How often should stats be printed to the logs
 print_stats_interval = 30
 
+# A DagRun can have a 'removed' task instance and still lead to DagRun 'success' state when False
 
 Review comment:
   Right now when evaluating the overall state of a DagRun the overall DagRun state can be set to SUCCESS even though one of the TaskInstance is in REMOVED state. With this new config set to True, when evaluating the overall state of a DagRun the overall DagRun state will be set to FAILED if any TaskInstance is in REMOVED state

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] stale[bot] closed pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
stale[bot] closed pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954
 
 
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-571383906
 
 
   > Can you add some tests?
   
   it is tough to reproduce the scenario.
   I tried below but it always says DAGRun is still 'running' rather than 'failed':
   `   def test_dagrun_remove_callback(self):
           def on_removed_callable(context):
               self.assertEqual(
                   context['dag_run'].dag_id,
                   'test_dagrun_remove_callback'
               )
   
           conf.set("scheduler", "REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE", "True")
   
           dag = DAG(
               dag_id='test_dagrun_remove_callback',
               start_date=datetime.datetime(2017, 1, 1),
               on_failure_callback=on_removed_callable,
           )
           dag_task1 = DummyOperator(
               task_id='test_state_removed1',
               dag=dag)
               
           initial_task_states = {
               'test_state_removed1': State.REMOVED
           }
   
           dag_run = self.create_dag_run(dag=dag,
                                         state=State.RUNNING,
                                         task_states=initial_task_states)
           # dag_run.verify_integrity()
           updated_dag_state = dag_run.update_state()
           self.assertEqual(State.FAILED, updated_dag_state)`

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r365207171
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,51 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if len(tis) == len(dag.tasks):
 
 Review comment:
   I'm not sure if this change here is right at this level -- I would have thought that we only want to check this when the new config option is True.  Or we do we need to check it at all? Do the number of tasks vs tis matter if we just look for a TI in Removed state anyway?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-579266753
 
 
   I am working on AIP-21 finalisation and have little time to spare, and have not much experience with that part - > i think it's better to wait for @mik-laj and @ashb  to come back to it :). I know @mik-laj finishes his part of AIP-21 so he might return to it shortly.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363543585
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,42 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
+            # REMOVED state counted as unfinished
+            unfinished_tasks = self.get_task_instances(
+                state=State.unfinished_or_removed(),
+                session=session
+            )
+
+        if len(tis) == len(dag.active_tasks):
 
 Review comment:
   ```suggestion
           if len(tis) == len(dag.tasks):
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r365209436
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,51 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if len(tis) == len(dag.tasks):
+            # if all roots finished and at least one failed, the run failed
+            if not unfinished_tasks and any(
+                leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
+            ):
+                self.log.info('Marking run %s failed', self)
+                self.set_state(State.FAILED)
+                dag.handle_callback(self, success=False, reason='task_failure',
+                                    session=session)
+
+            # if all leafs succeeded and no unfinished tasks, the run succeeded
+            elif not unfinished_tasks and all(
+                leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
+            ):
+                # removed tasks count as FAILURE
+                if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
+                    removed_tasks = self.get_task_instances(
+                        state=State.REMOVED,
+                        session=session
+                    )
+                    if removed_tasks and len(removed_tasks) > 0:
+                        self.log.info('Removed_tasks; Marking run %s failed', self)
+                        self.set_state(State.FAILED)
+                        dag.handle_callback(self, success=False, reason='removed_tasks',
+                                            session=session)
+                    else:
+                        self.log.info('Marking run %s successful', self)
 
 Review comment:
   I don't like that we've got duplicated else blocks here  -- it makes it harder to follow the logic here.
   
   If we change the conditonal to `if removed_tasks_lead_to_dagrun_failure and tis_in_removed_state:` then we can remove the duplication and a level of indentation (indentation being a sign of complex code)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-572718287
 
 
   @ashb pls merge

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


With regards,
Apache Git Services

[GitHub] [airflow] stale[bot] commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
stale[bot] commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-599810549
 
 
   This issue has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs. Thank you for your contributions.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r365205959
 
 

 ##########
 File path: tests/models/test_dagrun.py
 ##########
 @@ -559,3 +561,77 @@ def with_all_tasks_removed(dag):
         dagrun.verify_integrity()
         flaky_ti.refresh_from_db()
         self.assertEqual(State.NONE, flaky_ti.state)
+
+    @conf_vars({
+        ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'True'
+    })
+    def test_dagrun_removed_tasks_lead_to_dagrun_failure_true(self):
+        session = settings.Session()
+        on_failure_callback = mock.MagicMock()
+        dag = DAG(
+            'test_dagrun_removed_tasks_lead_to_dagrun_failure_true',
+            start_date=DEFAULT_DATE,
+            default_args={'owner': 'owner1'},
+            on_failure_callback=on_failure_callback
+        )
+        dag.clear()
+        with dag:
+            op1 = DummyOperator(task_id='A')
+            op2 = DummyOperator(task_id='B')
+            op2.set_upstream(op1)
+
+        dag.clear()
 
 Review comment:
   We shouldn't need dag.clear() in here twice
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363514850
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,42 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
+            # REMOVED state counted as unfinished
+            unfinished_tasks = self.get_task_instances(
+                state=State.unfinished_or_removed(),
+                session=session
+            )
 
 Review comment:
   It looks like a trick to me. What do you think to write it directly?
   ```
               
              removed_tasks_lead_to_dagrun_failure = REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE
              if removed_tasks_lead_to_dagrun_failure;
                 remove_tasks = self.get_task_instances(
                     state=[REMOVED],
                     session=session
                 )
                if len(remove_tasks) != 0:
                   set failed with descriptive reason
   ....
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.14%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.56%   -0.15%     
   ==========================================
     Files         680      679       -1     
     Lines       38556    38547       -9     
   ==========================================
   - Hits        32659    32596      -63     
   - Misses       5897     5951      +54
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/file.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9maWxlLnB5) | `86.3% <0%> (-5.17%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `68.75% <0%> (-2.95%)` | :arrow_down: |
   | ... and [30 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...0614edb](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363512087
 
 

 ##########
 File path: docs/scheduler.rst
 ##########
 @@ -58,3 +58,7 @@ In the UI, it appears as if Airflow is running your tasks a day **late**
     **Let’s Repeat That**, the scheduler runs your job one ``schedule_interval`` AFTER the start date, at the END of the period.
 
     You should refer :doc:`dag-run` for details on scheduling a DAG.
+
+.. note::
+
+    Task instances in REMOVED state are treated like SUCCESS state when determining overall DAG Run state. To make REMOVED get treated like FAILED, change removed_tasks_lead_to_dagrun_failure to True in scheduler section of .cfg
 
 Review comment:
   ```suggestion
       Task instances in ``REMOVED`` state are treated like ``SUCCESS`` state when determining overall DAG Run state. To make ``REMOVED`` get treated like ``FAILED``, change ``removed_tasks_lead_to_dagrun_failure`` to True in scheduler section of .cfg
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.4%`.
   > The diff coverage is `78.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #6954     +/-   ##
   =========================================
   + Coverage   84.56%   84.96%   +0.4%     
   =========================================
     Files         679      683      +4     
     Lines       38547    39164    +617     
   =========================================
   + Hits        32596    33276    +680     
   + Misses       5951     5888     -63
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | [airflow/utils/operator\_resources.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9vcGVyYXRvcl9yZXNvdXJjZXMucHk=) | `84.78% <ø> (ø)` | :arrow_up: |
   | ... and [204 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...3e2817c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.28%`.
   > The diff coverage is `100%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.68%   84.96%   +0.28%     
   ==========================================
     Files         680      683       +3     
     Lines       38686    39164     +478     
   ==========================================
   + Hits        32761    33276     +515     
   + Misses       5925     5888      -37
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.77% <100%> (+0.18%)` | :arrow_up: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.31% <0%> (-20.44%)` | :arrow_down: |
   | [airflow/contrib/operators/gcp\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3BfdHJhbnNmZXJfb3BlcmF0b3IucHk=) | `95.45% <0%> (-4.55%)` | :arrow_down: |
   | [airflow/providers/amazon/aws/operators/batch.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9vcGVyYXRvcnMvYmF0Y2gucHk=) | `93.75% <0%> (-2.09%)` | :arrow_down: |
   | [airflow/serialization/serialized\_objects.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZXJpYWxpemF0aW9uL3NlcmlhbGl6ZWRfb2JqZWN0cy5weQ==) | `90.03% <0%> (-1.1%)` | :arrow_down: |
   | [airflow/cli/commands/task\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvdGFza19jb21tYW5kLnB5) | `70.96% <0%> (-0.85%)` | :arrow_down: |
   | ... and [54 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...3e2817c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-571332930
 
 
   @mik-laj pls merge

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-571398463
 
 
   It works for me.
   ```
       @conf_vars({
           ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'True'
       })
       def test_dagrun_removed_tasks_lead_to_dagrun_failure_true(self):
           session = settings.Session()
           on_failure_callback = mock.MagicMock()
           dag = DAG(
               'test_dagrun_removed_tasks_lead_to_dagrun_failure_true',
               start_date=DEFAULT_DATE,
               default_args={'owner': 'owner1'},
               on_failure_callback=on_failure_callback
           )
           dag.clear()
           with dag:
               op1 = DummyOperator(task_id='A')
               op2 = DummyOperator(task_id='B')
               op2.set_upstream(op1)
   
           dag.clear()
           now = timezone.utcnow()
           dr = dag.create_dagrun(run_id='test_dagrun_deadlock',
                                  state=State.RUNNING,
                                  execution_date=now,
                                  start_date=now)
   
           ti_op1 = dr.get_task_instance(task_id=op1.task_id)
           ti_op1.set_state(state=State.REMOVED, session=session)
           ti_op2 = dr.get_task_instance(task_id=op2.task_id)
           ti_op2.set_state(state=State.SUCCESS, session=session)
           del dag.task_dict[ti_op1.task_id]
   
           dr.update_state()
           self.assertEqual(dr.state, State.FAILED)
           kall = on_failure_callback
           callback_context = kall.call_args[0][0]
           self.assertEqual('removed_tasks', callback_context['reason'])
   
       @conf_vars({
           ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'False'
       })
       def test_dagrun_removed_tasks_lead_to_dagrun_failure_false(self):
           session = settings.Session()
           on_failure_callback = mock.MagicMock()
           dag = DAG(
               'test_dagrun_removed_tasks_lead_to_dagrun_failure_false',
               start_date=DEFAULT_DATE,
               default_args={'owner': 'owner1'},
               on_success_callback=on_failure_callback
           )
           dag.clear()
           with dag:
               op1 = DummyOperator(task_id='A')
               op2 = DummyOperator(task_id='B')
               op2.set_upstream(op1)
   
           dag.clear()
           now = timezone.utcnow()
           dr = dag.create_dagrun(run_id='test_dagrun_deadlock',
                                  state=State.RUNNING,
                                  execution_date=now,
                                  start_date=now)
   
           ti_op1 = dr.get_task_instance(task_id=op1.task_id)
           ti_op1.set_state(state=State.REMOVED, session=session)
           ti_op2 = dr.get_task_instance(task_id=op2.task_id)
           ti_op2.set_state(state=State.SUCCESS, session=session)
           del dag.task_dict[ti_op1.task_id]
   
           dr.update_state()
           self.assertEqual(dr.state, State.SUCCESS)
           kall = on_failure_callback
           callback_context = kall.call_args[0][0]
           self.assertEqual('success', callback_context['reason'])
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363549714
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,42 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
+            # REMOVED state counted as unfinished
+            unfinished_tasks = self.get_task_instances(
+                state=State.unfinished_or_removed(),
+                session=session
+            )
 
 Review comment:
   updated

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-573004136
 
 
   > @ashb pls merge
   
   When I said: "Can you explain (to me here) what this does, so we can come up with a better doc comment here." I mean that we need to improve the doc text of that config option. So no, not yet until we come up with a better message there.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **decrease** coverage by `0.12%`.
   > The diff coverage is `69.23%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage   84.68%   84.55%   -0.13%     
   ==========================================
     Files         680      680              
     Lines       38686    38822     +136     
   ==========================================
   + Hits        32761    32827      +66     
   - Misses       5925     5995      +70
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `92.72% <69.23%> (-3.86%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/contrib/operators/gcp\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3BfdHJhbnNmZXJfb3BlcmF0b3IucHk=) | `95.45% <0%> (-4.55%)` | :arrow_down: |
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `88.84% <0%> (-0.46%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.62% <0%> (-0.34%)` | :arrow_down: |
   | ... and [13 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...1ff911c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3e2817c3e967a102c166dffd909a5cc030271e25?src=pr&el=desc) will **increase** coverage by `0.1%`.
   > The diff coverage is `93.65%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #6954     +/-   ##
   =========================================
   + Coverage   84.96%   85.07%   +0.1%     
   =========================================
     Files         683      791    +108     
     Lines       39164    40137    +973     
   =========================================
   + Hits        33276    34145    +869     
   - Misses       5888     5992    +104
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/macros/hive.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tYWNyb3MvaGl2ZS5weQ==) | `38.7% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/gcs\_to\_bq.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvZ2NzX3RvX2JxLnB5) | `70.58% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/cassandra\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2Fzc2FuZHJhX3RvX2djcy5weQ==) | `64.91% <ø> (ø)` | :arrow_up: |
   | [...flow/contrib/example\_dags/example\_qubole\_sensor.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2V4YW1wbGVfZGFncy9leGFtcGxlX3F1Ym9sZV9zZW5zb3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [.../contrib/operators/gcs\_to\_gcs\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3NfdG9fZ2NzX3RyYW5zZmVyX29wZXJhdG9yLnB5) | `0% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/python.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcHl0aG9uLnB5) | `95.13% <ø> (ø)` | |
   | [airflow/gcp/example\_dags/example\_dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvZXhhbXBsZV9kYWdzL2V4YW1wbGVfZGF0YWZsb3cucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcG9zdGdyZXNfdG9fZ2NzLnB5) | `85.29% <ø> (ø)` | :arrow_up: |
   | [...low/contrib/example\_dags/example\_winrm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2V4YW1wbGVfZGFncy9leGFtcGxlX3dpbnJtX29wZXJhdG9yLnB5) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/python\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcHl0aG9uX29wZXJhdG9yLnB5) | `100% <ø> (+4.19%)` | :arrow_up: |
   | ... and [450 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3e2817c...50a2bed](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r362263999
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +312,41 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
 
 Review comment:
   The documentation structure is a topic that is not taken seriously yet, as there is a lack of documentation for a large part of the options. If you have no idea where to place the documentation, you can create a new document in the `docs/howto` directory.  You can also call me and I will gladly give you tips in a specific situation.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-573328666
 
 
   updates made @ashb 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.23%`.
   > The diff coverage is `79.07%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.56%   84.79%   +0.23%     
   ==========================================
     Files         679      680       +1     
     Lines       38547    38822     +275     
   ==========================================
   + Hits        32596    32919     +323     
   + Misses       5951     5903      -48
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.01% <ø> (-0.08%)` | :arrow_down: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | ... and [170 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...c0720a6](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `91.3%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.68%   84.78%   +0.09%     
   ==========================================
     Files         680      680              
     Lines       38686    38766      +80     
   ==========================================
   + Hits        32761    32866     +105     
   + Misses       5925     5900      -25
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.65% <100%> (-0.31%)` | :arrow_down: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `95.77% <94.73%> (-0.82%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `88.84% <0%> (-0.46%)` | :arrow_down: |
   | ... and [14 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...a47a554](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r362263568
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +312,41 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
 
 Review comment:
   All documentation should be stored here: https://github.com/apache/airflow/tree/master/docs
   
   "airflow-site" repository contains only the landing pages and generated HTML files for archival purposes.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3e2817c3e967a102c166dffd909a5cc030271e25?src=pr&el=desc) will **increase** coverage by `0.1%`.
   > The diff coverage is `93.65%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #6954     +/-   ##
   =========================================
   + Coverage   84.96%   85.07%   +0.1%     
   =========================================
     Files         683      791    +108     
     Lines       39164    40137    +973     
   =========================================
   + Hits        33276    34145    +869     
   - Misses       5888     5992    +104
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/macros/hive.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tYWNyb3MvaGl2ZS5weQ==) | `38.7% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/gcs\_to\_bq.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvZ2NzX3RvX2JxLnB5) | `70.58% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/cassandra\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2Fzc2FuZHJhX3RvX2djcy5weQ==) | `64.91% <ø> (ø)` | :arrow_up: |
   | [...flow/contrib/example\_dags/example\_qubole\_sensor.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2V4YW1wbGVfZGFncy9leGFtcGxlX3F1Ym9sZV9zZW5zb3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [.../contrib/operators/gcs\_to\_gcs\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3NfdG9fZ2NzX3RyYW5zZmVyX29wZXJhdG9yLnB5) | `0% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/python.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcHl0aG9uLnB5) | `95.13% <ø> (ø)` | |
   | [airflow/gcp/example\_dags/example\_dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvZXhhbXBsZV9kYWdzL2V4YW1wbGVfZGF0YWZsb3cucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcG9zdGdyZXNfdG9fZ2NzLnB5) | `85.29% <ø> (ø)` | :arrow_up: |
   | [...low/contrib/example\_dags/example\_winrm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2V4YW1wbGVfZGFncy9leGFtcGxlX3dpbnJtX29wZXJhdG9yLnB5) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/python\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcHl0aG9uX29wZXJhdG9yLnB5) | `100% <ø> (+4.19%)` | :arrow_up: |
   | ... and [450 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3e2817c...50a2bed](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.21%`.
   > The diff coverage is `78.23%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.56%   84.77%   +0.21%     
   ==========================================
     Files         679      680       +1     
     Lines       38547    38822     +275     
   ==========================================
   + Hits        32596    32913     +317     
   + Misses       5951     5909      -42
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.01% <ø> (-0.08%)` | :arrow_down: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | ... and [170 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...1ff911c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.4%`.
   > The diff coverage is `78.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #6954     +/-   ##
   =========================================
   + Coverage   84.56%   84.96%   +0.4%     
   =========================================
     Files         679      683      +4     
     Lines       38547    39164    +617     
   =========================================
   + Hits        32596    33276    +680     
   + Misses       5951     5888     -63
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | [airflow/utils/operator\_resources.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9vcGVyYXRvcl9yZXNvdXJjZXMucHk=) | `84.78% <ø> (ø)` | :arrow_up: |
   | ... and [204 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...3e2817c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3e2817c3e967a102c166dffd909a5cc030271e25?src=pr&el=desc) will **decrease** coverage by `0.32%`.
   > The diff coverage is `93.65%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage   84.96%   84.64%   -0.33%     
   ==========================================
     Files         683      791     +108     
     Lines       39164    40137     +973     
   ==========================================
   + Hits        33276    33973     +697     
   - Misses       5888     6164     +276
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/macros/hive.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tYWNyb3MvaGl2ZS5weQ==) | `38.7% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/gcs\_to\_bq.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvZ2NzX3RvX2JxLnB5) | `70.58% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/cassandra\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2Fzc2FuZHJhX3RvX2djcy5weQ==) | `64.91% <ø> (ø)` | :arrow_up: |
   | [...flow/contrib/example\_dags/example\_qubole\_sensor.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2V4YW1wbGVfZGFncy9leGFtcGxlX3F1Ym9sZV9zZW5zb3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [.../contrib/operators/gcs\_to\_gcs\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3NfdG9fZ2NzX3RyYW5zZmVyX29wZXJhdG9yLnB5) | `0% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/python.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcHl0aG9uLnB5) | `95.13% <ø> (ø)` | |
   | [airflow/gcp/example\_dags/example\_dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvZXhhbXBsZV9kYWdzL2V4YW1wbGVfZGF0YWZsb3cucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcG9zdGdyZXNfdG9fZ2NzLnB5) | `52.94% <ø> (-32.36%)` | :arrow_down: |
   | [...low/contrib/example\_dags/example\_winrm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2V4YW1wbGVfZGFncy9leGFtcGxlX3dpbnJtX29wZXJhdG9yLnB5) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/operators/python\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvcHl0aG9uX29wZXJhdG9yLnB5) | `100% <ø> (+4.19%)` | :arrow_up: |
   | ... and [455 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3e2817c...50a2bed](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r364176253
 
 

 ##########
 File path: airflow/config_templates/default_airflow.cfg
 ##########
 @@ -565,6 +565,9 @@ dag_dir_list_interval = 300
 # How often should stats be printed to the logs
 print_stats_interval = 30
 
+# A DagRun can have a 'removed' task instance and still lead to DagRun 'success' state when False
 
 Review comment:
   Reading this I don't know what this does.
   
   Can you explain (to me here) what this does, so we can come up with a better doc comment here.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r362343434
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +312,41 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
 
 Review comment:
   docs/scheduler.rst updated. @mik-laj  pls merge

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r372676763
 
 

 ##########
 File path: airflow/config_templates/config.yml
 ##########
 @@ -1230,6 +1230,14 @@
       type: string
       example: ~
       default: "30"
+    - name: removed_tasks_lead_to_dagrun_running
+      description: |
+        Do we keep DagRun "running" when it has a task in the "removed" state? (Tasks are marked as
 
 Review comment:
   Didn't a previous version of this talk about marking the dag run as failed?
   
   Leaving the dagrun in the running state seems like it would be running forever which doesn't sound like what we want to happen.

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363543882
 
 

 ##########
 File path: airflow/models/dag.py
 ##########
 @@ -549,6 +549,11 @@ def tasks(self, val):
     def task_ids(self):
         return list(self.task_dict.keys())
 
+    @property
+    def active_tasks(self):
+        """Return tasks in a DAG. Needed as part of determining overall DagRun outcome"""
+        return [t for t in self.tasks]
 
 Review comment:
   In my opinion, you can achieve the same effect without creating a new field in this class.
   https://github.com/apache/airflow/pull/6954/files/9f5cf8f277365a2c7a4e3c56e86f651e5d83f618#r363543585

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.4%`.
   > The diff coverage is `78.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #6954     +/-   ##
   =========================================
   + Coverage   84.56%   84.96%   +0.4%     
   =========================================
     Files         679      683      +4     
     Lines       38547    39164    +617     
   =========================================
   + Hits        32596    33276    +680     
   + Misses       5951     5888     -63
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | [airflow/utils/operator\_resources.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9vcGVyYXRvcl9yZXNvdXJjZXMucHk=) | `84.78% <ø> (ø)` | :arrow_up: |
   | ... and [204 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...3e2817c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `91.3%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.68%   84.78%   +0.09%     
   ==========================================
     Files         680      680              
     Lines       38686    38766      +80     
   ==========================================
   + Hits        32761    32866     +105     
   + Misses       5925     5900      -25
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.65% <100%> (-0.31%)` | :arrow_down: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `95.77% <94.73%> (-0.82%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `88.84% <0%> (-0.46%)` | :arrow_down: |
   | ... and [14 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...a47a554](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 removed a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 removed a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-572718287
 
 
   @ashb pls merge

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363508864
 
 

 ##########
 File path: airflow/models/dag.py
 ##########
 @@ -549,6 +549,11 @@ def tasks(self, val):
     def task_ids(self):
         return list(self.task_dict.keys())
 
+    @property
+    def active_tasks(self):
+        """Return tasks in a DAG. Needed as part of determining overall DagRun outcome"""
+        return [t for t in self.tasks]
 
 Review comment:
   Can you explain the benefit of introducing this method? In my opinion we don't need 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363565513
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,51 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if len(tis) == len(dag.tasks):
 
 Review comment:
   Can you tell me a little more about this condition? In my opinion, it will always be fulfilled because we have a transactional database. When will this condition be not met?

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363625074
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +313,51 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if len(tis) == len(dag.tasks):
 
 Review comment:
   see 'def verify_integrity(', the number of TaskInstance records in the db can be changed frequently

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-571339599
 
 
   Can you add some tests?

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.4%`.
   > The diff coverage is `78.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #6954     +/-   ##
   =========================================
   + Coverage   84.56%   84.96%   +0.4%     
   =========================================
     Files         679      683      +4     
     Lines       38547    39164    +617     
   =========================================
   + Hits        32596    33276    +680     
   + Misses       5951     5888     -63
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | [airflow/utils/operator\_resources.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9vcGVyYXRvcl9yZXNvdXJjZXMucHk=) | `84.78% <ø> (ø)` | :arrow_up: |
   | ... and [204 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...3e2817c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r372702720
 
 

 ##########
 File path: airflow/config_templates/config.yml
 ##########
 @@ -1230,6 +1230,14 @@
       type: string
       example: ~
       default: "30"
+    - name: removed_tasks_lead_to_dagrun_running
+      description: |
+        Do we keep DagRun "running" when it has a task in the "removed" state? (Tasks are marked as
 
 Review comment:
   running (rather than failed) is what I've used in my patch in prod (for few months) to resolve it as eventually the tasks get into proper state on dynamic dag (keeping dagrun as 'running' is better than setting to 'failed' if task state is going into 'removed' on transient basis)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-571398463
 
 
   It works for me.
   ```python
       @conf_vars({
           ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'True'
       })
       def test_dagrun_removed_tasks_lead_to_dagrun_failure_true(self):
           session = settings.Session()
           on_failure_callback = mock.MagicMock()
           dag = DAG(
               'test_dagrun_removed_tasks_lead_to_dagrun_failure_true',
               start_date=DEFAULT_DATE,
               default_args={'owner': 'owner1'},
               on_failure_callback=on_failure_callback
           )
           dag.clear()
           with dag:
               op1 = DummyOperator(task_id='A')
               op2 = DummyOperator(task_id='B')
               op2.set_upstream(op1)
   
           dag.clear()
           now = timezone.utcnow()
           dr = dag.create_dagrun(run_id='test_dagrun_deadlock',
                                  state=State.RUNNING,
                                  execution_date=now,
                                  start_date=now)
   
           ti_op1 = dr.get_task_instance(task_id=op1.task_id)
           ti_op1.set_state(state=State.REMOVED, session=session)
           ti_op2 = dr.get_task_instance(task_id=op2.task_id)
           ti_op2.set_state(state=State.SUCCESS, session=session)
           del dag.task_dict[ti_op1.task_id]
   
           dr.update_state()
           self.assertEqual(dr.state, State.FAILED)
           kall = on_failure_callback
           callback_context = kall.call_args[0][0]
           self.assertEqual('removed_tasks', callback_context['reason'])
   
       @conf_vars({
           ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'False'
       })
       def test_dagrun_removed_tasks_lead_to_dagrun_failure_false(self):
           session = settings.Session()
           on_failure_callback = mock.MagicMock()
           dag = DAG(
               'test_dagrun_removed_tasks_lead_to_dagrun_failure_false',
               start_date=DEFAULT_DATE,
               default_args={'owner': 'owner1'},
               on_success_callback=on_failure_callback
           )
           dag.clear()
           with dag:
               op1 = DummyOperator(task_id='A')
               op2 = DummyOperator(task_id='B')
               op2.set_upstream(op1)
   
           dag.clear()
           now = timezone.utcnow()
           dr = dag.create_dagrun(run_id='test_dagrun_deadlock',
                                  state=State.RUNNING,
                                  execution_date=now,
                                  start_date=now)
   
           ti_op1 = dr.get_task_instance(task_id=op1.task_id)
           ti_op1.set_state(state=State.REMOVED, session=session)
           ti_op2 = dr.get_task_instance(task_id=op2.task_id)
           ti_op2.set_state(state=State.SUCCESS, session=session)
           del dag.task_dict[ti_op1.task_id]
   
           dr.update_state()
           self.assertEqual(dr.state, State.SUCCESS)
           kall = on_failure_callback
           callback_context = kall.call_args[0][0]
           self.assertEqual('success', callback_context['reason'])
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r362189303
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +312,41 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
 
 Review comment:
   Hi. Can you add some documentation?  We do not yet have a document describing the configuration options of the scheduler. Would you like to create it? If not, that's fine and you can also create a new document in the howto directory.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/7d97d16c47ba181839fd583b8653d2872dc4290b?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/www/app.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvYXBwLnB5) | `94.57% <0%> (ø)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [7d97d16...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `69.23%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.68%   84.77%   +0.09%     
   ==========================================
     Files         680      680              
     Lines       38686    38822     +136     
   ==========================================
   + Hits        32761    32913     +152     
   + Misses       5925     5909      -16
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `92.72% <69.23%> (-3.86%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/contrib/operators/gcp\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3BfdHJhbnNmZXJfb3BlcmF0b3IucHk=) | `95.45% <0%> (-4.55%)` | :arrow_down: |
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `88.84% <0%> (-0.46%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.62% <0%> (-0.34%)` | :arrow_down: |
   | ... and [19 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...1ff911c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `90.47%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.68%   84.7%   +0.02%     
   =========================================
     Files         680     679       -1     
     Lines       38686   38656      -30     
   =========================================
   - Hits        32761   32744      -17     
   + Misses       5925    5912      -13
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.98% <100%> (+0.02%)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/utils/config\_yaml\_to\_cfg.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb25maWdfeWFtbF90b19jZmcucHk=) | | |
   | [airflow/hooks/dbapi\_hook.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kYmFwaV9ob29rLnB5) | `91.73% <0%> (+0.82%)` | :arrow_up: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-571383906
 
 
   > Can you add some tests?
   
   it is tough to reproduce the scenario.
   I tried below but it always says DAGRun is still 'running' rather than 'failed':
   `
       def test_dagrun_remove_callback(self):
           def on_removed_callable(context):
               self.assertEqual(
                   context['dag_run'].dag_id,
                   'test_dagrun_remove_callback'
               )
   
           conf.set("scheduler", "REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE", "True")
   
           dag = DAG(
               dag_id='test_dagrun_remove_callback',
               start_date=datetime.datetime(2017, 1, 1),
               on_failure_callback=on_removed_callable,
           )
           dag_task1 = DummyOperator(
               task_id='test_state_removed1',
               dag=dag)
               
           initial_task_states = {
               'test_state_removed1': State.REMOVED
           }
   
           dag_run = self.create_dag_run(dag=dag,
                                         state=State.RUNNING,
                                         task_states=initial_task_states)
           # dag_run.verify_integrity()
           updated_dag_state = dag_run.update_state()
           self.assertEqual(State.FAILED, updated_dag_state)`

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `91.3%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.68%   84.78%   +0.09%     
   ==========================================
     Files         680      680              
     Lines       38686    38766      +80     
   ==========================================
   + Hits        32761    32866     +105     
   + Misses       5925     5900      -25
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.65% <100%> (-0.31%)` | :arrow_down: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `95.77% <94.73%> (-0.82%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `88.84% <0%> (-0.46%)` | :arrow_down: |
   | ... and [14 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...a47a554](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] dimberman commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r362262903
 
 

 ##########
 File path: airflow/models/dagrun.py
 ##########
 @@ -311,32 +312,41 @@ def update_state(self, session=None):
 
         leaf_tis = [ti for ti in tis if ti.task_id in {t.task_id for t in dag.leaves}]
 
-        # if all roots finished and at least one failed, the run failed
-        if not unfinished_tasks and any(
-            leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='task_failure',
-                                session=session)
-
-        # if all leafs succeeded and no unfinished tasks, the run succeeded
-        elif not unfinished_tasks and all(
-            leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis
-        ):
-            self.log.info('Marking run %s successful', self)
-            self.set_state(State.SUCCESS)
-            dag.handle_callback(self, success=True, reason='success', session=session)
-
-        # if *all tasks* are deadlocked, the run failed
-        elif (unfinished_tasks and none_depends_on_past and
-              none_task_concurrency and no_dependencies_met):
-            self.log.info('Deadlock; marking run %s failed', self)
-            self.set_state(State.FAILED)
-            dag.handle_callback(self, success=False, reason='all_tasks_deadlocked',
-                                session=session)
-
-        # finally, if the roots aren't done, the dag is still running
+        if conf.getboolean('scheduler', 'REMOVED_TASKS_LEAD_TO_DAGRUN_FAILURE', fallback=False):
 
 Review comment:
   @mik-laj should we make this a part of the airflow site? I want to start writing better docs for configurations and am not sure where would be the best place to put 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.14%`.
   > The diff coverage is `79.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #6954      +/-   ##
   =========================================
   + Coverage   84.56%   84.7%   +0.14%     
   =========================================
     Files         679     679              
     Lines       38547   38656     +109     
   =========================================
   + Hits        32596   32744     +148     
   + Misses       5951    5912      -39
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.08% <ø> (ø)` | :arrow_up: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <ø> (ø)` | :arrow_up: |
   | ... and [157 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...9f5cf8f](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/62f60702248a614ad4ddce46e04b6ee94f94faa9?src=pr&el=desc) will **decrease** coverage by `0.27%`.
   > The diff coverage is `86.95%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   - Coverage    84.7%   84.42%   -0.28%     
   ==========================================
     Files         680      680              
     Lines       38556    38624      +68     
   ==========================================
   - Hits        32659    32609      -50     
   - Misses       5897     6015     +118
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `93.1% <50%> (-3.2%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.87% <75%> (-0.09%)` | :arrow_down: |
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `96.19% <94.11%> (-0.4%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.59% <0%> (-0.29%)` | :arrow_down: |
   | [airflow/utils/cli.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jbGkucHk=) | `71.85% <0%> (+0.15%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [62f6070...5239fff](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r373401033
 
 

 ##########
 File path: airflow/config_templates/config.yml
 ##########
 @@ -1230,6 +1230,14 @@
       type: string
       example: ~
       default: "30"
+    - name: removed_tasks_lead_to_dagrun_running
+      description: |
+        Do we keep DagRun "running" when it has a task in the "removed" state? (Tasks are marked as
 
 Review comment:
   Eek! This is very very different behaviour to what we discussed when this PR was first opened.
   
   For most people and almost every other case this behaviour will be **very** wrong. If your dynamic dags are "randomly" removing tasks the fix is to not make them do that. Do you not get random task failures too as a result of that? (Where Airflow tries to run a task but by the time the executor starts the dag doesn't contain that dag)
   
   Is the current behaviour of Airflow such that (so long as the dagrun is still running) that when those tasks come back that the scheduler still picks them up?
   
   If, and this is a big if, we keep this change as it is this config option should be called `dagrun_stays_running_with_removed_tasks`, and we should make it clear in the docs that without something to add-back the missing dags, or a dagrun_timeout this will lead to dagruns that never 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r365205806
 
 

 ##########
 File path: tests/models/test_dagrun.py
 ##########
 @@ -559,3 +561,77 @@ def with_all_tasks_removed(dag):
         dagrun.verify_integrity()
         flaky_ti.refresh_from_db()
         self.assertEqual(State.NONE, flaky_ti.state)
+
+    @conf_vars({
+        ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'True'
+    })
+    def test_dagrun_removed_tasks_lead_to_dagrun_failure_true(self):
+        session = settings.Session()
+        on_failure_callback = mock.MagicMock()
+        dag = DAG(
+            'test_dagrun_removed_tasks_lead_to_dagrun_failure_true',
+            start_date=DEFAULT_DATE,
+            default_args={'owner': 'owner1'},
+            on_failure_callback=on_failure_callback
+        )
+        dag.clear()
+        with dag:
+            op1 = DummyOperator(task_id='A')
+            op2 = DummyOperator(task_id='B')
+            op2.set_upstream(op1)
+
+        dag.clear()
+        now = timezone.utcnow()
+        dr = dag.create_dagrun(run_id='test_dagrun_deadlock1' + now.isoformat(),
+                               state=State.RUNNING,
+                               execution_date=timezone.datetime(2019, 1, 11),
+                               start_date=now)
+
+        ti_op1 = dr.get_task_instance(task_id=op1.task_id)
+        ti_op1.set_state(state=State.REMOVED, session=session)
+        ti_op2 = dr.get_task_instance(task_id=op2.task_id)
+        ti_op2.set_state(state=State.SUCCESS, session=session)
+        del dag.task_dict[ti_op1.task_id]
+
+        dr.update_state()
+        self.assertEqual(dr.state, State.FAILED)
+        kall = on_failure_callback
+        callback_context = kall.call_args[0][0]
+        self.assertEqual('removed_tasks', callback_context['reason'])
+
+    @conf_vars({
+        ('scheduler', 'removed_tasks_lead_to_dagrun_failure'): 'False'
+    })
+    def test_dagrun_removed_tasks_lead_to_dagrun_failure_false(self):
+        session = settings.Session()
+        on_failure_callback = mock.MagicMock()
+        dag = DAG(
+            'test_dagrun_removed_tasks_lead_to_dagrun_failure_false',
+            start_date=DEFAULT_DATE,
+            default_args={'owner': 'owner1'},
+            on_success_callback=on_failure_callback
+        )
+        dag.clear()
+        with dag:
+            op1 = DummyOperator(task_id='A')
+            op2 = DummyOperator(task_id='B')
+            op2.set_upstream(op1)
+
+        dag.clear()
 
 Review comment:
   We shouldn't need `dag.clear()` in here twice

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/3d82e8cf6a9b6b0af3fafef0769f77ced1974cbf?src=pr&el=desc) will **increase** coverage by `0.11%`.
   > The diff coverage is `88.46%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.68%   84.79%   +0.11%     
   ==========================================
     Files         680      680              
     Lines       38686    38822     +136     
   ==========================================
   + Hits        32761    32919     +158     
   + Misses       5925     5903      -22
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/models/dagrun.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFncnVuLnB5) | `95.45% <88.46%> (-1.14%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `45.25% <0%> (-46.72%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | [...rflow/contrib/operators/kubernetes\_pod\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZF9vcGVyYXRvci5weQ==) | `78.75% <0%> (-20%)` | :arrow_down: |
   | [airflow/contrib/operators/gcp\_transfer\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9nY3BfdHJhbnNmZXJfb3BlcmF0b3IucHk=) | `95.45% <0%> (-4.55%)` | :arrow_down: |
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `88.84% <0%> (-0.46%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `90.62% <0%> (-0.34%)` | :arrow_down: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.01% <0%> (-0.08%)` | :arrow_down: |
   | ... and [19 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [3d82e8c...c0720a6](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-569545898
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=h1) Report
   > Merging [#6954](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0614edbb59657e4998d80e7092214c8e657861a0?src=pr&el=desc) will **increase** coverage by `0.21%`.
   > The diff coverage is `78.27%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/6954/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6954      +/-   ##
   ==========================================
   + Coverage   84.56%   84.77%   +0.21%     
   ==========================================
     Files         679      680       +1     
     Lines       38547    38822     +275     
   ==========================================
   + Hits        32596    32913     +317     
   + Misses       5951     5909      -42
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/gcp/operators/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3Avb3BlcmF0b3JzL2RhdGFmbG93LnB5) | `99.07% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/hooks/dataflow.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvaG9va3MvZGF0YWZsb3cucHk=) | `89.28% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/weight\_rule.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy93ZWlnaHRfcnVsZS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/docker\_copy\_data.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZG9ja2VyX2NvcHlfZGF0YS5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/example\_dags/example\_docker\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `97.43% <ø> (ø)` | :arrow_up: |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.01% <ø> (-0.08%)` | :arrow_down: |
   | [...flow/example\_dags/example\_docker\_swarm\_operator.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGFtcGxlX2RhZ3MvZXhhbXBsZV9kb2NrZXJfc3dhcm1fb3BlcmF0b3IucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/gcp/utils/mlengine\_operator\_utils.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy9nY3AvdXRpbHMvbWxlbmdpbmVfb3BlcmF0b3JfdXRpbHMucHk=) | `95.34% <ø> (ø)` | :arrow_up: |
   | [airflow/utils/state.py](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9zdGF0ZS5weQ==) | `96.29% <ø> (+3.19%)` | :arrow_up: |
   | ... and [170 more](https://codecov.io/gh/apache/airflow/pull/6954/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=footer). Last update [0614edb...1ff911c](https://codecov.io/gh/apache/airflow/pull/6954?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on issue #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#issuecomment-579992301
 
 
   gentle ping @ashb 

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


With regards,
Apache Git Services

[GitHub] [airflow] tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success

Posted by GitBox <gi...@apache.org>.
tooptoop4 commented on a change in pull request #6954: [AIRFLOW-4355] removed task should not lead to dagrun success
URL: https://github.com/apache/airflow/pull/6954#discussion_r363518094
 
 

 ##########
 File path: airflow/models/dag.py
 ##########
 @@ -549,6 +549,11 @@ def tasks(self, val):
     def task_ids(self):
         return list(self.task_dict.keys())
 
+    @property
+    def active_tasks(self):
+        """Return tasks in a DAG. Needed as part of determining overall DagRun outcome"""
+        return [t for t in self.tasks]
 
 Review comment:
   my understanding is Dag model has a static list of tasks that does not change, but dagrun could have a variable number of tasks due to db connectivity issues (ie REMOVED state) leading to potentially less tasks

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


With regards,
Apache Git Services