You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/08/11 14:31:58 UTC

[GitHub] [airflow] uranusjr opened a new pull request #17552: AIP 39: Documentation

uranusjr opened a new pull request #17552:
URL: https://github.com/apache/airflow/pull/17552


   Very much a work in progress. This is pushed to a branch in *this* repository (not a fork) for easier collaboration. I have very low confidence on my ability writing comprehensive prose 🙂 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709083505



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -270,77 +275,73 @@ Unit tests ensure that there is no incorrect code in your DAG. You can write uni
 
 .. code-block:: python
 
- from airflow.models import DagBag
- import unittest
+    import pytest
 
+    from airflow.models import DagBag
 
- class TestHelloWorldDAG(unittest.TestCase):
-     @classmethod
-     def setUpClass(cls):
-         cls.dagbag = DagBag()
 
-     def test_dag_loaded(self):
-         dag = self.dagbag.get_dag(dag_id="hello_world")
-         assert self.dagbag.import_errors == {}
-         assert dag is not None
-         assert len(dag.tasks) == 1
+    @pytest.fixture(scope="scope")

Review comment:
       ```suggestion
       @pytest.fixture(scope="module")
   ```
   
   I think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708444322



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we

Review comment:
       Sphinx renders triple-dash as an em-dash by default. This can be disabled, but from what I can tell we currently have it on.
   
   https://www.sphinx-doc.org/en/master/usage/configuration.html#confval-smartquotes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709079041



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we

Review comment:
       The LaTeX author in me says no spaces, but I don't have a strong preference.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r690237624



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``Non`` of there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+.. note::
+
+    In case you're wondering---yes, the argument and return value of
+    ``infer_data_interval`` are also internally combined into a ``DagRunInfo``.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically wan to schedule a run as soon as the data interval ends,
+``end`` and ``run_after`` above are generally the same. ``DagRunInfo`` therefore
+provides a shortcut for this:
+
+.. code-block:: python
+
+    info = DagRunInfo.interval(start=start, end=end)
+    assert info.data_interval.end == info.run_after  # Always True.
+
+For reference, here's our DAG file in its entirety:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :start-after: [START howto_timetable]
+    :end-before: [END howto_timetable]
+
+
+DAG Serialization and Parameterized Timetables
+----------------------------------------------
+
+Sometimes we need to pass some run-time arguments to the timetable. Continuing
+with out ``AfterWorkdayTimetable``, maybe we may have DAGs running on different
+timezones, and we want to schedule some DAGs at 8am the next day, instead of
+on midnight. Instead of creating a separate timetable for each purpose, we'd
+want to do something like:
+
+.. code-block:: python
+
+    from datetime import timedelta
+
+    from pendulum import DateTime, Time
+
+
+    class SometimeAfterWorkdayTimetable(Timetable):
+        def __init__(self, schedule_at: Time) -> None:
+            self._schedule_at = schedule_at
+
+        def next_dagrun_info(self, last_automated_dagrun, restriction):
+            ...
+            end = start + timedelta(days=1)
+            return DagRunInfo(
+                data_interval=DataInterval(start=start, end=end),
+                run_after=DateTime.combine(end.date(), self._schedule_at).replace(
+                    tzinfo=end.tzinfo
+                ),
+            )
+
+However, since the timetable is a part of the DAG, we need to tell Airflow how
+to serialize it with the context we provide in ``__init__``. This is done by
+implementing two additional methods on our timetable class:
+
+.. code-block:: python
+
+    def serialize(self) -> Dict[str, Any]:
+        return {"schedule_at": self._schedule_at.isoformat()}
+
+
+    @classmethod
+    def deserialize(cls, value: Dict[str, Any]) -> Timetable:
+        return cls(Time.fromisoformat(value["schedule_at"]))
+
+When the DAG is being serialized, ``serialize`` is called to obtain a
+JSON-serializable value. That value is passed to ``deserialize`` when the
+serialized DAG is accessed by the scheduler to reconstruct the timetable.

Review comment:
       There might be a thing we need to add there about "registering" the class too, but we can add that to the docs when we decide.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709084189



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -270,77 +275,73 @@ Unit tests ensure that there is no incorrect code in your DAG. You can write uni
 
 .. code-block:: python
 
- from airflow.models import DagBag
- import unittest
+    import pytest
 
+    from airflow.models import DagBag
 
- class TestHelloWorldDAG(unittest.TestCase):
-     @classmethod
-     def setUpClass(cls):
-         cls.dagbag = DagBag()
 
-     def test_dag_loaded(self):
-         dag = self.dagbag.get_dag(dag_id="hello_world")
-         assert self.dagbag.import_errors == {}
-         assert dag is not None
-         assert len(dag.tasks) == 1
+    @pytest.fixture(scope="scope")
+    def dagbag(self):
+        return DagBag()
+
+
+    def test_dag_loaded(self, dagbag):
+        dag = dagbag.get_dag(dag_id="hello_world")
+        assert dagbag.import_errors == {}
+        assert dag is not None
+        assert len(dag.tasks) == 1
+
 
 **Unit test a DAG structure:**
 This is an example test want to verify the structure of a code-generated DAG against a dict object
 
 .. code-block:: python
 
- import unittest
+      def assert_dag_dict_equal(source, dag):
+          assert dag.task_dict.keys() == source.keys()
+          for task_id, downstream_list in source.items():
+              assert dag.has_task(task_id)
+              task = dag.get_task(task_id)
+              assert task.downstream_task_ids == set(downstream_list)
 
 
- class testClass(unittest.TestCase):
-     def assertDagDictEqual(self, source, dag):
-         assert dag.task_dict.keys() == source.keys()
-         for task_id, downstream_list in source.items():
-             assert dag.has_task(task_id)
-             task = dag.get_task(task_id)
-             assert task.downstream_task_ids == set(downstream_list)
+      def test_dag():
+          assert_dag_dict_equal(
+              {
+                  "DummyInstruction_0": ["DummyInstruction_1"],
+                  "DummyInstruction_1": ["DummyInstruction_2"],
+                  "DummyInstruction_2": ["DummyInstruction_3"],
+                  "DummyInstruction_3": [],
+              },
+              dag,
+          )
 
-     def test_dag(self):
-         self.assertDagDictEqual(
-             {
-                 "DummyInstruction_0": ["DummyInstruction_1"],
-                 "DummyInstruction_1": ["DummyInstruction_2"],
-                 "DummyInstruction_2": ["DummyInstruction_3"],
-                 "DummyInstruction_3": [],
-             },
-             dag,
-         )
 
 **Unit test for custom operator:**
 
 .. code-block:: python
 
