This is an automated email from the ASF dual-hosted git repository.
potiuk pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git
The following commit(s) were added to refs/heads/main by this push:
new ac4073b737 Make it possible to change the default cron timetable
(#34851)
ac4073b737 is described below
commit ac4073b73783094f2877f0b6c89b606fae935b6b
Author: Collin McNulty <[email protected]>
AuthorDate: Thu Jan 4 12:59:02 2024 -0600
Make it possible to change the default cron timetable (#34851)
---------
Co-authored-by: Tzu-ping Chung <[email protected]>
---
airflow/config_templates/config.yml | 19 +++++++++++++++++++
airflow/models/dag.py | 8 ++++++--
2 files changed, 25 insertions(+), 2 deletions(-)
diff --git a/airflow/config_templates/config.yml
b/airflow/config_templates/config.yml
index 3f4b4e131f..1788e00593 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -2375,6 +2375,25 @@ scheduler:
type: string
example: ~
default: "^[A-Za-z0-9_.~:+-]+$"
+ create_cron_data_intervals:
+ description: |
+ Whether to create DAG runs that span an interval or one single point
in time for cron schedules, when
+ a cron string is provided to `schedule` argument of a DAG. If True,
+ CronDataIntervalTimetable is used, which is the legacy Airflow
behavior suitable
+ for DAGs with well-defined data_interval you get contiguous intervals
from the end of the previous
+ interval up to the scheduled datetime. If False, CronTriggerTimetable
is used,
+ which is closer to the behavior of cron itself.
+
+ Notably, for CronTriggerTimetable, the logical_date is the same as the
time the DAG Run will try to
+ schedule, while for CronDataIntervalTimetable, the logical_date is the
beginning of the data interval,
+ but the DAG Run will try to schedule at the end of the data interval.
For more differences
+ between the two Timetables, see
+
https://airflow.apache.org/docs/apache-airflow/stable/authoring-and-scheduling/timetable.html#differences-between-the-two-cron-timetables
+ version_added: 2.9.0
+ type: boolean
+ example: ~
+ default: "True"
+ see_also: ":ref:`Differences between the two cron timetables`"
triggerer:
description: ~
options:
diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index d0f46feed2..d2d22d48c9 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -116,6 +116,7 @@ from airflow.timetables.simple import (
NullTimetable,
OnceTimetable,
)
+from airflow.timetables.trigger import CronTriggerTimetable
from airflow.utils import timezone
from airflow.utils.dag_cycle_tester import check_cycle
from airflow.utils.dates import cron_presets, date_range as utils_date_range
@@ -226,8 +227,11 @@ def create_timetable(interval: ScheduleIntervalArg,
timezone: Timezone) -> Timet
if isinstance(interval, (timedelta, relativedelta)):
return DeltaDataIntervalTimetable(interval)
if isinstance(interval, str):
- return CronDataIntervalTimetable(interval, timezone)
- raise ValueError(f"{interval!r} is not a valid interval.")
+ if airflow_conf.getboolean("scheduler", "create_cron_data_intervals"):
+ return CronDataIntervalTimetable(interval, timezone)
+ else:
+ return CronTriggerTimetable(interval, timezone=timezone)
+ raise ValueError(f"{interval!r} is not a valid schedule_interval.")
def get_last_dagrun(dag_id, session, include_externally_triggered=False):