More technical details:
https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#how-to-communicate

On Tue, Mar 22, 2022 at 5:03 PM Jarek Potiuk <ja...@potiuk.com> wrote:

> When there are differing opinions but seems that there is a favourable
> option someone (actually anyone)  might call for a vote
> https://www.apache.org/foundation/voting.html#votes-on-code-modification
>
> For such votes, committers have binding votes. -1 is a veto (usually needs
> to be justified) and kills the proposal unless the person who vetoed will
> change their mind.
>
> J.
>
> On Tue, Mar 22, 2022 at 4:53 PM Philippe Lanoe <pla...@cloudera.com.invalid>
> wrote:
>
>> I agree with Jarek in the sense that the DAG developer **should** know
>> when the DAG should start, however in practice for time-based scheduling it
>> can be cumbersome to maintain especially:
>> - everytime the jobs evolves and gets updated / new version
>> - when developers have to maintain hundreds/thousands of independent
>> jobs, keeping track of start_date for each of them can be difficult
>>
>> Not to mention that many companies do not have state-of-the-art CI/CD
>> processes which could allow them to dynamically change the start date. In
>> many cases when a change is made to a job, the developers simply want to
>> update the job and the next run will take it into account.
>>
>> I also agree with Collin and Constance that the "run last interval" is a
>> valid use case and therefore this parameter could accept three values to
>> handle all of these cases.
>> However I would suggest:
>>
>> Catchup=True : run all intervals
>> Catchup=False: do not run any past interval
>> Catchup="Last Interval" (or any better name :))
>>
>> I know that the DAG authors who relied on Catchup=False to run the last
>> interval will need to adjust their DAG but if added a third option not to
>> trigger any run then the DAG authors who relied on catchup=False + set
>> start date will also need to update their DAG to have the proper value. And
>> in my opinion when I read Catchup=False the natural way of reading it is
>> "no catchup", therefore it would be better to fix it in the right direction.
>>
>> What is the next step here? Who can decide / approve such a new feature
>> request?
>>
>> Thanks,
>> Philippe
>>
>> On Mon, Mar 21, 2022 at 4:05 PM Constance Martineau
>> <consta...@astronomer.io.invalid> wrote:
>>
>>> I've had a variation of this debate a few times, and the behaviour you
>>> find intuitive in my opinion comes down to your background (software
>>> engineer vs data engineer vs BI developer vs data analyst), industry
>>> standards, and the scope of responsibility DAG authors have at your
>>> organization. My vote is to extend the catchup setting to either run all
>>> intervals (catchup=True today), run the most recent interval (catchup=False
>>> today) or schedule the next interval. I have seen organizations where both
>>> would be beneficial depending on the data pipeline in question.
>>>
>>> All of Alex's points are why I think we at least need the option.
>>>
>>> I came from an institutional investor, and we had plenty of DAGs that
>>> ran daily, weekly, monthly, quarterly and yearly.
>>>
>>> Many financial analysts - who were not DAG authors themselves - would
>>> have access to the Airflow Webserver in order to rerun tasks. They do not
>>> have the ability to adjust the start_date. During Audit season, it was
>>> common to see yearly dags being run for earlier years. To support this,
>>> means we needed to implement a start date for an earlier year. Saw DAG
>>> authors deal with this in two ways: Set the start_date to first day of
>>> prior year to get the DAG out and let it run, then modify the start_date to
>>> something earlier or set the start_date to something earlier, watch the DAG
>>> and quickly update the state of the dag to success (or fail). One is better
>>> than the other (no fun explaining to an executive why reports were
>>> accidentally sent externally), but neither are great. Option 3 - setting
>>> the start_date between the data interval period and leaving it - always
>>> caused confusion with other stakeholders.
>>>
>>> A global default, and DAG-level option would have been amazing.
>>>
>>>
>>>
>>> On Sun, Mar 20, 2022 at 5:15 PM Collin McNulty
>>> <col...@astronomer.io.invalid> wrote:
>>>
>>>> While that’s true, I think there are often stakeholders that expect a
>>>> DAG to run only on the day for which it is scheduled. It’s pretty
>>>> straightforward for me to explain to non-technical stakeholders that “aw
>>>> shucks we deployed just a little too late for this week’s run, we’ll run it
>>>> manually to fix it”. On the contrary, explaining why a DAG that I said
>>>> would run on Tuesdays sent out an alert on a Friday to a VP of Finance is …
>>>> rough. I understand that Airflow does not make guarantees about when tasks
>>>> will execute, but I try to scale such that when a task can start and when
>>>> it does start are close enough to not have to explain the difference to
>>>> other stakeholders.
>>>>
>>>> Editing start_date can also be tough in some conditions. If I’m baking
>>>> a DAG into an image, using build-once-deploy-to-many CI/CD, and testing in
>>>> a lower environment for longer than the interval between runs, I’m toast on
>>>> setting the start_date to avoid what I consider a spurious run. That’s a
>>>> lot of “ands” but I think it’s a fairly common set of circumstances we
>>>> should support.
>>>>
>>>> Collin McNulty
>>>>
>>>>
>>>>
>>>> On Sun, Mar 20, 2022 at 3:12 PM Jarek Potiuk <ja...@potiuk.com> wrote:
>>>>
>>>>> Good. Love some mental stretching :).
>>>>>
>>>>> I believe you should **not** base the time of your run on the time it
>>>>> is released. Should not the DAG author know when there is a "start date"
>>>>> planned for the DAG? Should the decision on when the DAG interval start be
>>>>> made on combination of both start date in the dag **and** the time of not
>>>>> only when it's merged, but actually when airflow first **parses** the DAG.
>>>>> Not even mentioning the time zone issues.
>>>>>
>>>>> Imagine you case when DAG is merged 5 minutes between the midnight
>>>>> Mon/Tue and you have many DAGs. So many that parsing all the DAGs can take
>>>>> 20 minutes. Then the fact if your DAG runs this interval or that depends
>>>>> not even on the decision of when it is merged but also how long it takes
>>>>> Airflow to get to parse your DAG for the first time.
>>>>>
>>>>> Sounds pretty crazy :).
>>>>>
>>>>> J.
>>>>>
>>>>>
>>>>> On Sun, Mar 20, 2022 at 9:02 PM Collin McNulty
>>>>> <col...@astronomer.io.invalid> wrote:
>>>>>
>>>>>> Jarek,
>>>>>>
>>>>>> I tend to agree with you on this, but let me play devil’s advocate.
>>>>>> If I have a DAG that runs a report every Tuesday, I might want it to run
>>>>>> every Tuesday starting whenever I am able to release the DAG. But if I
>>>>>> release on a Friday, I don’t want it to try to run “for” last Tuesday. In
>>>>>> this case, the correct start_date for the dag is the day I release the 
>>>>>> DAG,
>>>>>> but I don’t know this date ahead of time and it differs per environment.
>>>>>> Doing this properly seems doable with a CD process that edits the DAG to
>>>>>> insert the start_date, but that’s fairly sophisticated tooling for a
>>>>>> scenario that I imagine is quite common.
>>>>>>
>>>>>> Collin McNulty
>>>>>>
>>>>>> On Sun, Mar 20, 2022 at 1:55 PM Jarek Potiuk <ja...@potiuk.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Once again - why is it bad to set a start_date in the future, when -
>>>>>>> well - you **actually** want to run the first interval in the future
>>>>>>> ?
>>>>>>> What prevents you from setting the start-date to be a fixed time in
>>>>>>> the future, where the start date is within the interval you want to
>>>>>>> start first? Is it just "I do not want to specify conveniently
>>>>>>> whatever past date will be easy to type?"
>>>>>>> If this is the only reason,  then it has a big drawback - because
>>>>>>> "start_date" is **actually** supposed to be the piece of metadata for
>>>>>>> the DAG that will tell you what was the intention of the DAG writer
>>>>>>> on
>>>>>>> when to start it. And precisely one that allows you to start things
>>>>>>> in
>>>>>>> the future.
>>>>>>>
>>>>>>> Am I missing something?
>>>>>>>
>>>>>>> On Sun, Mar 20, 2022 at 7:42 PM Larry Komenda
>>>>>>> <avoicelikerunningwa...@gmail.com> wrote:
>>>>>>> >
>>>>>>> > Alex, that's a good point regarding the need to run a DAG for the
>>>>>>> most recent schedule interval right away. I hadn't thought of that 
>>>>>>> scenario
>>>>>>> as I haven't needed to build a DAG with that large of a scheduling gap. 
>>>>>>> In
>>>>>>> that case I agree with you - it seems like it would make more sense to 
>>>>>>> make
>>>>>>> this configurable.
>>>>>>> >
>>>>>>> > Perhaps there could be an additional DAG-level parameter that
>>>>>>> could be set alongside "catchup" to control this behavior. Or there 
>>>>>>> could
>>>>>>> be a new parameter that could eventually replace "catchup" that 
>>>>>>> supported 3
>>>>>>> options - "catchup", "run most recent interval only", and "run next
>>>>>>> interval only".
>>>>>>> >
>>>>>>> > On Sat, Mar 19, 2022 at 1:02 PM Alex Begg <alex.b...@gmail.com>
>>>>>>> wrote:
>>>>>>> >>
>>>>>>> >> I would not consider it a bug to have the latest data interval
>>>>>>> run when you enable a DAG that is set to catchup=False.
>>>>>>> >>
>>>>>>> >> I have legitimate use for that feature by having my production
>>>>>>> environment have catchup_by_default=True but my lower environments are
>>>>>>> using catchup_by_default=False, meaning if I want to test the DAG 
>>>>>>> behavior
>>>>>>> as scheduled in a lower environment I can just enable the DAG.
>>>>>>> >>
>>>>>>> >> For example, in a staging environment if I need to test out the
>>>>>>> functionality of a DAG that was scheduled for @monthly and there was no 
>>>>>>> way
>>>>>>> to test the most recent data interval, than to test a true data 
>>>>>>> interval of
>>>>>>> the DAG it could be many days, even weeks until they will occur.
>>>>>>> >>
>>>>>>> >> Triggering a DAG won’t run the latest data interval, it will use
>>>>>>> the current time as the logical_date, right? So that will won’t let me 
>>>>>>> test
>>>>>>> a single as scheduled data interval. So in that @monthly senecio it 
>>>>>>> will be
>>>>>>> impossible for me to test the functionality of a single data interval
>>>>>>> unless I wait multiple weeks.
>>>>>>> >>
>>>>>>> >> I see there could be a desire to not run the latest data interval
>>>>>>> and just start with whatever full interval follows the DAG being turned 
>>>>>>> on.
>>>>>>> However I think that should be configurable, not fixed permanently.
>>>>>>> >>
>>>>>>> >> Alternatively it could be ideal to have a way to trigger a
>>>>>>> specific run for a catchup=False DAG that just got enabled by adding a 
>>>>>>> 3d
>>>>>>> option to the trigger button drop down to trigger a past scheduled run.
>>>>>>> Then in that dialog the form can default to the most recent full data
>>>>>>> interval but then let you also specify a specific past interval based on
>>>>>>> the DAG's schedule. I often had to debug a DAG in production and I 
>>>>>>> wanted
>>>>>>> to trigger a specific past data interval, not just the most recent.
>>>>>>> >>
>>>>>>> >> Alex Begg
>>>>>>> >>
>>>>>>> >> On Thu, Mar 17, 2022 at 4:58 PM Larry Komenda <
>>>>>>> avoicelikerunningwa...@gmail.com> wrote:
>>>>>>> >>>
>>>>>>> >>> I agree with this. I'd much rather have to trigger a single
>>>>>>> manual run the first time I enable a DAG than to either wait to enable
>>>>>>> until after I want it to run or by editing the start_date of the DAG 
>>>>>>> itself.
>>>>>>> >>>
>>>>>>> >>> I'd be in favor of adjusting this behavior either permanently or
>>>>>>> by a configuration.
>>>>>>> >>>
>>>>>>> >>> On Fri, Mar 4, 2022 at 3:00 PM Philippe Lanoe
>>>>>>> <pla...@cloudera.com.invalid> wrote:
>>>>>>> >>>>
>>>>>>> >>>> Hello Daniel,
>>>>>>> >>>>
>>>>>>> >>>> Thank you for your answer. In your example, as I experienced,
>>>>>>> the first run would not be 2010-01-01 but 2022-03-03, 00:00:00 (it is
>>>>>>> currently March 4 - 21:00 here), which is the execution date 
>>>>>>> corresponding
>>>>>>> to the start of the previous data interval, but the result is the same: 
>>>>>>> an
>>>>>>> undesired dag run. (For instance, in case of cron schedule '00 22 * * 
>>>>>>> *',
>>>>>>> one dagrun would be started immediately with execution date of 
>>>>>>> 2022-03-02,
>>>>>>> 22:00:00)
>>>>>>> >>>>
>>>>>>> >>>> I also agree with you that it could be categorized as a bug and
>>>>>>> I would also vote for a fix.
>>>>>>> >>>>
>>>>>>> >>>> Would be great to have the feedback of others on this.
>>>>>>> >>>>
>>>>>>> >>>> On Fri, Mar 4, 2022 at 6:17 PM Daniel Standish
>>>>>>> <daniel.stand...@astronomer.io.invalid> wrote:
>>>>>>> >>>>>
>>>>>>> >>>>> You are saying, when you turn on for the first time a dag with
>>>>>>> e.g. @daily schedule, and catchup = False, if start date is 2010-01-01,
>>>>>>> then it would run first the 2010-01-01 run, then the current run 
>>>>>>> (whatever
>>>>>>> yesterday is)?  That sounds familiar.
>>>>>>> >>>>>
>>>>>>> >>>>> Yeah I don't like that behavior.  I agree that, as you say,
>>>>>>> it's not the intuitive behavior.  Seems it could reasonably be 
>>>>>>> categorized
>>>>>>> as a bug.  I'd prefer we just "fix" it rather than making it 
>>>>>>> configurable.
>>>>>>> But some might have concerns re backcompat.
>>>>>>> >>>>>
>>>>>>> >>>>> What do others think?
>>>>>>> >>>>>
>>>>>>> >>>>>
>>>>>>>
>>>>>> --
>>>>>>
>>>>>> Collin McNulty
>>>>>> Lead Airflow Engineer
>>>>>>
>>>>>> Email: col...@astronomer.io <john....@astronomer.io>
>>>>>> Time zone: US Central (CST UTC-6 / CDT UTC-5)
>>>>>>
>>>>>>
>>>>>> <https://www.astronomer.io/>
>>>>>>
>>>>> --
>>>>
>>>> Collin McNulty
>>>> Lead Airflow Engineer
>>>>
>>>> Email: col...@astronomer.io <john....@astronomer.io>
>>>> Time zone: US Central (CST UTC-6 / CDT UTC-5)
>>>>
>>>>
>>>> <https://www.astronomer.io/>
>>>>
>>>
>>>
>>> --
>>>
>>> Constance Martineau
>>> Product Manager
>>>
>>> Email: consta...@astronomer.io
>>> Time zone: US Eastern (EST UTC-5 / EDT UTC-4)
>>>
>>>
>>> <https://www.astronomer.io/>
>>>
>>>

Reply via email to