- import unittest
- from airflow.utils.state import State
-
- DEFAULT_DATE = "2019-10-03"
- TEST_DAG_ID = "test_my_custom_operator"
-
-
- class MyCustomOperatorTest(unittest.TestCase):
-     def setUp(self):
-         self.dag = DAG(
-             TEST_DAG_ID,
-             schedule_interval="@daily",
-             default_args={"start_date": DEFAULT_DATE},
-         )
-         self.op = MyCustomOperator(
-             dag=self.dag,
-             task_id="test",
-             prefix="s3://bucket/some/prefix",
-         )
-         self.ti = TaskInstance(task=self.op, execution_date=DEFAULT_DATE)
-
-     def test_execute_no_trigger(self):
-         self.ti.run(ignore_ti_state=True)
-         assert self.ti.state == State.SUCCESS
-         # Assert something related to tasks results
+    import pytest
+
+    from airflow.utils.state import State
+
+
+    def test_my_custom_oeprator_execute_no_trigger(dag_maker):
+        with dag_maker(

Review comment:
       The idea behind this doc is for users to be able to test their DAGs, but the `dag_maker` fixture is only available inside the airflow tests, so this won't work.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jedcunningham commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708357778



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -43,21 +43,26 @@ Please follow our guide on :ref:`custom Operators <custom_operator>`.
 Creating a task
 ---------------
 
-You should treat tasks in Airflow equivalent to transactions in a database. This implies that you should never produce
-incomplete results from your tasks. An example is not to produce incomplete data in ``HDFS`` or ``S3`` at the end of a task.
-
-Airflow can retry a task if it fails. Thus, the tasks should produce the same outcome on every re-run.
-Some of the ways you can avoid producing a different result -
-
-* Do not use INSERT during a task re-run, an INSERT statement might lead to duplicate rows in your database.
-  Replace it with UPSERT.
-* Read and write in a specific partition. Never read the latest available data in a task.
-  Someone may update the input data between re-runs, which results in different outputs.
-  A better way is to read the input data from a specific partition. You can use ``execution_date`` as a partition.
-  You should follow this partitioning method while writing data in S3/HDFS, as well.
-* The Python datetime ``now()`` function gives the current datetime object.
-  This function should never be used inside a task, especially to do the critical computation, as it leads to different outcomes on each run.
-  It's fine to use it, for example, to generate a temporary log.
+You should treat tasks in Airflow equivalent to transactions in a database. This
+implies that you should never produce incomplete results from your tasks. An
+example is not to produce incomplete data in ``HDFS`` or ``S3`` at the end of a
+task.
+
+Airflow can retry a task if it fails. Thus, the tasks should produce the same
+outcome on every re-run. Some of the ways you can avoid producing a different
+result -
+
+* Do not use INSERT during a task re-run, an INSERT statement might lead to
+  duplicate rows in your database. Replace it with UPSERT.
+* Read and write in a specific partition. Never read the latest available data
+  in a task. Someone may update the input data between re-runs, which results in
+  different outputs. A better way is to read the input data from a specific
+  partition. You can use ``data_interval_start`` as a partition. You should
+  follow this partitioning method while writing data in S3/HDFS, as well.

Review comment:
       ```suggestion
     follow this partitioning method while writing data in S3/HDFS as well.
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  describing the next run's data interval.
+* ``run_after``: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically want to schedule a run as soon as the data interval ends,

Review comment:
       ```suggestion
   However, since we typically want to schedule a run as soon as the data interval ends,
   ```
   
   Or just start with "Since"?

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we

Review comment:
       ```suggestion
   ``restriction.catchup`` also needs to be considered - if it's ``False``, we
   ```

##########
File path: docs/apache-airflow/logging-monitoring/errors.rst
##########
@@ -52,14 +52,17 @@ Name                                    Description
 ======================================= ==================================================
 ``dag_id``                              Dag name of the dag that failed
 ``task_id``                             Task name of the task that failed
-``execution_date``                      Execution date when the task failed
+``data_interval_start``                 Start of data interval when the task failed
+``data_interval_end``                   End of data interval when the task failed
 ``operator``                            Operator name of the task that failed
 ======================================= ==================================================
 
+For backward compatibility, and additional tag ``execution_date`` is also available the represent the logical date. The tag should be considered deprecated in favor of ``data_interval_start``.

Review comment:
       ```suggestion
   For backward compatibility, an additional tag ``execution_date`` is also available representing the logical date. The tag should be considered deprecated in favor of ``data_interval_start``.
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  describing the next run's data interval.
+* ``run_after``: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically want to schedule a run as soon as the data interval ends,
+``end`` and ``run_after`` above are generally the same. ``DagRunInfo`` therefore
+provides a shortcut for this:
+
+.. code-block:: python
+
+    info = DagRunInfo.interval(start=start, end=end)
+    assert info.data_interval.end == info.run_after  # Always True.
+
+For reference, here's our plugin and DAG files in their entirety:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :start-after: [START howto_timetable]
+    :end-before: [END howto_timetable]
+
+.. code-block:: python
+
+    import datetime
+
+    from airflow import DAG
+    from airflow.example_dags.plugins.workday import AfterWorkdayTimetable
+    from airflow.operators.dummy import DummyOperator
+
+
+    with DAG(
+        dag_id="example_workday_timetable",
+        start_date=datetime.datetime(2021, 1, 1),
+        timetable=AfterWorkdayTimetable(),
+        tags=["example", "timetable"],
+    ) as dag:
+        DummyOperator(task_id="run_this")
+
+
+Parameterized Timetables
+------------------------
+
+Sometimes we need to pass some run-time arguments to the timetable. Continuing
+with our ``AfterWorkdayTimetable`` example, maybe we have DAGs running on
+different timezones, and we want to schedule some DAGs at 8am the next day,
+instead of on midnight. Instead of creating a separate timetable for each
+purpose, we'd want to do something like:
+
+.. code-block:: python
+
+    class SometimeAfterWorkdayTimetable(Timetable):
+        def __init__(self, schedule_at: Time) -> None:
+            self._schedule_at = schedule_at
+
+        def next_dagrun_info(self, last_automated_dagrun, restriction):
+            ...
+            end = start + timedelta(days=1)
+            return DagRunInfo(
+                data_interval=DataInterval(start=start, end=end),
+                run_after=DateTime.combine(end.date(), self._schedule_at),
+            )
+
+However, since the timetable is a part of the DAG, we need to tell Airflow how
+to serialize it with the context we provide in ``__init__``. This is done by
+implementing two additional methods on our timetable class:
+
+.. code-block:: python
+
+    class SometimeAfterWorkdayTimetable(Timetable):
+        ...
+
+        def serialize(self) -> Dict[str, Any]:
+            return {"schedule_at": self._schedule_at.isoformat()}
+
+        @classmethod
+        def deserialize(cls, value: Dict[str, Any]) -> Timetable:
+            return cls(Time.fromisoformat(value["schedule_at"]))
+
+When the DAG is being serialized, ``serialize`` is called to obtain a
+JSON-serializable value. That value is passed to ``deserialize`` when the
+serialized DAG is accessed by the scheduler to reconstruct the timetable.
+
+
+Timetable Display in UI
+=======================
+
+By default, a custom timetable is displayed by their class name in the UI (e.g.
+the *Schedule* column in the "DAGs" table. It is possible to customize this
+by overriding the ``summary`` property. This is especially useful for
+parameterized timetables to include arguments provided in ``__init__``. For
+our ``SometimeAfterWorkdayTimetable`` class, for example, we could have:
+
+.. code-block:: python
+
+    @property
+    def summary(self) -> str:
+        return f"after each workday, at {self._schedule_at}"
+
+So for a DAG declared like this:
+
+.. code-block:: python
+
+    with DAG(
+        timetable=SometimeAfterWorkdayTimetable(Time(8)),  # 8am.
+        ...,
+    ) as dag:
+        ...
+
+The *Schedule* column would say ``after each workday, at 08:00:00``.
+
+
+.. seealso::
+
+    Module :mod:`airflow.timetables.base`
+        The public interface is heavily documented to explain what they should

Review comment:
       ```suggestion
           The public interface is heavily documented to explain what should
   ```

##########
File path: docs/apache-airflow/concepts/dags.rst
##########
@@ -317,7 +326,7 @@ Depends On Past
 
 You can also say a task can only run if the *previous* run of the task in the previous DAG Run succeeded. To use this, you just need to set the ``depends_on_past`` argument on your Task to ``True``.
 
-Note that if you are running the DAG at the very start of its life - specifically, that the ``execution_date`` matches the ``start_date`` - then the Task will still run, as there is no previous run to depend on.
+Note that if you are running the DAG at the very start of its life---specifically, its first ever *automated* run---then the Task will still run, as there is no previous run to depend on.

Review comment:
       ```suggestion
   Note that if you are running the DAG at the very start of its life (specifically, its first ever *automated* run) then the Task will still run, as there is no previous run to depend on.
   ```
   
   This looks better to my eyes?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r689287086



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the
+run is able to collect all the actual data within the time period. Therefore, a
+run covering the data period of 2020-01-01 will not start to run until
+2020-01-01 has ended, i.e. 2020-01-02 onwards.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` from previous Airflow version)
+of a DAG run, for example, usually denotes the start of the data interval, not
+when the DAG is actually executed. Similarly, since the ``start_date`` argument
+for the DAG and its tasks points to the same logical date, a run will only
+be created after that data interval ends. So a DAG with ``@daily`` schedule and
+``start_date`` of 2020-01-01, for example, will not be created until 2020-01-02.

Review comment:
       I decided to remove this example since it is too similar to the one in the previous paragraph and could be more confusing than clarifying.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r689285513



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the
+run is able to collect all the actual data within the time period. Therefore, a
+run covering the data period of 2020-01-01 will not start to run until
+2020-01-01 has ended, i.e. 2020-01-02 onwards.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` from previous Airflow version)
+of a DAG run, for example, usually denotes the start of the data interval, not
+when the DAG is actually executed. Similarly, since the ``start_date`` argument
+for the DAG and its tasks points to the same logical date, a run will only
+be created after that data interval ends. So a DAG with ``@daily`` schedule and
+``start_date`` of 2020-01-01, for example, will not be created until 2020-01-02.

Review comment:
       The previous paragraph tries to establish the idea of data intervals in general, and this paragraph explains the concept underlies almost all of Airflow’s date-related configurations, including (the most commonly misunderstood) `start_date`. The intuitive interpretation from the argument name is my DAG should *start running* after this date, while in fact only the DAG’s *first data interval* starts after it. I’ve reworded both paragraphs a bit to make them read more different.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709219641



##########
File path: docs/apache-airflow/concepts/operators.rst
##########
@@ -66,20 +66,20 @@ Jinja Templating
 ----------------
 Airflow leverages the power of `Jinja Templating <http://jinja.pocoo.org/docs/dev/>`_ and this can be a powerful tool to use in combination with :ref:`macros <templates-ref>`.
 
-For example, say you want to pass the execution date as an environment variable to a Bash script using the ``BashOperator``:
+For example, say you want to pass the start of the data interval as an environment variable to a Bash script using the ``BashOperator``:
 
 .. code-block:: python
 
-  # The execution date as YYYY-MM-DD
+  # The start of the data interval as YYYY-MM-DD
   date = "{{ ds }}"

Review comment:
       I was wondering if we are deprecating `{{ ds }}` -- i we aren't then this can stay as it is.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708733553



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,36 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+.. _data-interval:
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
+
+A DAG run is usually scheduled *after* its associated data interval has ended,
+to ensure the run is able to collect all the data within the time period. In
+other words, a run covering the data period of 2020-01-01 generally does not
+start to run until 2020-01-01 has ended, i.e. after 2020-01-02 00:00:00.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` in Airflow versions prior to 2.2)
+of a DAG run, for example, denotes the start of the data interval, not when the
+DAG is actually executed.
+
+Similarly, since the ``start_date`` argument for the DAG and its tasks points to
+the same logical date, it marks the start of *the DAG's fist data interval*, not
+when tasks in the DAG will start running. In other words, a DAG run will only be
+scheduled one interval after ``start_date``.
+
+.. tip::
+
+    If ``schedule_interval`` is not enough to express your DAG's schedule,
+    logical date, or data interval, see :doc:`Customizing imetables </howto/timetable>`.

Review comment:
       ```suggestion
       logical date, or data interval, see :doc:`Customizing timetables </howto/timetable>`.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-923011339


   Might need another rebase


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709220227



##########
File path: docs/apache-airflow-providers-elasticsearch/logging/index.rst
##########
@@ -38,7 +38,7 @@ First, to use the handler, ``airflow.cfg`` must be configured as follows:
 
     [elasticsearch]
     host = <host>:<port>
-    log_id_template = {dag_id}-{task_id}-{execution_date}-{try_number}
+    log_id_template = {dag_id}-{task_id}-{data_interval_start}-{try_number}

Review comment:
       At least conceptually it should be against the logcial date, even if _for now_ these have the same value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jedcunningham commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708453197



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we

Review comment:
       Ah, cool, TIL. Do we want `<space>---<space>` then so we have spaces?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r697817943



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       Some additional context:
   
   > it takes the execution time (which falls after the interval ends) and then adds 1 day to it to find the next start of the next interval. But that ends up skipping an interval, if I'm not mistaken.
   
   This makes me think maybe we should pass the entire `DagRunInfo` into `next_dagrun_info` instead of just the previous run date. It is of course not too difficult to reverse-infer the data interval from the run date in this example, but it might not be always as easy or even possible in more elaborated data interval logic.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r689289972



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.

Review comment:
       I chose the first one




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jedcunningham commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r687016473



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of

Review comment:
       ```suggestion
   its data interval would start at midnight of each day and end at midnight of
   ```

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the

Review comment:
       ```suggestion
   A DAG run is created *after* its associated data interval has ended, to ensure the
   ```
   
   "happens" is fine, otherwise it should be "is created". The latter might be a better choice, but really either work.

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the
+run is able to collect all the actual data within the time period. Therefore, a

Review comment:
       ```suggestion
   run is able to collect all the data within the time period. Therefore, a
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company may want to run a job after each weekday,
+to process data collected during the work day. The first intuitively answer

Review comment:
       ```suggestion
   For our example, let's say a company wants to run a job after each weekday
   to process data collected during the work day. The first intuitive answer
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company may want to run a job after each weekday,
+to process data collected during the work day. The first intuitively answer
+to this would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to
+Friday), but this means data collected on Friday will *not* be processed right
+after Friday, but on the next Monday, and that run's interval would be from
+midnight Friday to midnight *Monday*.
+
+This is, therefore, a case of the "holes" category; the intended schedule should
+leave the two weekend days. What we want is:

