Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[AIRFLOW-4232] Add none_skipped trigger rule #5032

Merged
merged 2 commits into from
Apr 6, 2019

Conversation

cmdoptesc
Copy link
Contributor

Make sure you have checked all steps below.

Jira

  • My PR addresses the following Airflow Jira issues and references them in the PR title.

Description

  • Here are some details about my PR, including screenshots of any UI changes:

Add a "none_skipped" trigger rule so that downstream tasks will only run if all their upstream tasks have succeeded or failed.

Example:
  • Task 1 is a transform task that creates a temp table.
  • Task 2 is a cleanup task that drops the temp table.

Task 1 >> Task 2

If Task 1 succeeds or fails, Task 2 should drop the temp table as cleanup. However if the task never runs, we should just skip it.

Or perhaps Task 2 is a logging task that only logs successes and failures, but doesn't log skips to cut down on noise.

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":
    1. Subject is separated from body by a blank line
    2. Subject is limited to 50 characters (not including Jira issue reference)
    3. Subject does not end with a period
    4. Subject uses the imperative mood ("add", not "adding")
    5. Body wraps at 72 characters
    6. 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 so we can assign it to a appropriate release

Code Quality

  • Passes flake8

@codecov-io
Copy link

codecov-io commented Apr 3, 2019

Codecov Report

Merging #5032 into master will decrease coverage by <.01%.
The diff coverage is 75%.

Impacted file tree graph

@@            Coverage Diff             @@
##           master    #5032      +/-   ##
==========================================
- Coverage   76.23%   76.23%   -0.01%     
==========================================
  Files         466      466              
  Lines       30098    30105       +7     
==========================================
+ Hits        22946    22951       +5     
- Misses       7152     7154       +2
Impacted Files Coverage Δ
airflow/models/__init__.py 93% <ø> (ø) ⬆️
airflow/utils/trigger_rule.py 100% <100%> (ø) ⬆️
airflow/ti_deps/deps/trigger_rule_dep.py 88.31% <66.66%> (-1.83%) ⬇️

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 25b6317...ad60239. Read the comment docs.

@kaxil
Copy link
Member

kaxil commented Apr 4, 2019

there is already a trigger_rule=all_done which does the same thing i.e. runs if all the task are completed (passed or failed)

@cmdoptesc
Copy link
Contributor Author

Hi @kaxil thanks for looking into my PR.

Currently, all_done counts skipped as a "done" state (along with success, failed, and upstream_failed), as evidenced here:

https://github.com/apache/airflow/blob/ad602391d600d8feadb240e5e5265c077e741de2/airflow/ti_deps/deps/trigger_rule_dep.py#L72

So unfortunately all_done is not a replacement for none_skipped.

Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks mostly good, I'd just like a bit more tests please

tests/ti_deps/deps/test_trigger_rule_dep.py Outdated Show resolved Hide resolved
@ashb
Copy link
Member

ashb commented Apr 5, 2019

Looks like you have a test failure:

----------------------------------------------------------------------
   Traceback (most recent call last):
    tests/ti_deps/deps/test_trigger_rule_dep.py line 337 in test_none_skipped_tr_failure
      session="Fake Session"))
    airflow/ti_deps/deps/trigger_rule_dep.py line 162 in _evaluate_trigger_rule
      ti.set_state(State.SKIPPED, session)
    airflow/utils/db.py line 69 in wrapper
      return func(*args, **kwargs)
    airflow/models/__init__.py line 990 in set_state
      session.merge(self)
   AttributeError: 'str' object has no attribute 'merge'```

@cmdoptesc cmdoptesc force-pushed the al/add_none_skipped_tr branch 2 times, most recently from 0f9b28f to dfad1d7 Compare April 5, 2019 20:22
upstream_failed=0,
done=2,
flag_upstream_failed=False,
session=session))
Copy link
Contributor Author

@cmdoptesc cmdoptesc Apr 5, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I assume there was the AttributeError: 'str' object has no attribute 'merge' because we were just passing a str instead of an actual session.

Downstream tasks should run as long as their parents are in
`success`, `failed`, or `upstream_failed` states.
@ashb
Copy link
Member

ashb commented Apr 5, 2019

NOT NULL constraint failed: task_instance.execution_date now

@ashb
Copy link
Member

ashb commented Apr 5, 2019

I've made an update to this branch to try and fix those tests @cmdoptesc - I didn't realise that none of the other paths hit the upstream_failed path :/

@cmdoptesc
Copy link
Contributor Author

Thanks a million @ashb 💯

Yeah, all the other tests were just using strings, which would probably error out if the flag_upstream_failed flag was set to true.

@ashb ashb merged commit ec63195 into apache:master Apr 6, 2019
ashb pushed a commit that referenced this pull request Apr 6, 2019
Downstream tasks should run as long as their parents are in
`success`, `failed`, or `upstream_failed` states.
ashb pushed a commit that referenced this pull request Apr 6, 2019
Downstream tasks should run as long as their parents are in
`success`, `failed`, or `upstream_failed` states.
@cmdoptesc cmdoptesc deleted the al/add_none_skipped_tr branch April 10, 2019 18:16
cthenderson pushed a commit to cthenderson/apache-airflow that referenced this pull request Apr 16, 2019
Downstream tasks should run as long as their parents are in
`success`, `failed`, or `upstream_failed` states.
andriisoldatenko pushed a commit to andriisoldatenko/airflow that referenced this pull request Jul 26, 2019
Downstream tasks should run as long as their parents are in
`success`, `failed`, or `upstream_failed` states.
wmorris75 pushed a commit to modmed/incubator-airflow that referenced this pull request Jul 29, 2019
Downstream tasks should run as long as their parents are in
`success`, `failed`, or `upstream_failed` states.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants