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 2022/06/29 13:06:44 UTC

[GitHub] [airflow] BasPH opened a new pull request, #24733: Add support for multiple cron expressions in schedule_interval

BasPH opened a new pull request, #24733:
URL: https://github.com/apache/airflow/pull/24733

   This PR implements a `MultiCronDataIntervalTimetable`, which is initiated when a user supplies a list of strings to the `schedule_interval`. For example:
   
   ```python
   with DAG(dag_id="foobar", schedule_interval=["0 3 * * *", "0 0 * * MON,TUE"]):
       ...
   ```
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragement file, named `{pr_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
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] BasPH commented on a diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r909893220


##########
docs/apache-airflow/concepts/dags.rst:
##########
@@ -140,28 +140,41 @@ You can also provide an ``.airflowignore`` file inside your ``DAG_FOLDER``, or a
 Running DAGs
 ------------
 
-DAGs will run in one of two ways:
+A DAG can run in two ways:
 
- - When they are *triggered* either manually or via the API
- - On a defined *schedule*, which is defined as part of the DAG
+ - *Manually triggered* via the UI or API
+ - Or *scheduled*, defined by ``schedule_interval`` on the DAG. For example::
 
-DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule_interval`` argument, like this::
 
-    with DAG("my_daily_dag", schedule_interval="@daily"):
-        ...
+    from airflow.models import DAG
 
-The ``schedule_interval`` argument takes any value that is a valid `Crontab <https://en.wikipedia.org/wiki/Cron>`_ schedule value, so you could also do::
 
-    with DAG("my_daily_dag", schedule_interval="0 * * * *"):
+    with DAG("my_dag", schedule_interval="0 0 * * *"):
         ...
 
-.. tip::
 
-    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+This DAG will run every day at 00:00, as defined by the cron expression given to ``schedule_interval``. The value of ``schedule_interval`` can take several types:
+
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Type                                     | When to use                                                                                                          | Example                              |
++==========================================+======================================================================================================================+======================================+
+| ``None``                                 | No schedule, use for manually triggered DAGs                                                                         | ``None``                             |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron expression (``str``)                | To run at cron-based intervals                                                                                       + ``"0 0 * * *"``                      |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron preset (``str``)                    | Convenience cron expression for readability                                                                          + ``"@daily"``                         |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| List of cron expressions/presets         | To run at intervals that cannot be expressed by a single cron expression.                                            + ``["0 3 * * *", "0 0 * * MON,TUE"]`` |

Review Comment:
   I can be both "and" and "or". For example, when given `["0 3 * * MON,TUE", "@daily"]`, each expression is looked up in the cron presets and converted if found. Shall I convert to "and/or"?



-- 
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 diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r927454669


##########
airflow/models/dag.py:
##########
@@ -105,7 +105,8 @@
 
 
 DagStateChangeCallback = Callable[[Context], None]
-ScheduleInterval = Union[None, str, timedelta, relativedelta]
+MultiCron = Union[List[str], Set[str], Tuple[str, ...]]

Review Comment:
   This additional type alias doesn’t seem to be necessary?
   
   (Also this can probably be simply `Collection[str]`)



-- 
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 diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r927456261


##########
airflow/models/dag.py:
##########
@@ -2500,7 +2503,11 @@ def bulk_write_to_db(cls, dags: Collection["DAG"], session=NEW_SESSION):
             orm_dag.max_active_tasks = dag.max_active_tasks
             orm_dag.max_active_runs = dag.max_active_runs
             orm_dag.has_task_concurrency_limits = any(t.max_active_tis_per_dag is not None for t in dag.tasks)
-            orm_dag.schedule_interval = dag.schedule_interval
+            orm_dag.schedule_interval = (
+                list(dag.schedule_interval)
+                if isinstance(dag.schedule_interval, set)
+                else dag.schedule_interval
+            )

Review Comment:
   Why is it necessary to special-case `set`?



-- 
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 diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r927457014


##########
airflow/timetables/interval.py:
##########
@@ -97,111 +96,95 @@ def next_dagrun_info(
         return DagRunInfo.interval(start=start, end=end)
 
 
-def _is_schedule_fixed(expression: str) -> bool:
-    """Figures out if the schedule has a fixed time (e.g. 3 AM every day).
-
-    :return: True if the schedule has a fixed time, False if not.
+class CronDataIntervalTimetable(_DataIntervalTimetable):
+    """
+    Timetable that schedules data intervals using one or more cron expressions.
 
-    Detection is done by "peeking" the next two cron trigger time; if the
-    two times have the same minute and hour value, the schedule is fixed,
-    and we *don't* need to perform the DST fix.
+    This corresponds to ``schedule_interval=<cron>``, where ``<cron>`` is either:
+    - A five-segment cron representation
+    - One of ``cron_presets``
+    - Or a collection containing values from the above
 
-    This assumes DST happens on whole minute changes (e.g. 12:59 -> 12:00).
+    Don't pass ``@once`` in here; use ``OnceTimetable`` instead.
     """
-    cron = croniter(expression)
-    next_a = cron.get_next(datetime.datetime)
-    next_b = cron.get_next(datetime.datetime)
-    return next_b.minute == next_a.minute and next_b.hour == next_a.hour
 
+    def __init__(
+        self, crons: Union[str, List[str], Set[str], Tuple[str, ...]], timezone: Union[str, Timezone]
+    ) -> None:

Review Comment:
   Instead of normalizing here, I wonder if it’s easier to make this only accept `List[str]` and normalize in `DAG.__init__` instead.



-- 
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 #24733: Add support for multiple cron expressions in schedule_interval

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

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


-- 
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] BasPH commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1170884726

   Merged the MultiCronDataIntervalTimetable into the CronDataIntervalTimetable, which now accepts both a single string and a list of strings.
   
   Converted the PR to draft for the moment because I had to change the logic on `_should_fix_dst` and plan to add tests to ensure correct functioning.
   
   @uranusjr Agreed, will add.


-- 
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] closed pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #24733: Add support for multiple cron expressions in schedule_interval
URL: https://github.com/apache/airflow/pull/24733


-- 
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] BasPH commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1190699365

   @uranusjr @ashb any more thoughts on this?


-- 
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] BasPH commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1170064157

   > My only question is is it worth exposing this via schedule_interval or should we make people use a timetable directly for this?
   
   I believe it's more convenient exposing it via `schedule_interval` since (1) that's currently how other timetables are exposed and (2) timetables to me feel like an implementation detail with lots of intricate details. I believe multiple crons is a common ask and therefore not worth requiring the user to initialize a timetable.
   
   > Do we also need a new timetable class, or could the existing CornDataIntervalTimetable be extended to take multiple patterns, and then all we change is what gets passed when upgrading schedule_interval.?
   
   We could extend the existing implementation to take a list of cron expressions (and convert a single string to a list of strings internally to align the business logic).
   
   > You also haven't provided a description field (which is IIRC what is shown in the UI) for the new timetable class.
   
   Added.


-- 
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 diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r909627473


##########
docs/apache-airflow/concepts/dags.rst:
##########
@@ -140,28 +140,41 @@ You can also provide an ``.airflowignore`` file inside your ``DAG_FOLDER``, or a
 Running DAGs
 ------------
 
-DAGs will run in one of two ways:
+A DAG can run in two ways:
 
- - When they are *triggered* either manually or via the API
- - On a defined *schedule*, which is defined as part of the DAG
+ - *Manually triggered* via the UI or API
+ - Or *scheduled*, defined by ``schedule_interval`` on the DAG. For example::
 
-DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule_interval`` argument, like this::
 
-    with DAG("my_daily_dag", schedule_interval="@daily"):
-        ...
+    from airflow.models import DAG
 
-The ``schedule_interval`` argument takes any value that is a valid `Crontab <https://en.wikipedia.org/wiki/Cron>`_ schedule value, so you could also do::
 
-    with DAG("my_daily_dag", schedule_interval="0 * * * *"):
+    with DAG("my_dag", schedule_interval="0 0 * * *"):
         ...
 
-.. tip::
 
-    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+This DAG will run every day at 00:00, as defined by the cron expression given to ``schedule_interval``. The value of ``schedule_interval`` can take several types:
+
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Type                                     | When to use                                                                                                          | Example                              |
++==========================================+======================================================================================================================+======================================+
+| ``None``                                 | No schedule, use for manually triggered DAGs                                                                         | ``None``                             |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron expression (``str``)                | To run at cron-based intervals                                                                                       + ``"0 0 * * *"``                      |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron preset (``str``)                    | Convenience cron expression for readability                                                                          + ``"@daily"``                         |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| List of cron expressions/presets         | To run at intervals that cannot be expressed by a single cron expression.                                            + ``["0 3 * * *", "0 0 * * MON,TUE"]`` |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| ``datetime.timedelta``                   | To run at frequency-based intervals. Useful if your interval cannot be expressed by cron e.g. ``timedelta(days=3)``. + ``timedelta(days=3)``                |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| ``dateutil.relativedelta.relativedelta`` | To express an interval in weeks, months, or years (which timedelta cannot do natively).                              + ``relativedelta(months=1)``          |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+
+.. tip::
+    A convenient tool for converting cron expressions to human language is `Crontab.guru <https://crontab.guru/>`_.
 
-    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
-    For more information on ``logical date``, see :ref:`data-interval` and

Review Comment:
   This link seems worth keeping



##########
docs/apache-airflow/concepts/dags.rst:
##########
@@ -140,28 +140,41 @@ You can also provide an ``.airflowignore`` file inside your ``DAG_FOLDER``, or a
 Running DAGs
 ------------
 
-DAGs will run in one of two ways:
+A DAG can run in two ways:
 
- - When they are *triggered* either manually or via the API
- - On a defined *schedule*, which is defined as part of the DAG
+ - *Manually triggered* via the UI or API
+ - Or *scheduled*, defined by ``schedule_interval`` on the DAG. For example::
 
-DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule_interval`` argument, like this::
 
-    with DAG("my_daily_dag", schedule_interval="@daily"):
-        ...
+    from airflow.models import DAG
 
-The ``schedule_interval`` argument takes any value that is a valid `Crontab <https://en.wikipedia.org/wiki/Cron>`_ schedule value, so you could also do::
 
-    with DAG("my_daily_dag", schedule_interval="0 * * * *"):
+    with DAG("my_dag", schedule_interval="0 0 * * *"):
         ...
 
-.. tip::
 
-    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+This DAG will run every day at 00:00, as defined by the cron expression given to ``schedule_interval``. The value of ``schedule_interval`` can take several types:
+
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Type                                     | When to use                                                                                                          | Example                              |
++==========================================+======================================================================================================================+======================================+
+| ``None``                                 | No schedule, use for manually triggered DAGs                                                                         | ``None``                             |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron expression (``str``)                | To run at cron-based intervals                                                                                       + ``"0 0 * * *"``                      |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron preset (``str``)                    | Convenience cron expression for readability                                                                          + ``"@daily"``                         |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| List of cron expressions/presets         | To run at intervals that cannot be expressed by a single cron expression.                                            + ``["0 3 * * *", "0 0 * * MON,TUE"]`` |

Review Comment:
   This should say if the mode is "and" or "or" (as implemented it's "or")



-- 
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 #24733: Add support for multiple cron expressions in schedule_interval

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

   Exposing this in `schedule_interval` makes sense to me; this is probably the only semantic that makes sense anyway.
   
   I would propose we accept any `Collection[str]` input, instead of just list. Passing in a set makes as much sense here as list, maybe even more?


-- 
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] moreaupascal56 commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by "moreaupascal56 (via GitHub)" <gi...@apache.org>.
moreaupascal56 commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1632425186

   Hi all,
   +1 for @Inetov, I can try to help on this as well. I am using this timetable for now ([https://stackoverflow.com/a/72492370](https://stackoverflow.com/a/72492370)) but a native option would be great 👍 


-- 
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 #24733: Add support for multiple cron expressions in schedule_interval

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1647143572

   Please refrain from posting messages such as “+1” since it does not add to the conversation nor move forward the feature. If you are interested in seeing the functionality, please start a pull request yourself.


-- 
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] jprumsey commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by "jprumsey (via GitHub)" <gi...@apache.org>.
jprumsey commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1642202499

   +1, native option here would be great


-- 
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 diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r927458042


##########
docs/apache-airflow/concepts/dags.rst:
##########
@@ -197,6 +210,9 @@ schedule interval put in place, the logical date is going to indicate the time
 at which it marks the start of the data interval, where the DAG run's start
 date would then be the logical date + scheduled interval.
 
+.. tip::
+    For more information on ``logical date``, see :ref:`data-interval` and :ref:`faq:what-does-execution-date-mean`.

Review Comment:
   ```suggestion
       For more information on *logical date*, see :ref:`data-interval` and :ref:`faq:what-does-execution-date-mean`.
   ```
   
   nit (this word is not code so let’s not make it look like code)



-- 
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 diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r927455304


##########
airflow/models/dag.py:
##########
@@ -168,7 +169,9 @@ def create_timetable(interval: ScheduleIntervalArg, timezone: Timezone) -> Timet
         return OnceTimetable()
     if isinstance(interval, (timedelta, relativedelta)):
         return DeltaDataIntervalTimetable(interval)
-    if isinstance(interval, str):
+    if isinstance(interval, str) or (
+        isinstance(interval, (list, set, tuple)) and all(isinstance(element, str) for element in interval)
+    ):

Review Comment:
   ```suggestion
       if isinstance(interval, str) or (
           isinstance(interval, Collection) and all(isinstance(element, str) for element in interval)
       ):
   ```



-- 
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] BasPH commented on a diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r909738114


##########
docs/apache-airflow/concepts/dags.rst:
##########
@@ -140,28 +140,41 @@ You can also provide an ``.airflowignore`` file inside your ``DAG_FOLDER``, or a
 Running DAGs
 ------------
 
-DAGs will run in one of two ways:
+A DAG can run in two ways:
 
- - When they are *triggered* either manually or via the API
- - On a defined *schedule*, which is defined as part of the DAG
+ - *Manually triggered* via the UI or API
+ - Or *scheduled*, defined by ``schedule_interval`` on the DAG. For example::
 
-DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule_interval`` argument, like this::
 
-    with DAG("my_daily_dag", schedule_interval="@daily"):
-        ...
+    from airflow.models import DAG
 
-The ``schedule_interval`` argument takes any value that is a valid `Crontab <https://en.wikipedia.org/wiki/Cron>`_ schedule value, so you could also do::
 
-    with DAG("my_daily_dag", schedule_interval="0 * * * *"):
+    with DAG("my_dag", schedule_interval="0 0 * * *"):
         ...
 
-.. tip::
 
-    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+This DAG will run every day at 00:00, as defined by the cron expression given to ``schedule_interval``. The value of ``schedule_interval`` can take several types:
+
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Type                                     | When to use                                                                                                          | Example                              |
++==========================================+======================================================================================================================+======================================+
+| ``None``                                 | No schedule, use for manually triggered DAGs                                                                         | ``None``                             |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron expression (``str``)                | To run at cron-based intervals                                                                                       + ``"0 0 * * *"``                      |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron preset (``str``)                    | Convenience cron expression for readability                                                                          + ``"@daily"``                         |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| List of cron expressions/presets         | To run at intervals that cannot be expressed by a single cron expression.                                            + ``["0 3 * * *", "0 0 * * MON,TUE"]`` |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| ``datetime.timedelta``                   | To run at frequency-based intervals. Useful if your interval cannot be expressed by cron e.g. ``timedelta(days=3)``. + ``timedelta(days=3)``                |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| ``dateutil.relativedelta.relativedelta`` | To express an interval in weeks, months, or years (which timedelta cannot do natively).                              + ``relativedelta(months=1)``          |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+
+.. tip::
+    A convenient tool for converting cron expressions to human language is `Crontab.guru <https://crontab.guru/>`_.
 
-    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
-    For more information on ``logical date``, see :ref:`data-interval` and

Review Comment:
   `logical_date` is not mentioned anywhere above, nor is there any text leading up to anything about `logical_date`. I don't see where it fits in here.
   
   Reading again, we could move it down, between the last sentence of this section and the header "DAG Assignment".



-- 
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] BasPH commented on a diff in pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on code in PR #24733:
URL: https://github.com/apache/airflow/pull/24733#discussion_r911122094


##########
docs/apache-airflow/concepts/dags.rst:
##########
@@ -140,28 +140,41 @@ You can also provide an ``.airflowignore`` file inside your ``DAG_FOLDER``, or a
 Running DAGs
 ------------
 
-DAGs will run in one of two ways:
+A DAG can run in two ways:
 
- - When they are *triggered* either manually or via the API
- - On a defined *schedule*, which is defined as part of the DAG
+ - *Manually triggered* via the UI or API
+ - Or *scheduled*, defined by ``schedule_interval`` on the DAG. For example::
 
-DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule_interval`` argument, like this::
 
-    with DAG("my_daily_dag", schedule_interval="@daily"):
-        ...
+    from airflow.models import DAG
 
-The ``schedule_interval`` argument takes any value that is a valid `Crontab <https://en.wikipedia.org/wiki/Cron>`_ schedule value, so you could also do::
 
-    with DAG("my_daily_dag", schedule_interval="0 * * * *"):
+    with DAG("my_dag", schedule_interval="0 0 * * *"):
         ...
 
-.. tip::
 
-    For more information on ``schedule_interval`` values, see :doc:`DAG Run </dag-run>`.
+This DAG will run every day at 00:00, as defined by the cron expression given to ``schedule_interval``. The value of ``schedule_interval`` can take several types:
+
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Type                                     | When to use                                                                                                          | Example                              |
++==========================================+======================================================================================================================+======================================+
+| ``None``                                 | No schedule, use for manually triggered DAGs                                                                         | ``None``                             |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron expression (``str``)                | To run at cron-based intervals                                                                                       + ``"0 0 * * *"``                      |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| Cron preset (``str``)                    | Convenience cron expression for readability                                                                          + ``"@daily"``                         |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| List of cron expressions/presets         | To run at intervals that cannot be expressed by a single cron expression.                                            + ``["0 3 * * *", "0 0 * * MON,TUE"]`` |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| ``datetime.timedelta``                   | To run at frequency-based intervals. Useful if your interval cannot be expressed by cron e.g. ``timedelta(days=3)``. + ``timedelta(days=3)``                |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+| ``dateutil.relativedelta.relativedelta`` | To express an interval in weeks, months, or years (which timedelta cannot do natively).                              + ``relativedelta(months=1)``          |
++------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------------------------------------+
+
+.. tip::
+    A convenient tool for converting cron expressions to human language is `Crontab.guru <https://crontab.guru/>`_.
 
-    If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables </howto/timetable>`.
-    For more information on ``logical date``, see :ref:`data-interval` and

Review Comment:
   Moved it down, after `logical_date` was explained.



-- 
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] BasPH commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by GitBox <gi...@apache.org>.
BasPH commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1172230918

   @uranusjr Added support for sets (& tuples). Didn't set the type to `Collection` though because that also includes dicts, which is not supported. Since there's no special type for Collections expect dicts (as far as I know?) the type is now `Union[List[str], Set[str], Tuple[str, ...]]`, does that work for you?


-- 
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] Inetov commented on pull request #24733: Add support for multiple cron expressions in schedule_interval

Posted by "Inetov (via GitHub)" <gi...@apache.org>.
Inetov commented on PR #24733:
URL: https://github.com/apache/airflow/pull/24733#issuecomment-1630574062

   Hi @BasPH !
   what does it take to finish this? very useful feature!


-- 
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