Review comment:
       ```suggestion
   This is, therefore, an example in the "holes" category above; the intended schedule should
   not include the two weekend days. What we want is:
   ```

##########
File path: docs/apache-airflow/concepts/dags.rst
##########
@@ -148,14 +148,24 @@ The ``schedule_interval`` argument takes any value that is a valid `Crontab <htt
     with DAG("my_daily_dag", schedule_interval="0 * * * *"):
         ...
 
-Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined ``execution_date``, which identifies the *logical* date and time it is running for - not the *actual* time when it was started.
+.. tip::
+
+    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+
+    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
+
+Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the *logical* date and time range it is running for - not the *actual* time when it was started.
 
 As an example of why this is useful, consider writing a DAG that processes a daily set of experimental data. It's been rewritten, and you want to run it on the previous 3 months of data - no problem, since Airflow can *backfill* the DAG and run copies of it for every day in those previous 3 months, all at once.
 
-Those DAG Runs will all have been started on the same actual day, but their ``execution_date`` values will cover those last 3 months, and that's what all the tasks, operators and sensors inside the DAG look at when they run.
+Those DAG Runs will all have been started on the same actual day, but their data intervals will cover those last 3 months, and that's what all the tasks, operators and sensors inside the DAG look at when they run.

Review comment:
       ```suggestion
   Those DAG Runs will all have been started on the same actual day, but each DAG run will have a data interval covering a single day in that 3 month period, and that data interval is all the tasks, operators and sensors inside the DAG look at when they run.
   ```
   
   I think this is more clear, but I'm not hellbent on this one.

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the
+run is able to collect all the actual data within the time period. Therefore, a
+run covering the data period of 2020-01-01 will not start to run until
+2020-01-01 has ended, i.e. 2020-01-02 onwards.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` from previous Airflow version)
+of a DAG run, for example, usually denotes the start of the data interval, not
+when the DAG is actually executed. Similarly, since the ``start_date`` argument
+for the DAG and its tasks points to the same logical date, a run will only
+be created after that data interval ends. So a DAG with ``@daily`` schedule and
+``start_date`` of 2020-01-01, for example, will not be created until 2020-01-02.

Review comment:
       ```suggestion
   be created after that data interval ends. So a DAG with a ``@daily`` schedule and
   a ``start_date`` of 2020-01-01, for example, will not be created until 2020-01-02.
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company may want to run a job after each weekday,
+to process data collected during the work day. The first intuitively answer
+to this would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to
+Friday), but this means data collected on Friday will *not* be processed right
+after Friday, but on the next Monday, and that run's interval would be from
+midnight Friday to midnight *Monday*.
+
+This is, therefore, a case of the "holes" category; the intended schedule should
+leave the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from the midnight of each day, to the midnight

Review comment:
       ```suggestion
     run's data interval would cover from midnight of each day, to midnight
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.

