I agree that for the crontab interval with `catchup=False`, the state_date does not make sense. However, the start_date is still very useful when having catchup=True, whose default value is `True`, https://github.com/apache/airflow/blob/main/airflow/config_templates/default_airflow.cfg#L989. If the stae_date defaults to None, this makes the dag not-portable, since the start_date could be different in different airflow envs.
If we want to default the state_date to None, we need some rules to let users know in some cases start_date cannot be None. Thanks, Ping On Mon, May 9, 2022 at 10:02 AM Jarek Potiuk <ja...@potiuk.com> wrote: > Coincidentally - this discussion in Github Discussions started just now > has a clear use cases when omitting start_date makes perfect sense: > https://github.com/apache/airflow/discussions/23594 > > On Mon, May 9, 2022 at 4:01 PM Bas Harenslak <b...@astronomer.io.invalid> > wrote: > >> I never understood the requirement for start_date — 99% of the use cases >> simply want to start from the time the DAG is first added and do not >> explicitly need to start on a certain date. There is certainly a use case >> for start_date, but defaulting to None would make more sense IMO, and we >> could internally register the “first added date” as a start date instead. >> >> Bas >> >> On 9 May 2022, at 09:35, Jarek Potiuk <ja...@potiuk.com> wrote: >> >> I think the only real need for start_date is the "catchup=True". >> I think start_date is really part of the metadata of the DAG - that is >> really useful in order to determine range of backfill for example. So it's >> more an intention of the DAG author to describe when we actually want the >> DAG livecycle started. >> As such it is nice to keep in the "records" - if we do not have it, we >> simply do not know when the DAG should "start". I mean - we could see it by >> historical DagRuns, but the problem is that if DagRuns are removed, that >> information is lost. >> >> But it does not have to be specified in the DAG() object in Python IMHO >> >> I do not think we should actually remove the "start_dag" from Dag model, >> but also I think it should be perfectly fine to simply set start_date in >> Dag model to "NOW()" if it is not passed. the NOW() should not be NOW() >> really I think - because of the intricacies of "execution_date" >> "start_interval", "end_interval" it should be automatically adjusted. And >> here I am not sure exactly - either so that when you create a DAG without >> start_date, it starts immediately for the current interval, or starts for >> the future interval (not 100% sure how well it will play with custom >> timetables but I think it can be worked out rather easily. >> >> J. >> >> >> >> On Thu, May 5, 2022 at 2:30 PM Malthe <mbo...@gmail.com> wrote: >> >>> There's been some prior discussion on removing the requirement for a >>> DAG without a schedule: >>> >>> - https://issues.apache.org/jira/browse/AIRFLOW-3739 >>> - https://github.com/apache/airflow/pull/5423 >>> >>> But why actually have the requirement at all. >>> >>> The documentation isn't particularly clear on why we need "start_date" >>> and the whole idea seems somewhat confusing: >>> >>> >>> https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-s-the-deal-with-start-date >>> >>> Consider: >>> >>> croniter("*/5 * * * *", start_time=None).get_next(datetime.datetime) >>> >>> My UTC time is "2022-05-05T12:22:16.914769" and the above expression >>> evaluates to: >>> >>> 2022-05-05T12:25:00 >>> >>> That is, it's nicely aligned as you would expect. I would assume from >>> reading the code that this carries over to `CronDataIntervalTimetable` >>> since it uses croniter in exactly this way. >>> >>> Must we require a "start_date" – ? >>> >> >>