For my problem, and the one mentioned earlier for those of us in the financial 
world dealing with holidays this could be a solid solution. 
For my example below you could derive DAG and add a max_interval property that 
is a timedelta and if the delta between dttm and the value coming out of 
following/previous schedule is greater than that property, return dttm + 
max_interval. 
You might actually be able to do it without adding a property and just look at 
the delta for other runs and derive it from that.
For the holidays, one could probably just check if the return value of a 
super() following/previous schedule is in your holiday list, and then just 
super following/previous_schedule again until it’s not a holiday. 

While these are somewhat orthogonal to whether this PR should be merged, it is 
a helpful conversation for dealing with funky scheduling logic. 
Thanks for the idea Max!

-James


> On Sep 6, 2019, at 1:40 AM, Maxime Beauchemin <maximebeauche...@gmail.com> 
> wrote:
> 
> Just had a thought and looked a tiny bit at the source code to assess
> feasibility, but it seems like you could just derive the DAG class and
> override `previous_schedule` and `following_schedule` methods. The
> signature of both is you get a `datetime.datetime` and have to return
> another. It's pretty easy to put your arbitrarily complex logic in there.
> 
> There may be a few hiccups to sort out things like like
> `airflow.utils.dates.date_range` (where duplicated time-step logic exist)
> to make sure that all time-step logic aligns with these two methods I just
> mentioned, but from that point it could be become the official way to
> incorporate funky date-step logic.
> 
> Max
> 
> On Wed, Sep 4, 2019 at 12:54 PM Daniel Standish <dpstand...@gmail.com>
> wrote:
> 
>> Re:
>> 
>>> For example, if I need to run a DAG every 20 minutes between 8 AM and 4
>>> PM...
>> 
>> 
>> This makes a lot of sense!  Thank you for providing this example.  My
>> initial thought of course is "well can't you just set it to run */20
>> between 7:40am and 3:40pm," but I don't think that is possible in cron.
>> Which is why you have to do hacky shit as you've said and it indeed sounds
>> terrible.  I never had to achieve a schedule like this, and yeah -- it
>> should not be this hard.
>> 
>> Re:
>> 
>>> I can’t see how adding a property to Dagrun that is essentially
>>> identical to next_execution_date would add any benefit.
>> 
>> That's why i was like what the hell is the point of this thing!   I thought
>> it was just purely cosmetic, so that in effect "execution_date" would
>> optionally mean "run_date".
>> 
>> 
>> 
>> 
>> 
>> 
>> 
>>> On Wed, Sep 4, 2019 at 12:10 PM James Coder <jcode...@gmail.com> wrote:
>>> 
>>> I can’t see how adding a property to Dagrun that is essentially
>>> identical to next_execution_date would add any benefit. The way I see
>>> it the issue at hand here is not the availability of dates. There are
>>> plenty of options in the template context for dates before and after
>>> execution date. My view point is the problem this is trying to solve
>>> is that waiting until the right edge of an interval has passed to
>>> schedule a dag run has some shortcomings. Mainly that if your
>>> intervals vary in length you are forced to put scheduling logic that
>>> should reside in the scheduler in your DAGs. For example, if I need to
>>> run a DAG every 20 minutes between 8 AM and 4 PM, in it's current
>>> form, the scheduler won't schedule that 4PM run until 8 AM the next
>>> day. "Just use next_execution_date" you say, well that's all well and
>>> good between 8AM and 3:40 PM, but when 4:01 PM rolls around and you
>>> don't have the results because they won't be available until after 8
>>> the next day, that doesn't sound so good, does it? In order to work
>>> around this, you have to add additional runs and short circuit
>>> operators over and over. It's a hassle.  Allowing for scheduling dags
>>> at the left edge of an interval and allowing it to behave more like
>>> cron, where it runs at the time specified, not schedule + interval,
>>> would make things much less complicated for users like myself that
>>> can't always wait until the right edge of the interval.
>>> 
>>> 
>>> James Coder
>>> 
>>>> On Sep 3, 2019, at 11:14 PM, Daniel Standish <dpstand...@gmail.com>
>>> wrote:
>>>> 
>>>> What if we merely add a property "run_date" to DagRun?  At present
>>>> this would be essentially same as "next_execution_date".
>>>> 
>>>> Then no change to scheduler would be required, and no new dag parameter
>>> or
>>>> config.  Perhaps you could add a toggle to the DAGs UI view that lets
>> you
>>>> choose whether to display "last run" by "run_date" or "execution_date".
>>>> 
>>>> If you want your dags to be parameterized by the date when they meant
>> to
>>> be
>>>> run -- as opposed to their implicit interval-of-interest -- then you
>> can
>>>> reference "run_date".
>>>> 
>>>> One potential source of confusion with this is backfilling: what does
>>>> "run_date" mean in the context of a backfill?  You could say it means
>>>> essentially "initial run date", i.e. "do not run before date", i.e.
>> "run
>>>> after date" or "run-at date".  So, for a daily, job the 2019-01-02
>>>> "run_date" corresponds to a 2019-01-01 execution_date.  This makes
>> sense
>>>> right?
>>>> 
>>>> Perhaps in the future, the relationship between "run_date" and
>>>> "execution_date" can be more dynamic.  Perhaps in the future we rename
>>>> "execution_date" for clarity, or to be more generic.  But it makes
>> sense
>>>> that a dag run will always have a run date, so it doesn't seem like a
>>>> terrible idea to add a property representing this.
>>>> 
>>>> Would this meet the goals of the PR?
>>>> 
>>>> 
>>>> 
>>>> 
>>>> On Wed, Aug 28, 2019 at 11:50 AM James Meickle
>>>> <jmeic...@quantopian.com.invalid> wrote:
>>>> 
>>>>> Totally agree with Daniel here. I think that if we implement this
>>> feature
>>>>> as proposed, it will actively discourage us from implementing a better
>>>>> data-aware feature that would remain invisible to most users while
>>> neatly
>>>>> addressing a lot of edge cases that currently require really ugly
>>> hacks. I
>>>>> believe that having more data awareness features in Airflow (like the
>>> data
>>>>> lineage work, or other metadata integrations) is worth investing in if
>>> we
>>>>> can do it without too much required user-facing complexity. The
>> Airflow
>>>>> project isn't a full data warehouse suite but it's also not just "cron
>>> with
>>>>> a UI", so we should try to be pragmatic and fit in power-user features
>>>>> where we can do so without compromising the project's overall goals.
>>>>> 
>>>>> On Wed, Aug 28, 2019 at 2:24 PM Daniel Standish <dpstand...@gmail.com
>>> 
>>>>> wrote:
>>>>> 
>>>>>> I am just thinking there is the potential for a more comprehensive
>>>>>> enhancement here, and I worry that this is a band-aid that, like all
>>> new
>>>>>> features has the potential to constrain future options.  It does not
>>> help
>>>>>> us to do anything we cannot already do.
>>>>>> 
>>>>>> The source of this problem is that scheduling and
>> interval-of-interest
>>>>> are
>>>>>> mixed together.
>>>>>> 
>>>>>> My thought is there may be a way to separate scheduling and
>>>>>> interval-of-interest to uniformly resolve "execution_date" vs
>>> "run_date"
>>>>>> confusion.  We could make *explicit* instead of *implicit* the
>>>>> relationship
>>>>>> between run_date *(not currently a concept in airflow)* and
>>>>>> "interval-of-interest" *(currently represented by execution_date)*.
>>>>>> 
>>>>>> I also see in this the potential to unlock some other improvements:
>>>>>> * support a greater diversity of incremental processes
>>>>>> * allow more flexible backfilling
>>>>>> * provide better views of data you have vs data you don't.
>>>>>> 
>>>>>> The canonical airflow job is date-partitioned idempotent data pull.
>>> Your
>>>>>> interval of interest is from execution_date to execution_date + 1
>>>>>> interval.  Schedule_interval is not just the scheduling cadence but
>> it
>>> is
>>>>>> also your interval-of-interest partition function.   If that doesn't
>>> work
>>>>>> for your job, you set catchup=False and roll your own.
>>>>>> 
>>>>>> What if there was a way to generalize?  E.g. could we allow for more
>>>>>> flexible partition function that deviated from scheduler cadence?
>> E.g.
>>>>>> what if your interval-of-interest partitions could be governed by
>> "min
>>> 1
>>>>>> day, max 30 days".  Then on on-going basis, your daily loads would
>> be a
>>>>>> range of 1 day but then if server down for couple days, this could be
>>>>>> caught up in one task and if you backfill it could be up to 30-day
>>>>> batches.
>>>>>> 
>>>>>> Perhaps there is an abstraction that could be used by a greater
>>> diversity
>>>>>> of incremental processes.  Such a thing could support a nice "data
>>>>>> contiguity view". I imagine a horizontal bar that is solid where we
>>> have
>>>>>> the data and empty where we don't.  Then you click on a "missing"
>>> section
>>>>>> and you can  trigger a backfill task with that date interval
>> according
>>> to
>>>>>> your partitioning rules.
>>>>>> 
>>>>>> I can imagine using this for an incremental job where each time we
>> pull
>>>>> the
>>>>>> new data since last time; in the `execute` method the operator could
>>> set
>>>>>> `self.high_watermark` with the max datetime processed.  Or maybe a
>>>>> callback
>>>>>> function could be used to gather this value.  This value could be
>> used
>>> in
>>>>>> next run, and cold be depicted in a view.
>>>>>> 
>>>>>> Default intervals of interest could be status quo -- i.e. partitions
>>>>> equal
>>>>>> to schedule interval -- but could be overwritten using templating or
>>>>>> callbacks or setting it during `execute`.
>>>>>> 
>>>>>> So anyway, I don't have a master plan all figured out.  But I think
>>> there
>>>>>> is opportunity in this area for more comprehensive enhancement that
>>> goes
>>>>>> more directly at the root of the problem.
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> On Tue, Aug 27, 2019 at 10:00 AM Maxime Beauchemin <
>>>>>> maximebeauche...@gmail.com> wrote:
>>>>>> 
>>>>>>> How about an alternative approach that would introduce 2 new keyword
>>>>>>> arguments that are clear (something like, but maybe better than
>>>>>>> `period_start_dttm`, `period_end_dttm`) and leave `execution_date`
>>>>>>> unchanged, but plan it's deprecation. As a first step
>> `execution_date`
>>>>>>> would be inferred from the new args, and warn about deprecation when
>>>>>> used.
>>>>>>> 
>>>>>>> Max
>>>>>>> 
>>>>>>> On Tue, Aug 27, 2019 at 9:26 AM Bolke de Bruin <bdbr...@gmail.com>
>>>>>> wrote:
>>>>>>> 
>>>>>>>> Execution date is execution date for a dag run no matter what.
>> There
>>>>> is
>>>>>>> no
>>>>>>>> end interval or start interval for a dag run. The only time this is
>>>>>>>> relevant is when we calculate the next or previous dagrun.
>>>>>>>> 
>>>>>>>> So I don't Daniels rationale makes sense (?)
>>>>>>>> 
>>>>>>>> Sent from my iPhone
>>>>>>>> 
>>>>>>>>> On 27 Aug 2019, at 17:40, Philippe Gagnon <philgagn...@gmail.com>
>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>> I agree with Daniel's rationale but I am also worried about
>>>>> backwards
>>>>>>>>> compatibility as this would perhaps be the most disruptive
>> breaking
>>>>>>>> change
>>>>>>>>> possible. I think maybe we should write down the different options
>>>>>>>>> available to us (AIP?) and call for a vote. What does everyone
>>>>> think?
>>>>>>>>> 
>>>>>>>>>> On Tue, Aug 27, 2019 at 9:25 AM James Coder <jcode...@gmail.com>
>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>> Can't execution date can already mean different things depending
>>>>> on
>>>>>> if
>>>>>>>> the
>>>>>>>>>> dag run was initiated via the scheduler or manually via command
>>>>>>>> line/API?
>>>>>>>>>> I agree that making it consistent might make it easier to explain
>>>>> to
>>>>>>> new
>>>>>>>>>> users, but should we exchange that for breaking pretty much every
>>>>>>>> existing
>>>>>>>>>> dag by re-defining what execution date is?
>>>>>>>>>> -James
>>>>>>>>>> 
>>>>>>>>>> On Mon, Aug 26, 2019 at 11:12 PM Daniel Standish <
>>>>>>> dpstand...@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> To Daniel’s concerns, I would argue this is not a change to
>>>>> what a
>>>>>>> dag
>>>>>>>>>>> run
>>>>>>>>>>>> is, it is rather a change to WHEN that dag run will be
>>>>> scheduled.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> Execution date is part of the definition of a dag_run; it is
>>>>>> uniquely
>>>>>>>>>>> identified by an execution_date and dag_id.
>>>>>>>>>>> 
>>>>>>>>>>> When someone asks what is a dag_run, we should be able to
>> provide
>>>>>> an
>>>>>>>>>>> answer.
>>>>>>>>>>> 
>>>>>>>>>>> Imagine trying to explain what a dag run is, when execution_date
>>>>>> can
>>>>>>>> mean
>>>>>>>>>>> different things.
>>>>>>>>>>>  Admin: "A dag run is an execution_date and a dag_id".
>>>>>>>>>>>  New user: "Ok. Clear as a bell. What's an execution_date?"
>>>>>>>>>>>  Admin: "Well, it can be one of two things.  It *could* be when
>>>>>> the
>>>>>>>>>> dag
>>>>>>>>>>> will be run... but it could *also* be 'the time when dag should
>>>>> be
>>>>>>> run
>>>>>>>>>>> minus one schedule interval".  It depends on whether you choose
>>>>>> 'end'
>>>>>>>> or
>>>>>>>>>>> 'start' for 'schedule_interval_edge.'  If you choose 'start'
>> then
>>>>>>>>>>> execution_date means 'when dag will be run'.  If you choose
>> 'end'
>>>>>>> then
>>>>>>>>>>> execution_date means 'when dag will be run minus one interval.'
>>>>> If
>>>>>>> you
>>>>>>>>>>> change the parameter after some time, then we don't necessarily
>>>>>> know
>>>>>>>> what
>>>>>>>>>>> it means at all times".
>>>>>>>>>>> 
>>>>>>>>>>> Why would we do this to ourselves?
>>>>>>>>>>> 
>>>>>>>>>>> Alternatively, we can give dag_run a clear, unambiguous meaning:
>>>>>>>>>>> * dag_run is dag_id + execution_date
>>>>>>>>>>> * execution_date is when dag will be run (notwithstanding
>>>>> scheduler
>>>>>>>>>> delay,
>>>>>>>>>>> queuing)
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> Execution_date is defined as "run-at date minus 1 interval".
>> The
>>>>>>>>>>> assumption in this is that you tasks care about this particular
>>>>>> date.
>>>>>>>>>>> Obviously this makes sense for some tasks but not for others.
>>>>>>>>>>> 
>>>>>>>>>>> I would prop
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> On Sat, Aug 24, 2019 at 5:08 AM James Coder <
>> jcode...@gmail.com
>>>>>> 
>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> I think this is a great improvement and should be merged. To
>>>>>>> Daniel’s
>>>>>>>>>>>> concerns, I would argue this is not a change to what a dag run
>>>>> is,
>>>>>>> it
>>>>>>>>>> is
>>>>>>>>>>>> rather a change to WHEN that dag run will be scheduled.
>>>>>>>>>>>> I had implemented a similar change in my own version but
>>>>>> ultimately
>>>>>>>>>>> backed
>>>>>>>>>>>> so I didn’t have to patch after each new release. In my opinion
>>>>>> the
>>>>>>>>>> main
>>>>>>>>>>>> flaw in the current scheduler, and I have brought this up
>>>>> before,
>>>>>> is
>>>>>>>>>> when
>>>>>>>>>>>> you don’t have a consistent schedule interval (e.g. only run
>>>>> M-F).
>>>>>>>>>> After
>>>>>>>>>>>> backing out the “schedule at interval start” I had to switch to
>>>>> a
>>>>>>>> daily
>>>>>>>>>>>> schedule and go through and put a short circuit operator in
>> each
>>>>>> of
>>>>>>> my
>>>>>>>>>>> M-F
>>>>>>>>>>>> dags to get the behavior that I wanted. This results in putting
>>>>>>>>>>> scheduling
>>>>>>>>>>>> logic inside the dag, when scheduling logic should be in the
>>>>>>>> scheduler.
>>>>>>>>>>>> 
>>>>>>>>>>>> -James
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> On Aug 23, 2019, at 3:14 PM, Daniel Standish <
>>>>>> dpstand...@gmail.com
>>>>>>>> 
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Re
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> What are people's feelings on changing the default execution
>>>>> to
>>>>>>>>>>> schedule
>>>>>>>>>>>>>> interval start
>>>>>>>>>>>>> 
>>>>>>>>>>>>> and
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I'm in favor of doing that, but then exposing new variables
>> of
>>>>>>>>>>>>>> "interval_start" and "interval_end", etc. so that people
>> write
>>>>>>>>>>>>>> clearer-looking at-a-glance DAGs
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> While I am def on board with the spirit of this PR, I would
>>>>> vote
>>>>>> we
>>>>>>>>>> do
>>>>>>>>>>>> not
>>>>>>>>>>>>> accept this PR as is, because it cements a confusing option.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> *What is the right representation of a dag run?*
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Right now the representation is "dag run-at date minus 1
>>>>>> interval".
>>>>>>>>>> It
>>>>>>>>>>>>> should just be "dag run-at date".
>>>>>>>>>>>>> 
>>>>>>>>>>>>> We don't need to address the question of whether execution
>> date
>>>>>> is
>>>>>>>>>> the
>>>>>>>>>>>>> start or the end of an interval; it doesn't matter.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> In all cases, a given dag run will be targeted for *some*
>>>>> initial
>>>>>>>>>>> "run-at
>>>>>>>>>>>>> time"; so *that* should be the time that is part of the PK of
>> a
>>>>>> dag
>>>>>>>>>>> run,
>>>>>>>>>>>>> and *that *is the time that should be exposed as the dag run
>>>>>>>>>> "execution
>>>>>>>>>>>>> date"
>>>>>>>>>>>>> 
>>>>>>>>>>>>> *Interval of interest is not a dag_run attribute*
>>>>>>>>>>>>> 
>>>>>>>>>>>>> We also mix in this question of the date interval that the
>>>>>> *tasks*
>>>>>>>>>> are
>>>>>>>>>>>>> interested in.  But the *dag run* need not concern itself with
>>>>>> this
>>>>>>>>>> in
>>>>>>>>>>>> any
>>>>>>>>>>>>> way.  That is for the tasks to figure out: if they happen to
>>>>> need
>>>>>>>>>> "dag
>>>>>>>>>>>>> run-at date," then they can reference that; if they want the
>>>>>> prior
>>>>>>>>>> one,
>>>>>>>>>>>> ask
>>>>>>>>>>>>> for the prior one.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Previously, I was in the camp that thought it was a great idea
>>>>> to
>>>>>>>>>>> rename
>>>>>>>>>>>>> "execution_date" to "period_start" or "interval_start".  But I
>>>>>> now
>>>>>>>>>>> think
>>>>>>>>>>>>> this is folly.  It invokes this question of the "interval of
>>>>>>>>>> interest"
>>>>>>>>>>> or
>>>>>>>>>>>>> "period of interest".  But the dag doesn't need to know
>>>>> anything
>>>>>>>>>> about
>>>>>>>>>>>>> that.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Within the same dag you may have tasks with different
>> intervals
>>>>>> of
>>>>>>>>>>>>> interest.  So why make assumptions in the dag; just give the
>>>>>> facts:
>>>>>>>>>>> this
>>>>>>>>>>>> is
>>>>>>>>>>>>> my run date; this is the prior run date, etc.  It would be a
>>>>>>>>>> regression
>>>>>>>>>>>>> from the perspective of providing accurate names.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> *Proposal*
>>>>>>>>>>>>> 
>>>>>>>>>>>>> So, I would propose we change "execution_date" to mean "dag
>>>>>> run-at
>>>>>>>>>>> date"
>>>>>>>>>>>> as
>>>>>>>>>>>>> opposed to "dag run-at date minus 1".  But we should do so
>>>>>> without
>>>>>>>>>>>>> reference to interval end or interval start.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> *Configurability*
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The more configuration options we have, the more noise there
>> is
>>>>>> as
>>>>>>> a
>>>>>>>>>>> user
>>>>>>>>>>>>> trying to understand how to use airflow, so I'd rather us not
>>>>>> make
>>>>>>>>>> this
>>>>>>>>>>>>> configurable at all.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> That said, perhaps a more clear and more explicit means making
>>>>>> this
>>>>>>>>>>>>> configurable would be to define an integer param
>>>>>>>>>>>>> "dag_run_execution_date_interval_offset", which would control
>>>>> how
>>>>>>>>>> many
>>>>>>>>>>>>> intervals back from actual "dag run-at date" the "execution
>>>>> date"
>>>>>>>>>>> should
>>>>>>>>>>>>> be.  (current behavior = 1, new behavior = 0).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> *Side note*
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Hopefully not to derail discussion: I think there are
>>>>> additional,
>>>>>>>>>>> related
>>>>>>>>>>>>> task attributes that may want to come into being: namely,
>>>>>>>>>> low_watermark
>>>>>>>>>>>> and
>>>>>>>>>>>>> high_watermark.  There is the potential, with attributes like
>>>>>> this,
>>>>>>>>>> for
>>>>>>>>>>>>> adding better out-of-the-box support for common data workflows
>>>>>> that
>>>>>>>>>> we
>>>>>>>>>>>> now
>>>>>>>>>>>>> need to use xcom for, namely incremental loads.  But I want to
>>>>>> give
>>>>>>>>>> it
>>>>>>>>>>>> more
>>>>>>>>>>>>> thought before proposing anything specific.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Fri, Aug 23, 2019 at 9:42 AM Jarek Potiuk <
>>>>>>>>>> jarek.pot...@polidea.com
>>>>>>>>>>>> 
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Good one Damian. I will have a list of issues that can be
>>>>>> possible
>>>>>>>>>> to
>>>>>>>>>>>>>> handle at the workshop, so that one goes there.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> J.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Principal Software Engineer
>>>>>>>>>>>>>> Phone: +48660796129
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> pt., 23 sie 2019, 11:09 użytkownik Shaw, Damian P. <
>>>>>>>>>>>>>> damian.sha...@credit-suisse.com> napisał:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I can't understate what a conceptual improvement this would
>>>>> be
>>>>>>> for
>>>>>>>>>>> the
>>>>>>>>>>>>>> end
>>>>>>>>>>>>>>> users of Airflow in our environment. I've written a lot of
>>>>> code
>>>>>>> so
>>>>>>>>>>> all
>>>>>>>>>>>>>> our
>>>>>>>>>>>>>>> configuration works like this anyway. But the UI still shows
>>>>>> the
>>>>>>>>>>>> Airflow
>>>>>>>>>>>>>>> dates which still to this day sometimes confuse me.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I'll be at the NY meet ups on Monday and Tuesday, maybe some
>>>>> of
>>>>>>> my
>>>>>>>>>>>> first
>>>>>>>>>>>>>>> PRs could be additional test cases around edge cases to do
>>>>> with
>>>>>>> DST
>>>>>>>>>>> and
>>>>>>>>>>>>>>> cron scheduling that I have concerns about :)
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> -----Original Message-----
>>>>>>>>>>>>>>> From: Ash Berlin-Taylor [mailto:a...@apache.org]
>>>>>>>>>>>>>>> Sent: Friday, August 23, 2019 6:50 AM
>>>>>>>>>>>>>>> To: dev@airflow.apache.org
>>>>>>>>>>>>>>> Subject: Setting to add choice of schedule at end or
>> schedule
>>>>>> at
>>>>>>>>>>> start
>>>>>>>>>>>> of
>>>>>>>>>>>>>>> interval
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> This has come up a few times before, someone has now opened
>> a
>>>>>> PR
>>>>>>>>>> that
>>>>>>>>>>>>>>> makes this a global+per-dag setting:
>>>>>>>>>>>>>>> https://github.com/apache/airflow/pull/5787 and it also
>>>>>> includes
>>>>>>>>>>> docs
>>>>>>>>>>>>>>> that I think does a good job of illustrating the two modes.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Does anyone object to this being merged? If no one says
>>>>>> anything
>>>>>>> by
>>>>>>>>>>>>>> midday
>>>>>>>>>>>>>>> on Tuesday I will take that as assent and will merge it.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> The docs from the PR included below.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> Ash
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Scheduled Time vs Execution Time
>>>>>>>>>>>>>>> ''''''''''''''''''''''''''''''''
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> A DAG with a ``schedule_interval`` will execute once per
>>>>>>> interval.
>>>>>>>>>> By
>>>>>>>>>>>>>>> default, the execution of a DAG will occur at the **end** of
>>>>>> the
>>>>>>>>>>>>>>> schedule interval.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> A few examples:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - A DAG with ``schedule_interval='@hourly'``: The DAG run
>>>>> that
>>>>>>>>>>>> processes
>>>>>>>>>>>>>>> 2019-08-16 17:00 will start running just after 2019-08-16
>>>>>>> 17:59:59,
>>>>>>>>>>>>>>> i.e. once that hour is over.
>>>>>>>>>>>>>>> - A DAG with ``schedule_interval='@daily'``: The DAG run
>> that
>>>>>>>>>>> processes
>>>>>>>>>>>>>>> 2019-08-16 will start running shortly after 2019-08-17
>> 00:00.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> The reasoning behind this execution vs scheduling behaviour
>>>>> is
>>>>>>> that
>>>>>>>>>>>>>>> data for the interval to be processed won't be fully
>>>>> available
>>>>>>>>>> until
>>>>>>>>>>>>>>> the interval has elapsed.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> In cases where you wish the DAG to be executed at the
>>>>> **start**
>>>>>>> of
>>>>>>>>>>> the
>>>>>>>>>>>>>>> interval, specify ``schedule_at_interval_end=False``, either
>>>>> in
>>>>>>>>>>>>>>> ``airflow.cfg``, or on a per-DAG basis.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>> 
>> ===============================================================================
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Please access the attached hyperlink for an important
>>>>>> electronic
>>>>>>>>>>>>>>> communications disclaimer:
>>>>>>>>>>>>>>> 
>>>>> http://www.credit-suisse.com/legal/en/disclaimer_email_ib.html
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>> 
>> ===============================================================================
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>> 
>> 

Reply via email to