Review comment:
       ```suggestion
     interval would be a more natural representation.
   ```
   
   Or
   
   ```suggestion
     interval would make the task specification more natural.
   ```

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the
+run is able to collect all the actual data within the time period. Therefore, a
+run covering the data period of 2020-01-01 will not start to run until
+2020-01-01 has ended, i.e. 2020-01-02 onwards.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` from previous Airflow version)

Review comment:
       ```suggestion
   "logical date" (also called ``execution_date`` in Airflow versions before 2.2)
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company may want to run a job after each weekday,
+to process data collected during the work day. The first intuitively answer
+to this would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to
+Friday), but this means data collected on Friday will *not* be processed right
+after Friday, but on the next Monday, and that run's interval would be from

Review comment:
       ```suggestion
   after Friday ends, but on the next Monday, and that run's interval would be from
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the
+  previous sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would task specification more natural.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company may want to run a job after each weekday,
+to process data collected during the work day. The first intuitively answer
+to this would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to
+Friday), but this means data collected on Friday will *not* be processed right
+after Friday, but on the next Monday, and that run's interval would be from
+midnight Friday to midnight *Monday*.
+
+This is, therefore, a case of the "holes" category; the intended schedule should
+leave the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from the midnight of each day, to the midnight
+  of the next day.

Review comment:
       ```suggestion
     of the next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
   ```
   
   Maybe add an example?

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,63 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks on different times each day. For example, an astronomer may find it
+  useful to run a task on each sunset, to process data collected from the

Review comment:
       ```suggestion
   * Run tasks at different times each day. For example, an astronomer may find it
     useful to run a task at sunset to process data collected from the
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] eladkal commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
eladkal commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r686930871



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the
+run is able to collect all the actual data within the time period. Therefore, a
+run covering the data period of 2020-01-01 will not start to run until
+2020-01-01 has ended, i.e. 2020-01-02 onwards.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` from previous Airflow version)
+of a DAG run, for example, usually denotes the start of the data interval, not
+when the DAG is actually executed. Similarly, since the ``start_date`` argument
+for the DAG and its tasks points to the same logical date, a run will only
+be created after that data interval ends. So a DAG with ``@daily`` schedule and
+``start_date`` of 2020-01-01, for example, will not be created until 2020-01-02.

Review comment:
       What is the idea we want to deliver here? (It feels like the same information given in rows 60-63)

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the

Review comment:
       ```suggestion
   A DAG run created *after* its associated data interval has ended, to ensure the
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r689273243



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,31 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day, and end at midnight of
+the next day.
+
+A DAG run happens *after* its associated data interval has ended, to ensure the

Review comment:
       I was trying to avoid the passive “is created” because the data interval’s end does not technically trigger the run’s creation, but merely triggers a request to create a run. I’m not sure whether this distinction is meaningful to the user (from an end user’s perspective, this means the run won’t always be created immediately after the data interval ends, but only sometime later). I switched to use “is scheduled” here instead, but other suggestions are still welcomed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709108678



##########
File path: docs/apache-airflow/faq.rst
##########
@@ -295,7 +311,7 @@ commonly attempted in ``user_defined_macros``.
 
         bo = BashOperator(task_id="my_task", bash_command="echo {{ my_custom_macro }}", dag=dag)
 
-This will echo "day={{ ds }}" instead of "day=2020-01-01" for a dagrun with the execution date 2020-01-01 00:00:00.
+This will echo "day={{ ds }}" instead of "day=2020-01-01" for a dagrun with ``data_interval_start`` 2020-01-01 00:00:00.

Review comment:
       ```suggestion
   This will echo "day={{ ds }}" instead of "day=2020-01-01" for a dagrun with a ``data_interval_start`` of 2020-01-01 00:00:00.
   ```

##########
File path: docs/apache-airflow/concepts/dags.rst
##########
@@ -148,11 +148,20 @@ The ``schedule_interval`` argument takes any value that is a valid `Crontab <htt
     with DAG("my_daily_dag", schedule_interval="0 * * * *"):
         ...
 
-Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined ``execution_date``, which identifies the *logical* date and time it is running for - not the *actual* time when it was started.
+.. tip::
+
+    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+
+    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
+
+Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the *logical* date and time range it is running for - not the *actual* time when it was started.

Review comment:
       ```suggestion
   Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the period of data the tasks should operate on.
   ```

##########
File path: docs/apache-airflow/concepts/dags.rst
##########
@@ -148,11 +148,20 @@ The ``schedule_interval`` argument takes any value that is a valid `Crontab <htt
     with DAG("my_daily_dag", schedule_interval="0 * * * *"):
         ...
 
-Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined ``execution_date``, which identifies the *logical* date and time it is running for - not the *actual* time when it was started.
+.. tip::
+
+    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+
+    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
+
+Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the *logical* date and time range it is running for - not the *actual* time when it was started.

Review comment:
       ```suggestion
   Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the period of data the tasks should operate on.
   ```

##########
File path: docs/apache-airflow/concepts/operators.rst
##########
@@ -66,20 +66,20 @@ Jinja Templating
 ----------------
 Airflow leverages the power of `Jinja Templating <http://jinja.pocoo.org/docs/dev/>`_ and this can be a powerful tool to use in combination with :ref:`macros <templates-ref>`.
 
-For example, say you want to pass the execution date as an environment variable to a Bash script using the ``BashOperator``:
+For example, say you want to pass the start of the data interval as an environment variable to a Bash script using the ``BashOperator``:
 
 .. code-block:: python
 
-  # The execution date as YYYY-MM-DD
+  # The start of the data interval as YYYY-MM-DD
   date = "{{ ds }}"

Review comment:
       Should we use something other than `ds` here now?

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,36 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+.. _data-interval:
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
+
+A DAG run is usually scheduled *after* its associated data interval has ended,
+to ensure the run is able to collect all the data within the time period. In
+other words, a run covering the data period of 2020-01-01 generally does not
+start to run until 2020-01-01 has ended, i.e. after 2020-01-02 00:00:00.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` in Airflow versions prior to 2.2)
+of a DAG run, for example, denotes the start of the data interval, not when the

Review comment:
       I didn't think logical date and stat date were strictly tied to each other -- couldn't a custom timetable choose to do something else here? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708789842



##########
File path: docs/apache-airflow/faq.rst
##########
@@ -216,20 +216,35 @@ actually start. If this were not the case, the backfill just would not start.
 What does ``execution_date`` mean?
 ----------------------------------
 
-Airflow was developed as a solution for ETL needs. In the ETL world, you typically summarize data. So, if you want to
-summarize data for 2016-02-19, You would do it at 2016-02-20 midnight UTC, which would be right after all data for
-2016-02-19 becomes available.
-
-This datetime value is available to you as :ref:`Template variables<templates:variables>` with various formats in Jinja templated
-fields. They are also included in the context dictionary given to an Operator's execute function.
+*Execution date* or ``execution_date`` is a historical name for what is called a
+*logical date*, and also usually the start of the data interval represented by a
+DAG run.
+
+Airflow was developed as a solution for ETL needs. In the ETL world, you
+typically summarize data. So, if you want to summarize data for 2016-02-19, You
+would do it at 2016-02-20 midnight UTC, which would be right after all data for
+2016-02-19 becomes available. This interval between midnights of 2016-02-19 and
+2016-02-20 is called the *data interval*, and since the it represents data in
+the date of 2016-02-19, this date is thus called the run's *logical date*, or
+the date that this DAG run is executed for, thus *execution date*.

Review comment:
       Not sure about making UTC inline code; added ticks around date.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r695019071



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` oi there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after``: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+.. note::
+
+    In case you're wondering---yes, the argument and return value of
+    ``infer_data_interval`` are also internally combined into a ``DagRunInfo``.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically wan to schedule a run as soon as the data interval ends,
+``end`` and ``run_after`` above are generally the same. ``DagRunInfo`` therefore
+provides a shortcut for this:
+
+.. code-block:: python
+
+    info = DagRunInfo.interval(start=start, end=end)
+    assert info.data_interval.end == info.run_after  # Always True.
+
+For reference, here's our DAG file in its entirety:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :start-after: [START howto_timetable]
+    :end-before: [END howto_timetable]
+
+
+DAG Serialization and Parameterized Timetables
+----------------------------------------------
+
+Sometimes we need to pass some run-time arguments to the timetable. Continuing
+with our ``AfterWorkdayTimetable`` example, maybe we may have DAGs running on

