[AIRFLOW-558] Add Support for dag.backfill=(True|False) Option#1830
[AIRFLOW-558] Add Support for dag.backfill=(True|False) Option#1830btallman wants to merge 1 commit into
Conversation
|
Overall this looks good - from a first review. However, I am not sure about having a global option in. Changing this option would fundamentally change the behavior of what airflow does and will create questions on the mailling list "why doesnt my dag backfill?". Secondly, I would like to have this properly documented. |
|
Good points... I will look at the documentation to figure out where to put that. As to the global option, I think as long as it is well documented and doesn't change the default it should be ok... Get Outlook for iOS |
|
@bolkedebruin and @r39132 and @plypaul and others, I added docs. VERY open to suggestions for improvements. |
Current coverage is 67.01% (diff: 67.39%)@@ master #1830 diff @@
==========================================
Files 135 135
Lines 10255 10297 +42
Methods 0 0
Messages 0 0
Branches 0 0
==========================================
+ Hits 6873 6901 +28
- Misses 3382 3396 +14
Partials 0 0
|
|
I do have some questions about this functionality as well. Is the idea that some companies don't want backfill at all? If so, I'd like to see some votes for this feature (via the JIRA). Alternate functionality could be to have a UI feature like "pause" that is exposed on the UI per DAG and have it disabled by default (e.g. I feel that functionality is the right way to handle this. |
|
That actually doesn't stop backfill. As soon as you un pause the dag, it backfills. Get Outlook for iOS |
There was a problem hiding this comment.
Please dont mention this here. Documentation will be release together with Airflow itself, so there is no need.
|
Hi! To reproduce a case where this does not work, try However, a similar DAG with This suggests a problem with the combination of |
|
It looks like the issue above has been fixed but I am now seeing an issue with a DAG that has The code for the problematic DAG is below : |
|
Testing this change which fixes that Depends on past here: ` ` |
|
@btallman Did you test your fix against my attached code? Because, it still doesn't work against my provided example. I am resetting the DB before running by the way. |
2197ab1 to
9fe4f83
Compare
|
OK, this is passing CI now. Ready for you to look at it @r39132. |
|
voting for this PR! IMHO it will make Airflow behave more like enterprise workflow management tools |
There was a problem hiding this comment.
I would expect this to be part of class DagRun not of TaskInstance.
There was a problem hiding this comment.
I put it here, because ti.previous_dagrun() seems cleaner than dr.previous_dagrun(ti)
It is a helper function, as we could also leverage something like:
tip = ti.previous_ti()
if tip:
drp = tip.get_dagrun()
There was a problem hiding this comment.
Dr = Ti.get_dagrun()
Previous = dr.previous_dagrun()
DagRuns are the basis where to work from. TIs should only handle stuff within their own unit. Using previous here disregards that.
There was a problem hiding this comment.
OK, cleaned it up... Good idea. Interestingly, there was no dr.get_previous_dagrun, so I added that.
There was a problem hiding this comment.
A decision on style for ti.previous_ti vs ti.get_previous_ti() is needed as well, but that isn't something that I introduced.
4714683 to
4ac201d
Compare
|
BTW, while it says fail here, all 6 are succeeding on my fork, although I did have to kick one to run again... |
|
@btallman Please rebase. I will be happy to test once ready. |
4ac201d to
f758c01
Compare
|
@r39132 I went ahead and rebased... |
|
What is the landscape error? Btw, we are running this on half of our production... Get Outlook for iOS |
567076d to
6c87204
Compare
There was a problem hiding this comment.
this should be named six_hours_ago_to_the_hour
There was a problem hiding this comment.
@zodiac - Missed that one when we switched to testing a longer period. Good catch.
There was a problem hiding this comment.
These can be named three_minutes_ago and two_hours_and_three_minutes_ago for consistency
There was a problem hiding this comment.
@zodiac - That would be better, not sure if this will make it into a PR.
|
This overload new meaning into the term backfill: "a manually triggered CLI process to run or re-run task instances in the past" I'd suggest renaming |
There was a problem hiding this comment.
allow_scheduler_catchup=configuration.getboolean('scheduler', 'allow_scheduler_catchup_default')
There was a problem hiding this comment.
clearly describes how it affects the scheduler behavihor around DagRun creation
There was a problem hiding this comment.
# Default setting for `DAG.allow_scheduler_catchup` [clearly describe how it affects DagRun creation]
allow_scheduler_catchup_default = True
|
@mistercrunch I would suggest using |
99b67f9 to
754d96f
Compare
|
@btallman it looks like there is a lot of interest in this PR.. have a look at the comments above and I can help shepherd this through. |
There was a problem hiding this comment.
This was added in a recent PR, please don't remove it
There was a problem hiding this comment.
Yeah, that was a rebase error!!! Sorry.
There was a problem hiding this comment.
While it is outside the scope of this PR, this is potentially a quite expensive operation on the DB.
There was a problem hiding this comment.
Please leave description in. See below.
There was a problem hiding this comment.
Much cleaner. Thanks!
There was a problem hiding this comment.
Going to have to dig into this. Apparently, moving to a DR based previous_ti causes issues when NOT using a DR (backfill). Finally figure that out this afternoon. Will attempt to fix tonight.
|
Look much better! Just the final dots on I and then we're there. Thanks for all the work and bearing with us. |
|
Yeah. Have a bit of cleaning still to do.
Thanks,
Ben
…--
Ben Tallman | Senior Technologist - Google Cloud | tallmanb@google.com | 503-680-5709
Sent with help from my rotary phone...
On Sat, Dec 10, 2016 at 1:24 AM -0800, "bolkedebruin" <notifications@github.com> wrote:
Look much better! Just the final dots on I and then we're there. Thanks for all the work and bearing with us.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub, or mute the thread.
|
|
Hey @btallman would be nice to have this in before the new year. Can you rebase and see why the builds are failing? If needed I can provide some help. |
|
I will revisit in the morning. Sorry about the delayed update.
Thanks,
Ben
…--
Ben Tallman | Senior Technologist - Google Cloud | tallmanb@google.com | 503-680-5709
Sent with help from my rotary phone...
On Sun, Dec 25, 2016 at 5:56 AM -0800, "bolkedebruin" <notifications@github.com> wrote:
Hey @btallman would be nice to have this in before the new year. Can you rebase and see why the builds are failing? If needed I can provide some help.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub, or mute the thread.
|
|
Rebased and running tests on a side branch on my fork now. When they pass,
I'll push to the tracked branch.
Thanks,
Ben
…--
Ben Tallman - 503.680.5709
On Mon, Dec 26, 2016 at 6:08 PM, Ben Tallman ***@***.***> wrote:
I will revisit in the morning. Sorry about the delayed update.
Thanks,
Ben
--
Ben Tallman | Senior Technologist - Google Cloud | ***@***.*** |
503-680-5709 <(503)%20680-5709>
Sent with help from my rotary phone...
On Sun, Dec 25, 2016 at 5:56 AM -0800, "bolkedebruin" <
***@***.***> wrote:
Hey @btallman <https://github.com/btallman> would be nice to have this in
> before the new year. Can you rebase and see why the builds are failing? If
> needed I can provide some help.
>
> —
> You are receiving this because you were mentioned.
> Reply to this email directly, view it on GitHub
> <#1830 (comment)>,
> or mute the thread
> <https://github.com/notifications/unsubscribe-auth/AAKnfvbsxIzFj11ljZ5E5uxcXgQtug_Rks5rLnYdgaJpZM4KRoW8>
> .
>
|
|
Seeing errors about api-test stuff. Will investigate tomorrow.
Thanks,
Ben
…--
Ben Tallman - 503.680.5709
On Tue, Dec 27, 2016 at 10:01 PM, Ben Tallman ***@***.***> wrote:
Rebased and running tests on a side branch on my fork now. When they pass,
I'll push to the tracked branch.
Thanks,
Ben
--
Ben Tallman - 503.680.5709 <(503)%20680-5709>
On Mon, Dec 26, 2016 at 6:08 PM, Ben Tallman ***@***.***> wrote:
> I will revisit in the morning. Sorry about the delayed update.
>
> Thanks,
> Ben
>
> --
>
> Ben Tallman | Senior Technologist - Google Cloud | ***@***.*** |
> 503-680-5709 <(503)%20680-5709>
>
> Sent with help from my rotary phone...
>
>
>
>
> On Sun, Dec 25, 2016 at 5:56 AM -0800, "bolkedebruin" <
> ***@***.***> wrote:
>
> Hey @btallman <https://github.com/btallman> would be nice to have this
>> in before the new year. Can you rebase and see why the builds are failing?
>> If needed I can provide some help.
>>
>> —
>> You are receiving this because you were mentioned.
>> Reply to this email directly, view it on GitHub
>> <#1830 (comment)>,
>> or mute the thread
>> <https://github.com/notifications/unsubscribe-auth/AAKnfvbsxIzFj11ljZ5E5uxcXgQtug_Rks5rLnYdgaJpZM4KRoW8>
>> .
>>
>
|
754d96f to
f296939
Compare
|
@bolkedebruin and @r39132 - I finally was able to figure out what the issue was (turns out using DagRun to get previous TI only works if there IS a dag run (so not on backfill, go figure) so that minor change that I agreed to for Bolke turned out to be an issue. Tracked that down and the tests are now passing. Please review ASAP. |
|
@btallman I need to look in to your comment on backfills, as in master backfills actually create dag runs. Can you elaborate a bit further what you think went wrong? |
|
It may not actually be backfill, it's just that at least one of the tests generates a taskinstance from dag.tasks[0] dynamically and then runs the task in which case there's actually no dagrun. I trapped for that and it works as expected. Possibly the test should be rewritten instead.
Thanks,
Ben
…--
Ben Tallman | Senior Technologist - Google Cloud | tallmanb@google.com | 503-680-5709
Sent with help from my rotary phone...
On Wed, Jan 4, 2017 at 8:05 AM -0800, "bolkedebruin" <notifications@github.com> wrote:
@btallman I need to look in to your comment on backfills, as in master backfills actually create dag runs. Can you elaborate a bit further what you think went wrong?
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub, or mute the thread.
|
|
Gotcha. That makes sense. |
bolkedebruin
left a comment
There was a problem hiding this comment.
Some minor things. I will start integration testing now.
| DagRun.execution_date.desc() | ||
| ).first() | ||
|
|
||
| @provide_session |
There was a problem hiding this comment.
I'm not sure if I understand the difference between get_previous_dagrun and get_previous_scheduled_dagrun . Should get_previous_dagrun also not return the same as get_previous_scheduled_dagrun? Or do you want to distinguish between backfilled and intervalled dag runs?
Please rename to "get_previous_scheduled" it is already part of DagRun.
| dag = self.task.dag | ||
| if dag: | ||
| dr = self.get_dagrun(session=session) | ||
| if not dr: |
There was a problem hiding this comment.
I understand why you are doing this, although the comment does not reflect the requirement I think. Please mark it as FIXME (it should be removed in the future / throw an exception)
| if not previous_scheduled_date: | ||
| return None | ||
| else: | ||
| return TaskInstance(task=self.task, execution_date=previous_scheduled_date) |
There was a problem hiding this comment.
This means that whether or not there is a "previous_scheduled_date" (i.e. there won't be one by convention before start_date) one always gets a TaskInstance. I don't think that should be the case.
|
|
||
| return self.dag | ||
|
|
||
| @provide_session |
There was a problem hiding this comment.
please rename to "get_previous", it is already part of DagRun (or even make it a property)
| authenticate = true | ||
| max_threads = 2 | ||
| catchup_by_default = True | ||
| scheduler_zombie_task_threshold = 300 |
There was a problem hiding this comment.
This is a duplicate from above
|
In initial testing it seems to behave quite ok (thats an understatement for currently flawless) |
|
What's left (apart from the minor changes) is to make sure it is documented, maybe with a small example. |
|
Yes, I had it documented, but that fell out when I moved it to catchup...
Totally forgot it.
Thanks,
Ben
…--
Ben Tallman - 503.680.5709
On Fri, Jan 6, 2017 at 6:13 AM, bolkedebruin ***@***.***> wrote:
What's left (apart from the minor changes) is to make sure it is
documented, maybe with a small example.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#1830 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AAKnfvyZtgiaEzhBeDjyqsXh03sGHN3Oks5rPkwbgaJpZM4KRoW8>
.
|
|
Hey @btallman can you add the documentation, I would really like to merge this and get it into 1.8. (And the minor nits) |
f296939 to
de0c044
Compare
|
Done.
Thanks,
Ben
…--
Ben Tallman - 503.680.5709
On Tue, Jan 10, 2017 at 11:53 PM, bolkedebruin ***@***.***> wrote:
Hey @btallman <https://github.com/btallman> can you add the
documentation, I would really like to merge this and get it into 1.8.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#1830 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AAKnflyk1p7cVbLJVMQ3t7kZkVeGLL0tks5rRIqVgaJpZM4KRoW8>
.
|
|
BTW - Although this failed tests on here, it succeeded on my fork... I'll re-push a NON update to force a re-test I guess. - https://travis-ci.org/btallman/incubator-airflow |
Added a dag.catchup option and modified the scheduler to look at the value when scheduling DagRuns (by moving dag.start_date up to dag.previous_schedule), and added a config option catchup_by_default (defaults to True) that allows users to set this to False for all dags modifying the existing DAGs In addition, we added a test to jobs.py (test_dag_catchup_option)
de0c044 to
d2a5f46
Compare
|
That seems to be an error in an assumption on one of the tests. Tests were executed at 23 hours. The test was doing now.hour() + 1, while hours should stay within 0..23 . Will merge. Thanks a lot @btallman ! |
Added a dag.catchup option and modified the scheduler to look at the value when scheduling DagRuns (by moving dag.start_date up to dag.previous_schedule), and added a config option catchup_by_default (defaults to True) that allows users to set this to False for all dags modifying the existing DAGs In addition, we added a test to jobs.py (test_dag_catchup_option) Closes apache#1830 from btallman/NoBackfill_clean_feature


Dear Airflow Maintainers,
Please accept this PR that addresses the following issues:
Add Support for dag.backfill=(True|False) Option
Added a dag.backfill option and modified the scheduler to look at
the value when scheduling DagRuns (by moving dag.start_date up to
dag.previous_schedule), and added a config option scheduler.backfill_by_default (defaults to True)
that allows users to set this to False for all dags modifying the existing DAGs
In addition, we added a test to jobs.py (test_dag_backfill_option)