Review comment:
       ```suggestion
   with our ``AfterWorkdayTimetable`` example, maybe we have DAGs running on
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r690235892



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,33 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+A DAG run is scheduled *after* its associated data interval has ended, to ensure
+the run is able to collect all the data within the time period. Therefore, a run
+covering the data period of 2020-01-01 will not start to run until 2020-01-01
+has ended, i.e. after 2020-01-02 00:00:00.

Review comment:
       Technically yes; I wrote it like this to avoid going into too much “well except” stuff this early in the documentation. It’d definitely be best if we can explain this default “DAG skips one interval” behaviour in another way.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708791466



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -43,21 +43,26 @@ Please follow our guide on :ref:`custom Operators <custom_operator>`.
 Creating a task
 ---------------
 
-You should treat tasks in Airflow equivalent to transactions in a database. This implies that you should never produce
-incomplete results from your tasks. An example is not to produce incomplete data in ``HDFS`` or ``S3`` at the end of a task.
-
-Airflow can retry a task if it fails. Thus, the tasks should produce the same outcome on every re-run.
-Some of the ways you can avoid producing a different result -
-
-* Do not use INSERT during a task re-run, an INSERT statement might lead to duplicate rows in your database.
-  Replace it with UPSERT.
-* Read and write in a specific partition. Never read the latest available data in a task.
-  Someone may update the input data between re-runs, which results in different outputs.
-  A better way is to read the input data from a specific partition. You can use ``execution_date`` as a partition.
-  You should follow this partitioning method while writing data in S3/HDFS, as well.
-* The Python datetime ``now()`` function gives the current datetime object.
-  This function should never be used inside a task, especially to do the critical computation, as it leads to different outcomes on each run.
-  It's fine to use it, for example, to generate a temporary log.
+You should treat tasks in Airflow equivalent to transactions in a database. This
+implies that you should never produce incomplete results from your tasks. An
+example is not to produce incomplete data in ``HDFS`` or ``S3`` at the end of a
+task.
+
+Airflow can retry a task if it fails. Thus, the tasks should produce the same
+outcome on every re-run. Some of the ways you can avoid producing a different
+result -
+
+* Do not use INSERT during a task re-run, an INSERT statement might lead to
+  duplicate rows in your database. Replace it with UPSERT.
+* Read and write in a specific partition. Never read the latest available data
+  in a task. Someone may update the input data between re-runs, which results in
+  different outputs. A better way is to read the input data from a specific
+  partition. You can use ``data_interval_start`` as a partition. You should
+  follow this partitioning method while writing data in S3/HDFS, as well.

Review comment:
       Side note, there are a lot of *blah blah blah, as well* usages in the documentation, so it seems like whoever wrote it previously has a particular style. (I’ve removed the comma here.)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r695017703



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` oi there are no ``start_date`` arguments
+  found at all.

Review comment:
       ```suggestion
   * ``earliest``: The earliest time the DAG may be scheduled. This is a
     ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
     the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
     found at all.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709087582



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -270,77 +275,73 @@ Unit tests ensure that there is no incorrect code in your DAG. You can write uni
 
 .. code-block:: python
 
- from airflow.models import DagBag
- import unittest
+    import pytest
 
+    from airflow.models import DagBag
 
- class TestHelloWorldDAG(unittest.TestCase):
-     @classmethod
-     def setUpClass(cls):
-         cls.dagbag = DagBag()
 
-     def test_dag_loaded(self):
-         dag = self.dagbag.get_dag(dag_id="hello_world")
-         assert self.dagbag.import_errors == {}
-         assert dag is not None
-         assert len(dag.tasks) == 1
+    @pytest.fixture(scope="scope")

Review comment:
       Oh yes, my bad.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709082411



##########
File path: docs/apache-airflow-providers-elasticsearch/logging/index.rst
##########
@@ -38,7 +38,7 @@ First, to use the handler, ``airflow.cfg`` must be configured as follows:
 
     [elasticsearch]
     host = <host>:<port>
-    log_id_template = {dag_id}-{task_id}-{execution_date}-{try_number}
+    log_id_template = {dag_id}-{task_id}-{data_interval_start}-{try_number}

Review comment:
       The logs should be against the logical/execution date, not the data interval start (as data_interval_start is not guaranteed to be unique.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708788427



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we

Review comment:
       I quickly searched and there doesn’t seem to be an established rule (both are generally accepted), so I’ll defer this to whatever the majority of reviewers think.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil merged pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #17552:
URL: https://github.com/apache/airflow/pull/17552


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-922139213


   (Just rebased)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-900252382


   Tests are failing because we haven’t actually implemented `DAG(timetable=...)` yet. Otherwise CI seems satisfied.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r695018461



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` oi there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after``: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+.. note::
+
+    In case you're wondering---yes, the argument and return value of
+    ``infer_data_interval`` are also internally combined into a ``DagRunInfo``.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically wan to schedule a run as soon as the data interval ends,

Review comment:
       ```suggestion
   But since we typically want to schedule a run as soon as the data interval ends,
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r690236723



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``Non`` of there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+.. note::
+
+    In case you're wondering---yes, the argument and return value of
+    ``infer_data_interval`` are also internally combined into a ``DagRunInfo``.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically wan to schedule a run as soon as the data interval ends,
+``end`` and ``run_after`` above are generally the same. ``DagRunInfo`` therefore
+provides a shortcut for this:
+
+.. code-block:: python
+
+    info = DagRunInfo.interval(start=start, end=end)
+    assert info.data_interval.end == info.run_after  # Always True.
+
+For reference, here's our DAG file in its entirety:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :start-after: [START howto_timetable]
+    :end-before: [END howto_timetable]
+
+
+DAG Serialization and Parameterized Timetables
+----------------------------------------------
+
+Sometimes we need to pass some run-time arguments to the timetable. Continuing
+with out ``AfterWorkdayTimetable``, maybe we may have DAGs running on different
+timezones, and we want to schedule some DAGs at 8am the next day, instead of
+on midnight. Instead of creating a separate timetable for each purpose, we'd
+want to do something like:
+
+.. code-block:: python
+
+    from datetime import timedelta
+
+    from pendulum import DateTime, Time
+
+
+    class SometimeAfterWorkdayTimetable(Timetable):
+        def __init__(self, schedule_at: Time) -> None:
+            self._schedule_at = schedule_at
+
+        def next_dagrun_info(self, last_automated_dagrun, restriction):
+            ...
+            end = start + timedelta(days=1)
+            return DagRunInfo(
+                data_interval=DataInterval(start=start, end=end),
+                run_after=DateTime.combine(end.date(), self._schedule_at).replace(
+                    tzinfo=end.tzinfo
+                ),
+            )
+
+However, since the timetable is a part of the DAG, we need to tell Airflow how
+to serialize it with the context we provide in ``__init__``. This is done by
+implementing two additional methods on our timetable class:
+
+.. code-block:: python
+
+    def serialize(self) -> Dict[str, Any]:
+        return {"schedule_at": self._schedule_at.isoformat()}
+
+
+    @classmethod
+    def deserialize(cls, value: Dict[str, Any]) -> Timetable:
+        return cls(Time.fromisoformat(value["schedule_at"]))
+
+When the DAG is being serialized, ``serialize`` is called to obtain a
+JSON-serializable value. That value is passed to ``deserialize`` when the
+serialized DAG is accessed by the scheduler to reconstruct the timetable.

Review comment:
       Yes, I’m anticipating the actual code change in #17414.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r703507582



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       Regarding also passing in the data interval. This turns out to be more work than I originally anticipated because it means we’ll need to deprecate `DAG.following_schedule()` (which operates on `execution_date` only).  I’ll work on it in the following days.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-900076465


   I created an example DAG showcasing how timetable subclassing works and finished the howto guide referencing the example DAG. I’m more or less done for now, but since the howto’s content depends on changes in #17414, this will stay a draft until that PR is merged (or at least close to it).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jhtimmins commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
jhtimmins commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r697792179



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       Reminder @uranusjr that there is a bug in this example. The next interval should start at a delta from the start of the incoming dagrun, not the dagrun execution time.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709219116



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,36 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+.. _data-interval:
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
+
+A DAG run is usually scheduled *after* its associated data interval has ended,
+to ensure the run is able to collect all the data within the time period. In
+other words, a run covering the data period of 2020-01-01 generally does not
+start to run until 2020-01-01 has ended, i.e. after 2020-01-02 00:00:00.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` in Airflow versions prior to 2.2)
+of a DAG run, for example, denotes the start of the data interval, not when the

Review comment:
       Ah yeah, fair.
   
   Longer term I do want to remove the unqiue constraint on execution_date, but we aren't there yet.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709233224



##########
File path: docs/apache-airflow-providers-elasticsearch/logging/index.rst
##########
@@ -38,7 +38,7 @@ First, to use the handler, ``airflow.cfg`` must be configured as follows:
 
     [elasticsearch]
     host = <host>:<port>
-    log_id_template = {dag_id}-{task_id}-{execution_date}-{try_number}
+    log_id_template = {dag_id}-{task_id}-{data_interval_start}-{try_number}

Review comment:
       Maybe we should use `run_id` instead, that (together with `dag_id` and `task_id`) will forever ever be unique.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r707090443



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,33 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+A DAG run is scheduled *after* its associated data interval has ended, to ensure
+the run is able to collect all the data within the time period. Therefore, a run
+covering the data period of 2020-01-01 will not start to run until 2020-01-01
+has ended, i.e. after 2020-01-02 00:00:00.

Review comment:
       I made the wording less certain on this to acount for the custom timetable edge cases. I don’t think running a DAG before its data interval ends makes enough general sense to complicate the explaination, so this is done with some “usually” and “generally”s.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-918742831


   I think this is ready for review. In addition to the new timetable how-to page, I also rewrote many *execution date* and `execution_date` occurrences to refer to the data intervals instead where it makes sense.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r689286094



##########
File path: docs/apache-airflow/concepts/dags.rst
##########
@@ -148,14 +148,24 @@ The ``schedule_interval`` argument takes any value that is a valid `Crontab <htt
     with DAG("my_daily_dag", schedule_interval="0 * * * *"):
         ...
 
-Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined ``execution_date``, which identifies the *logical* date and time it is running for - not the *actual* time when it was started.
+.. tip::
+
+    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+
+    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
+
+Every time you run a DAG, you are creating a new instance of that DAG which Airflow calls a :doc:`DAG Run </dag-run>`. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the *logical* date and time range it is running for - not the *actual* time when it was started.
 
 As an example of why this is useful, consider writing a DAG that processes a daily set of experimental data. It's been rewritten, and you want to run it on the previous 3 months of data - no problem, since Airflow can *backfill* the DAG and run copies of it for every day in those previous 3 months, all at once.
 
-Those DAG Runs will all have been started on the same actual day, but their ``execution_date`` values will cover those last 3 months, and that's what all the tasks, operators and sensors inside the DAG look at when they run.
+Those DAG Runs will all have been started on the same actual day, but their data intervals will cover those last 3 months, and that's what all the tasks, operators and sensors inside the DAG look at when they run.

Review comment:
       Definitely clearer than the previous wording!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r707030238



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       New `next_dagrun_info` implementation using data interval instead was merged in #18088. I’ve rewritten the example to match the actual implementation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-920859324


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709117613



##########
File path: docs/apache-airflow/concepts/operators.rst
##########
@@ -66,20 +66,20 @@ Jinja Templating
 ----------------
 Airflow leverages the power of `Jinja Templating <http://jinja.pocoo.org/docs/dev/>`_ and this can be a powerful tool to use in combination with :ref:`macros <templates-ref>`.
 
-For example, say you want to pass the execution date as an environment variable to a Bash script using the ``BashOperator``:
+For example, say you want to pass the start of the data interval as an environment variable to a Bash script using the ``BashOperator``:
 
 .. code-block:: python
 
-  # The execution date as YYYY-MM-DD
+  # The start of the data interval as YYYY-MM-DD
   date = "{{ ds }}"

Review comment:
       I guess we can use `{{ run_id }}`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-919587937


   cc @ jwitz


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-920850410


   Suggestions applied. I also changed the `ti.xcom_pull()` usage to `XCom.get_one()` anticipating #18274.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil edited a comment on pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#issuecomment-919587937


   cc @jwitz


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709121296



##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,36 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+.. _data-interval:
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+Data Interval
+-------------
+
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
+
+A DAG run is usually scheduled *after* its associated data interval has ended,
+to ensure the run is able to collect all the data within the time period. In
+other words, a run covering the data period of 2020-01-01 generally does not
+start to run until 2020-01-01 has ended, i.e. after 2020-01-02 00:00:00.
+
+All dates in Airflow are tied to the data interval concept in some way. The
+"logical date" (also called ``execution_date`` in Airflow versions prior to 2.2)
+of a DAG run, for example, denotes the start of the data interval, not when the

Review comment:
       Right now logical date is hard-wired to start of data interval in the data structure. Theoratically a custom timetable can set it to something else, but it’s not exposed as a part of the API (must subclass `DataInterval`) nor explained anywhere. I think it’s best to do this as long as `executable_date` still has UNIQUE constraint to avoid people doing silly things (intentionally or not).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r703446316



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       Hmm, on a closer look, the current logic seems to be correct? The default behaviour of `DagRunInfo` actually defines the execution date as the _start_ of the interval, so I think using `execution_date + delta` is correct. See the `logical_date` property defined here:
   
   https://github.com/apache/airflow/blob/873d4d1f1aad214b36218d80000f5175ae284f6f/airflow/timetables/base.py#L85-L91




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r703397252



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       Yeah this sounds like a good idea




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r703446316



##########
File path: airflow/example_dags/example_workday_timetable.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""Example DAG demostrating how to implement a custom timetable for a DAG."""
+
+# [START howto_timetable]
+from datetime import timedelta
+from typing import Optional
+
+from pendulum import Date, DateTime, Time, timezone
+
+from airflow import DAG
+from airflow.operators.dummy import DummyOperator
+from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
+
+UTC = timezone("UTC")
+
+
+class AfterWorkdayTimetable(Timetable):
+
+    # [START howto_timetable_infer_data_interval]
+    def infer_data_interval(self, run_after: DateTime) -> DataInterval:
+        weekday = run_after.weekday()
+        if weekday in (0, 6):  # Monday and Sunday -- interval is last Friday.
+            days_since_friday = (run_after.weekday() - 4) % 7
+            delta = timedelta(days=days_since_friday)
+        else:  # Otherwise the interval is yesterday.
+            delta = timedelta(days=1)
+        start = DateTime.combine((run_after - delta).date(), Time.min).replace(tzinfo=UTC)
+        return DataInterval(start=start, end=(start + timedelta(days=1)))
+
+    # [END howto_timetable_infer_data_interval]
+
+    # [START howto_timetable_next_dagrun_info]
+    def next_dagrun_info(
+        self,
+        last_automated_dagrun: Optional[DateTime],
+        restriction: TimeRestriction,
+    ) -> Optional[DagRunInfo]:
+        if last_automated_dagrun is not None:
+            # There was a previous run on the regular schedule.
+            # last_automated_dagrun os the last run's logical date.
+            weekday = last_automated_dagrun.weekday()
+            if 0 <= weekday < 4:  # Monday through Thursday -- next is tomorrow.
+                delta = timedelta(days=1)
+            else:  # Week is ending -- skip to next Monday.
+                delta = timedelta(days=(7 - weekday))
+            start = DateTime.combine((last_automated_dagrun + delta).date(), Time.min)

Review comment:
       Hmm, on a closer look, the current logic seems to be correct? The default behaviour of `DagRunInfo` actually defines the execution date as the _start_ of the interval, so I think using `execution_date + delta` is correct. See the `logical_date` property defined here:
   
   https://github.com/apache/airflow/blob/873d4d1f1aad214b36218d80000f5175ae284f6f/airflow/timetables/base.py#L56-L91




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708451254



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,298 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at dawn to process data collected from the previous
+  night-time period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Timetable Registration
+----------------------
+
+A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`,
+and be registered as a part of a :doc:`plugin </plugins>`. The following is a
+skeleton for us to implement a new timetable:
+
+.. code-block:: python
+
+    from airflow.plugins_manager import AirflowPlugin
+    from airflow.timetables.base import Timetable
+
+
+    class AfterWorkdayTimetable(Timetable):
+        pass
+
+
+    class WorkdayTimetablePlugin(AirflowPlugin):
+        name = "workday_timetable_plugin"
+        timetables = [AfterWorkdayTimetable]
+
+Next, we'll start putting code into ``AfterWorkdayTimetable``. After the
+implementation is finished, we should be able to use the timetable in our DAG
+file:
+
+.. code-block:: python
+
+    from airflow import DAG
+
+
+    with DAG(timetable=AfterWorkdayTimetable(), tags=["example", "timetable"]) as dag:
+        ...
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/plugins/workday.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+:class:`~airflow.timetables.base.DataInterval` instance indicating the data
+interval of this DAG's previous non-manually-triggered run, or ``None`` if this
+is the first time ever the DAG is being scheduled. ``restriction`` encapsulates
+how the DAG and its tasks specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  ``pendulum.DateTime`` calculated from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we

Review comment:
       emdash ftw. Nice typography makes me happy.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r709221806



##########
File path: docs/apache-airflow/concepts/operators.rst
##########
@@ -66,20 +66,20 @@ Jinja Templating
 ----------------
 Airflow leverages the power of `Jinja Templating <http://jinja.pocoo.org/docs/dev/>`_ and this can be a powerful tool to use in combination with :ref:`macros <templates-ref>`.
 
-For example, say you want to pass the execution date as an environment variable to a Bash script using the ``BashOperator``:
+For example, say you want to pass the start of the data interval as an environment variable to a Bash script using the ``BashOperator``:
 
 .. code-block:: python
 
-  # The execution date as YYYY-MM-DD
+  # The start of the data interval as YYYY-MM-DD
   date = "{{ ds }}"

Review comment:
       We are not, but the semantic meaning changed (it is now backed by `data_interval_start`, which is another reason why logical date is hard wired to equal to `data_interval_start` right now).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r708734047



##########
File path: docs/apache-airflow/faq.rst
##########
@@ -216,20 +216,35 @@ actually start. If this were not the case, the backfill just would not start.
 What does ``execution_date`` mean?
 ----------------------------------
 
-Airflow was developed as a solution for ETL needs. In the ETL world, you typically summarize data. So, if you want to
-summarize data for 2016-02-19, You would do it at 2016-02-20 midnight UTC, which would be right after all data for
-2016-02-19 becomes available.
-
-This datetime value is available to you as :ref:`Template variables<templates:variables>` with various formats in Jinja templated
-fields. They are also included in the context dictionary given to an Operator's execute function.
+*Execution date* or ``execution_date`` is a historical name for what is called a
+*logical date*, and also usually the start of the data interval represented by a
+DAG run.
+
+Airflow was developed as a solution for ETL needs. In the ETL world, you
+typically summarize data. So, if you want to summarize data for 2016-02-19, You
+would do it at 2016-02-20 midnight UTC, which would be right after all data for
+2016-02-19 becomes available. This interval between midnights of 2016-02-19 and
+2016-02-20 is called the *data interval*, and since the it represents data in
+the date of 2016-02-19, this date is thus called the run's *logical date*, or
+the date that this DAG run is executed for, thus *execution date*.

Review comment:
       ```suggestion
   typically summarize data. So, if you want to summarize data for ``2016-02-19``, You
   would do it at ``2016-02-20`` midnight ``UTC``, which would be right after all data for
   ``2016-02-19`` becomes available. This interval between midnights of ``2016-02-19`` and
   ``2016-02-20`` is called the *data interval*, and since it represents data in
   the date of ``2016-02-19``, this date is thus called the run's *logical date*, or
   the date that this DAG run is executed for, thus *execution date*.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #17552: AIP 39: Documentation

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17552:
URL: https://github.com/apache/airflow/pull/17552#discussion_r690227197



##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``Non`` of there are no ``start_date`` arguments

Review comment:
       ```suggestion
     the DAG and its tasks, or ``None`` if there are no ``start_date`` arguments
   ```

##########
File path: docs/apache-airflow/dag-run.rst
##########
@@ -54,17 +54,33 @@ Cron Presets
 Your DAG will be instantiated for each schedule along with a corresponding
 DAG Run entry in the database backend.
 
-.. note::
+Data Interval
+-------------
 
-    If you run a DAG on a schedule_interval of one day, the run stamped 2020-01-01
-    will be triggered soon after 2020-01-01T23:59. In other words, the job instance is
-    started once the period it covers has ended.  The ``execution_date`` available in the context
-    will also be 2020-01-01.
+Each DAG run in Airflow has an assigned "data interval" that represents the time
+range it operates in. For a DAG scheduled with ``@daily``, for example, each of
+its data interval would start at midnight of each day and end at midnight of the
+next day.
 
-    The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG.
-    Subsequent DAG Runs are created by the scheduler process, based on your DAG’s ``schedule_interval``,
-    sequentially. If your start_date is 2020-01-01 and schedule_interval is @daily, the first run
-    will be created on 2020-01-02 i.e., after your start date has passed.
+A DAG run is scheduled *after* its associated data interval has ended, to ensure
+the run is able to collect all the data within the time period. Therefore, a run
+covering the data period of 2020-01-01 will not start to run until 2020-01-01
+has ended, i.e. after 2020-01-02 00:00:00.

Review comment:
       This doesn't _have_ to be the case, it's up to the timetable to decide when it's schedulable isn't it?

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``Non`` of there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+.. note::
+
+    In case you're wondering---yes, the argument and return value of
+    ``infer_data_interval`` are also internally combined into a ``DagRunInfo``.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically wan to schedule a run as soon as the data interval ends,
+``end`` and ``run_after`` above are generally the same. ``DagRunInfo`` therefore
+provides a shortcut for this:
+
+.. code-block:: python
+
+    info = DagRunInfo.interval(start=start, end=end)
+    assert info.data_interval.end == info.run_after  # Always True.
+
+For reference, here's our DAG file in its entirety:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :start-after: [START howto_timetable]
+    :end-before: [END howto_timetable]
+
+
+DAG Serialization and Parameterized Timetables
+----------------------------------------------
+
+Sometimes we need to pass some run-time arguments to the timetable. Continuing
+with out ``AfterWorkdayTimetable``, maybe we may have DAGs running on different

Review comment:
       ```suggestion
   with our ``AfterWorkdayTimetable`` example, maybe we may have DAGs running on different
   ```

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``Non`` of there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after: A ``pendulum.DateTime`` instance that tells the scheduler when
+  the DAG run can be scheduled.
+
+.. note::
+
+    In case you're wondering---yes, the argument and return value of
+    ``infer_data_interval`` are also internally combined into a ``DagRunInfo``.
+
+A ``DagRunInfo`` can be created like this:
+
+.. code-block:: python
+
+    info = DagRunInfo(
+        data_interval=DataInterval(start=start, end=end),
+        run_after=run_after,
+    )
+
+But since we typically wan to schedule a run as soon as the data interval ends,
+``end`` and ``run_after`` above are generally the same. ``DagRunInfo`` therefore
+provides a shortcut for this:
+
+.. code-block:: python
+
+    info = DagRunInfo.interval(start=start, end=end)
+    assert info.data_interval.end == info.run_after  # Always True.
+
+For reference, here's our DAG file in its entirety:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :start-after: [START howto_timetable]
+    :end-before: [END howto_timetable]
+
+
+DAG Serialization and Parameterized Timetables
+----------------------------------------------
+
+Sometimes we need to pass some run-time arguments to the timetable. Continuing
+with out ``AfterWorkdayTimetable``, maybe we may have DAGs running on different
+timezones, and we want to schedule some DAGs at 8am the next day, instead of
+on midnight. Instead of creating a separate timetable for each purpose, we'd
+want to do something like:
+
+.. code-block:: python
+
+    from datetime import timedelta
+
+    from pendulum import DateTime, Time
+
+
+    class SometimeAfterWorkdayTimetable(Timetable):
+        def __init__(self, schedule_at: Time) -> None:
+            self._schedule_at = schedule_at
+
+        def next_dagrun_info(self, last_automated_dagrun, restriction):
+            ...
+            end = start + timedelta(days=1)
+            return DagRunInfo(
+                data_interval=DataInterval(start=start, end=end),
+                run_after=DateTime.combine(end.date(), self._schedule_at).replace(
+                    tzinfo=end.tzinfo
+                ),
+            )
+
+However, since the timetable is a part of the DAG, we need to tell Airflow how
+to serialize it with the context we provide in ``__init__``. This is done by
+implementing two additional methods on our timetable class:
+
+.. code-block:: python
+
+    def serialize(self) -> Dict[str, Any]:
+        return {"schedule_at": self._schedule_at.isoformat()}
+
+
+    @classmethod
+    def deserialize(cls, value: Dict[str, Any]) -> Timetable:
+        return cls(Time.fromisoformat(value["schedule_at"]))
+
+When the DAG is being serialized, ``serialize`` is called to obtain a
+JSON-serializable value. That value is passed to ``deserialize`` when the
+serialized DAG is accessed by the scheduler to reconstruct the timetable.

Review comment:
       (Looks fine, but this code isn't landed yet, right?)

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.

Review comment:
       ```suggestion
     useful to run a task at dawn to process data collected from the previous
     night-time period.
   ```
   
   Seems more realistic :)

##########
File path: docs/apache-airflow/howto/timetable.rst
##########
@@ -0,0 +1,261 @@
+ .. Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+ ..   http://www.apache.org/licenses/LICENSE-2.0
+
+ .. Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+
+Customizing DAG Scheduling with Timetables
+==========================================
+
+A DAG's scheduling strategy is determined by its internal "timetable". This
+timetable can be created by specifying the DAG's ``schedule_interval`` argument,
+as described in :doc:`DAG Run </dag-run>`. The timetable also dictates the data
+interval and the logical time of each run created for the DAG.
+
+However, there are situations when a cron expression or simple ``timedelta``
+periods cannot properly express the schedule. Some of the examples are:
+
+* Data intervals with "holes" between. (Instead of continuous, as both the cron
+  expression and ``timedelta`` schedules represent.)
+* Run tasks at different times each day. For example, an astronomer may find it
+  useful to run a task at sunset to process data collected from the previous
+  sunlight period.
+* Schedules not following the Gregorian calendar. For example, create a run for
+  each month in the `Traditional Chinese Calendar`_. This is conceptually
+  similar to the sunset case above, but for a different time scale.
+* Rolling windows, or overlapping data intervals. For example, one may want to
+  have a run each day, but make each run cover the period of the previous seven
+  days. It is possible to "hack" this with a cron expression, but a custom data
+  interval would be a more natural representation.
+
+.. _`Traditional Chinese Calendar`: https://en.wikipedia.org/wiki/Chinese_calendar
+
+
+For our example, let's say a company wants to run a job after each weekday to
+process data collected during the work day. The first intuitive answer to this
+would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but
+this means data collected on Friday will *not* be processed right after Friday
+ends, but on the next Monday, and that run's interval would be from midnight
+Friday to midnight *Monday*.
+
+This is, therefore, an example in the "holes" category above; the intended
+schedule should not include the two weekend days. What we want is:
+
+* Schedule a run for each Monday, Tuesday, Wednesday, Thursday, and Friday. The
+  run's data interval would cover from midnight of each day, to midnight of the
+  next day (e.g. 2021-01-01 00:00:00 to 2021-01-02 00:00:00).
+* Each run would be created right after the data interval ends. The run covering
+  Monday happens on midnight Tuesday and so on. The run covering Friday happens
+  on midnight Saturday. No runs happen on midnights Sunday and Monday.
+
+For simplicity, we will only deal with UTC datetimes in this example.
+
+
+Define Scheduling Logic
+-----------------------
+
+When Airflow's scheduler encounters a DAG, it calls one of the two methods to
+know when to schedule the DAG's next run.
+
+* ``next_dagrun_info``: The scheduler uses this to learn the timetable's regular
+  schedule, i.e. the "one for every workday, run at the end of it" part in our
+  example.
+* ``infer_data_interval``: When a DAG run is manually triggered (from the web
+  UI, for example), the scheduler uses this method to learn about how to
+  reverse-infer the out-of-schedule run's data interval.
+
+We'll start with ``infer_data_interval`` since it's the easier of the two:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_infer_data_interval]
+    :end-before: [END howto_timetable_infer_data_interval]
+
+The method accepts one argument ``run_after``, a ``pendulum.DateTime`` object
+that indicates when the DAG is externally triggered. Since our timetable creates
+a data interval for each complete work day, the data interval inferred here
+should usually start at the midnight one day prior to ``run_after``, but if
+``run_after`` falls on a Sunday or Monday (i.e. the prior day is Saturday or
+Sunday), it should be pushed further back to the previous Friday. Once we know
+the start of the interval, the end is simply one full day after it. We then
+create a :class:`~airflow.timetables.base.DataInterval` object to describe this
+interval.
+
+Next is the implementation of ``next_dagrun_info``:
+
+.. exampleinclude:: /../../airflow/example_dags/example_workday_timetable.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_timetable_next_dagrun_info]
+    :end-before: [END howto_timetable_next_dagrun_info]
+
+This method accepts two arguments. ``last_automated_dagrun`` is a
+``pendulum.DateTime`` object indicating the logical date of this DAG's previous
+non-manually-triggered run, or ``None`` if this is the first time ever the DAG
+is being scheduled. ``restriction`` encapsulates how the DAG and its tasks
+specify the schedule, and contains three attributes:
+
+* ``earliest``: The earliest time the DAG may be scheduled. This is a
+  calculated ``pendulum.DateTime`` from all the ``start_date`` arguments from
+  the DAG and its tasks, or ``Non`` of there are no ``start_date`` arguments
+  found at all.
+* ``latest``: Similar to ``earliest``, this is the latest time the DAG may be
+  scheduled, calculated from ``end_date`` arguments.
+* ``catchup``: A boolean reflecting the DAG's ``catchup`` argument.
+
+.. note::
+
+    Both ``earliest`` and ``latest`` apply to the DAG run's logical date
+    (the *start* of the data interval), not when the run will be scheduled
+    (usually after the end of the data interval).
+
+If there was a run scheduled previously, we should now schedule for the next
+weekday, i.e. plus one day if the previous run was on Monday through Thursday,
+or three days if it was on Friday. If there was not a previous scheduled run,
+however, we pick the next workday's midnight after ``restriction.earliest``
+(unless it *is* a workday's midnight; in which case it's used directly).
+``restriction.catchup`` also needs to be considered---if it's ``False``, we
+can't schedule before the current time, even if ``start_date`` values are in the
+past. Finally, if our calculated data interval is later than
+``restriction.latest``, we must respect it and not schedule a run by returning
+``None``.
+
+If we decide to schedule a run, we need to describe it with a
+:class:`~airflow.timetables.base.DagRunInfo`. This type has two arguments and
+attributes:
+
+* ``data_interval``: A :class:`~airflow.timetables.base.DataInterval` instance
+  like ``infer_data_interval``'s return value. This describes the next run's
+  data interval.
+* ``run_after: A ``pendulum.DateTime`` instance that tells the scheduler when

Review comment:
       ```suggestion
   * ``run_after``: A ``pendulum.DateTime`` instance that tells the scheduler when
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org