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/07/29 19:30:32 UTC

[GitHub] [airflow] dstandish opened a new pull request, #25410: One true schedule param

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

   Consolidate the three scheduling params to one param `schedule`


-- 
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 diff in pull request #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/dag-run.rst:
##########
@@ -117,8 +116,8 @@ DAG run fails.
 Catchup
 -------
 
-An Airflow DAG with a ``start_date``, possibly an ``end_date``, and a ``schedule_interval`` defines a
-series of intervals which the scheduler turns into individual DAG Runs and executes. The scheduler, by default, will
+An Airflow DAG defined with a ``start_date``, possibly an ``end_date``, and a either a cron expression or timetable, defines a series of intervals which the scheduler turns into individual DAG Runs and executes.

Review Comment:
   ```suggestion
   An Airflow DAG defined with a ``start_date``, possibly an ``end_date``, and either a cron expression or timetable, defines a series of intervals which the scheduler turns into individual DAG Runs and executes.
   ```
   
   I think the sentence still need a little love.



-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
tests/test_utils/perf/dags/elastic_dag.py:
##########
@@ -175,13 +175,13 @@ class DagShape(Enum):
                     f"DAGS_COUNT={dag_no}_of_{DAG_COUNT}",
                     f"TASKS_COUNT=${TASKS_COUNT}",
                     f"START_DATE=${START_DATE_ENV}",
-                    f"SCHEDULE_INTERVAL=${SCHEDULE_INTERVAL_ENV}",
+                    f"schedule=${SCHEDULE_INTERVAL_ENV}",

Review Comment:
   i guess it's to generate a dag_id. i'll make it uppercase.



-- 
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] dstandish commented on pull request #25410: Consolidate to one `schedule` param

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

   > @dstandish - the failing test in "providers 2.2" indicates backwards-incompatible changes in providers I am afraid.
   
   Ah... yes... this makes sense.... because... airflow 2.2 won't have new `schedule` param.... will have to come up with something for backcompat in the example dags stored in providers.


-- 
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 #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -236,11 +237,16 @@ class DAG(LoggingMixin):
     :param dag_id: The id of the DAG; must consist exclusively of alphanumeric
         characters, dashes, dots and underscores (all ASCII)
     :param description: The description for the DAG to e.g. be shown on the webserver
+    :param schedule: Defines the rules according to which DAG runs are scheduled. Can
+        accept cron string, timedelta object, Timetable, or list of Dataset objects.
+        See also :doc:`/howto/timetable`.
     :param schedule_interval: Defines how often that DAG runs, this
         timedelta object gets added to your latest task instance's
-        execution_date to figure out the next schedule
+        execution_date to figure out the next schedule.
+        Note: deprecated in Airflow 2.4; use `schedule` instead.
     :param timetable: Specify which timetable to use (in which case schedule_interval
         must not be set). See :doc:`/howto/timetable` for more information
+        Note: deprecated in Airflow 2.4; use `schedule` instead.

Review Comment:
   Should we just remove these entirely from the docstring? New users don’t need to know about them, and those who are using them don’t need to read the docs on them anyway.



##########
airflow/models/dag.py:
##########
@@ -313,7 +319,7 @@ class DAG(LoggingMixin):
         to render templates as native Python types. If False, a Jinja
         ``Environment`` is used to render templates as string values.
     :param tags: List of tags to help filtering DAGs in the UI.
-    :param schedule_on: List of upstream datasets if for use in triggering DAG runs.
+    :param schedule: List of upstream datasets if for use in triggering DAG runs.

Review Comment:
   Should be removed entirely.



##########
airflow/models/dag.py:
##########
@@ -460,18 +467,46 @@ def __init__(
             self.default_args['end_date'] = timezone.convert_to_utc(self.default_args['end_date'])
 
         # sort out DAG's scheduling behavior
-        scheduling_args = [schedule_interval, timetable, schedule_on]
+        scheduling_args = [schedule_interval, timetable, schedule]
         if not at_most_one(*scheduling_args):
-            raise ValueError(
-                "At most one allowed for args 'schedule_interval', 'timetable', and 'schedule_on'."
+            raise ValueError("At most one allowed for args 'schedule_interval', 'timetable', and 'schedule'.")
+        if schedule_interval is not NOTSET:
+            warnings.warn(
+                "Param `schedule_interval` is deprecated and will be removed in a future release. "
+                "Please use `schedule` instead. ",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if timetable is not None:
+            warnings.warn(
+                "Param `timetable` is deprecated and will be removed in a future release. "
+                "Please use `schedule` instead. ",
+                DeprecationWarning,
+                stacklevel=2,
             )
-
         self.timetable: Timetable
         self.schedule_interval: ScheduleInterval
-        self.schedule_on: Optional[List["Dataset"]] = list(schedule_on) if schedule_on else None
-        if schedule_on:
-            if not isinstance(schedule_on, Sequence):
-                raise ValueError("Param `schedule_on` must be Sequence[Dataset]")
+        self.dataset_triggers: Optional[List[Dataset]] = None
+
+        if schedule is not NOTSET:
+            if isinstance(schedule, List):
+                # if List, only support List[Dataset]
+                if any(isinstance(x, Dataset) for x in schedule):
+                    if not all(isinstance(x, Dataset) for x in schedule):
+                        raise ValueError(
+                            "If scheduling DAG with List[Dataset], all elements must be Dataset."
+                        )
+                    self.dataset_triggers = list(schedule)
+                else:
+                    raise ValueError(
+                        "Use of List object with `schedule` param is only supported for List[Dataset]."
+                    )
+            elif isinstance(schedule, Timetable):
+                timetable = schedule
+            else:  # assumed to be ScheduleIntervalArg
+                schedule_interval = schedule

Review Comment:
   ```suggestion
           if isinstance(schedule, List):
               # if List, only support List[Dataset]
               if any(isinstance(x, Dataset) for x in schedule):
                   if not all(isinstance(x, Dataset) for x in schedule):
                       raise ValueError(
                           "If scheduling DAG with List[Dataset], all elements must be Dataset."
                       )
                   self.dataset_triggers = list(schedule)
               else:
                   raise ValueError(
                       "Use of List object with `schedule` param is only supported for List[Dataset]."
                   )
           elif isinstance(schedule, Timetable):
               timetable = schedule
           elif schedule is not None:
               schedule_interval = schedule
   ```



-- 
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] potiuk commented on pull request #25410: Consolidate to one `schedule` param

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

   ![image](https://user-images.githubusercontent.com/595491/183364388-4bec2d10-670d-4ab4-aee0-3da7ff2a4dd6.png)
   


-- 
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] potiuk commented on pull request #25410: Consolidate to one `schedule` param

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

   @dstandish - the failing test in "providers 2.2" indicates backwards-incompatible changes in providers I am afraid.


-- 
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] dstandish commented on pull request #25410: Consolidate to one `schedule` param

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

   er i guess it could help just because in a big "example dags" PR you don't really need to scrutinize anything as carefully, and in a "core" change PR you would know to look at everything 🤷 


-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/templates-ref.rst:
##########
@@ -89,16 +89,11 @@ existing code to use other variables instead.
 Deprecated Variable                     Description
 =====================================   ====================================
 ``{{ execution_date }}``                the execution date (logical date), same as ``dag_run.logical_date``
-``{{ next_execution_date }}``           the next execution date (if available) (`pendulum.DateTime`_)
-                                        if ``{{ execution_date }}`` is ``2018-01-01 00:00:00`` and
-                                        ``schedule_interval`` is ``@weekly``, ``{{ next_execution_date }}``
-                                        will be ``2018-01-08 00:00:00``

Review Comment:
   it was intentional.  firstly, these are all *deprecated*, so folks reading the docs should not be using them anyway.  secondly, by referencing "logical date"  in the definition, we defer to the example in the section of non-deprecated items.
   
   note there is also no example for the also-deprecated value `execution_date`
   
   the example and explanation were confusing anyway so this to me  seem better



-- 
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 #25410: Consolidate to one `schedule` param

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


##########
tests/test_utils/perf/dags/elastic_dag.py:
##########
@@ -175,13 +175,13 @@ class DagShape(Enum):
                     f"DAGS_COUNT={dag_no}_of_{DAG_COUNT}",
                     f"TASKS_COUNT=${TASKS_COUNT}",
                     f"START_DATE=${START_DATE_ENV}",
-                    f"SCHEDULE_INTERVAL=${SCHEDULE_INTERVAL_ENV}",
+                    f"schedule=${SCHEDULE_INTERVAL_ENV}",

Review Comment:
   Not quite sure what this one is for, it’s not actually DAG arguments.



-- 
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 diff in pull request #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/dag-run.rst:
##########
@@ -82,7 +81,7 @@ 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 each day at midnight (00:00) and end at midnight
+ts data interval would start each day at midnight (00:00) and end at midnight

Review Comment:
   ```suggestion
   its data interval would start each day at midnight (00:00) and end at midnight
   ```



##########
docs/apache-airflow/faq.rst:
##########
@@ -41,13 +41,13 @@ There are very many reasons why your task might not be getting scheduled. Here a
   "airflow" and "DAG" in order to prevent the DagBag parsing from importing all python
   files collocated with user's DAGs.
 
-- Is your ``start_date`` set properly? The Airflow scheduler triggers the
-  task soon after the ``start_date + schedule_interval`` is passed.
+- Is your ``start_date`` set properly? The Airflow scheduler won't trigger the task until the

Review Comment:
   ```suggestion
   - Is your ``start_date`` set properly? The Airflow scheduler won't trigger the task until
   ```



##########
scripts/in_container/verify_providers.py:
##########
@@ -200,6 +200,7 @@ class ProviderPackageDetails(NamedTuple):
     "distutils Version classes are deprecated. Use packaging.version instead.",
     "the imp module is deprecated in favour of importlib; "
     "see the module's documentation for alternative uses",
+    "see the module's documentation for alternative uses",

Review Comment:
   ```suggestion
   ```



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,3 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.

Review Comment:
   Might be worth adding a code example, "this becomes this" type thing.



##########
docs/apache-airflow/faq.rst:
##########
@@ -41,13 +41,13 @@ There are very many reasons why your task might not be getting scheduled. Here a
   "airflow" and "DAG" in order to prevent the DagBag parsing from importing all python
   files collocated with user's DAGs.
 
-- Is your ``start_date`` set properly? The Airflow scheduler triggers the
-  task soon after the ``start_date + schedule_interval`` is passed.
+- Is your ``start_date`` set properly? The Airflow scheduler won't trigger the task until the
+  after the first schedule interval following the start date has passed.

Review Comment:
   Might be worth adding a simple example here, since the "start_date+schedule_interval" doesn't work as cleanly any longer.



##########
docs/apache-airflow/dag-run.rst:
##########
@@ -117,8 +116,8 @@ DAG run fails.
 Catchup
 -------
 
-An Airflow DAG with a ``start_date``, possibly an ``end_date``, and a ``schedule_interval`` defines a
-series of intervals which the scheduler turns into individual DAG Runs and executes. The scheduler, by default, will
+An Airflow DAG defined with a ``start_date``, possibly an ``end_date``, and a either a cron expression or timetable, defines a series of intervals which the scheduler turns into individual DAG Runs and executes.

Review Comment:
   ```suggestion
   An Airflow DAG defined with a ``start_date``(and possibly an ``end_date``), and either a cron expression or timetable, defines a series of intervals which the scheduler turns into individual DAG Runs and executes.
   ```
   
   I think the sentence still need a little love.



-- 
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 #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/concepts/tasks.rst:
##########
@@ -170,7 +170,7 @@ To read more about configuring the emails, see :doc:`/howto/email-config`.
 
 .. note::
 
-    Only scheduled tasks will be checked against SLA. For example, manually triggered tasks will not invoke an SLA miss. For more information on ``schedule_interval`` values see :doc:`DAG Run </dag-run>`.
+    Manually-triggered tasks and tasks in event-driven dags will not be checked for an SLA miss. For more information on DAG ``schedule`` values see :doc:`DAG Run </dag-run>`.

Review Comment:
   ```suggestion
       Manually-triggered tasks and tasks in event-driven DAGs will not be checked for an SLA miss. For more information on DAG ``schedule`` values see :doc:`DAG Run </dag-run>`.
   ```



-- 
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 diff in pull request #25410: Consolidate to one `schedule` param

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


##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.
+
+So if you previously used the ``@daily`` cron preset, your dag may have looked like this:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule_interval='@daily',
+    ) as dag:
+        ...
+
+Going forward, instead you should use the ``schedule`` param instead:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule='@daily',
+    ) as dag:
+        ...
+
+And the same is true if you used a custom timetable.  Previously you would have used the ``timetable`` param:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]),
+    ) as dag:
+        ...
+
+Now you should use the ``schedule`` param:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]),

Review Comment:
   ```suggestion
           schedule=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]),
   ```



-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -460,18 +467,46 @@ def __init__(
             self.default_args['end_date'] = timezone.convert_to_utc(self.default_args['end_date'])
 
         # sort out DAG's scheduling behavior
-        scheduling_args = [schedule_interval, timetable, schedule_on]
+        scheduling_args = [schedule_interval, timetable, schedule]
         if not at_most_one(*scheduling_args):
-            raise ValueError(
-                "At most one allowed for args 'schedule_interval', 'timetable', and 'schedule_on'."
+            raise ValueError("At most one allowed for args 'schedule_interval', 'timetable', and 'schedule'.")
+        if schedule_interval is not NOTSET:
+            warnings.warn(
+                "Param `schedule_interval` is deprecated and will be removed in a future release. "
+                "Please use `schedule` instead. ",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if timetable is not None:
+            warnings.warn(
+                "Param `timetable` is deprecated and will be removed in a future release. "
+                "Please use `schedule` instead. ",
+                DeprecationWarning,
+                stacklevel=2,
             )
-
         self.timetable: Timetable
         self.schedule_interval: ScheduleInterval
-        self.schedule_on: Optional[List["Dataset"]] = list(schedule_on) if schedule_on else None
-        if schedule_on:
-            if not isinstance(schedule_on, Sequence):
-                raise ValueError("Param `schedule_on` must be Sequence[Dataset]")
+        self.dataset_triggers: Optional[List[Dataset]] = None
+
+        if schedule is not NOTSET:
+            if isinstance(schedule, List):
+                # if List, only support List[Dataset]
+                if any(isinstance(x, Dataset) for x in schedule):
+                    if not all(isinstance(x, Dataset) for x in schedule):
+                        raise ValueError(
+                            "If scheduling DAG with List[Dataset], all elements must be Dataset."
+                        )
+                    self.dataset_triggers = list(schedule)
+                else:
+                    raise ValueError(
+                        "Use of List object with `schedule` param is only supported for List[Dataset]."
+                    )
+            elif isinstance(schedule, Timetable):
+                timetable = schedule
+            else:  # assumed to be ScheduleIntervalArg
+                schedule_interval = schedule

Review Comment:
   why this change @uranusjr 
   



-- 
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] potiuk commented on pull request #25410: Consolidate to one `schedule` param

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

   > Wow, 356 files changed. Would it be a good idea to split out the part that consolidates arguement handling, and change the examples and documentation in a later PR?
   
   I'd prefer to keep those together, even if it's a big change. There is a little value in splitting them, Especially with the new Github Review experience with "change file tree" it's easy to separate out the "code" from "documentation changes during review - and keeping docs and code changes in the same PR makes it better easier to cherry-pick and reason about.


-- 
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] dstandish commented on pull request #25410: Consolidate to one `schedule` param

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

   @uranusjr @potiuk re separation... yeah since it would just be splitting out files, rather than e.g. steps in series of changes, not sure it would help review all that much.  but also... the main reason i included example dags is because in past at least we had CI tests that checked for warnings, and every example dag would produce a deprecation warning if i did not update them.


-- 
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 diff in pull request #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/templates-ref.rst:
##########
@@ -89,16 +89,11 @@ existing code to use other variables instead.
 Deprecated Variable                     Description
 =====================================   ====================================
 ``{{ execution_date }}``                the execution date (logical date), same as ``dag_run.logical_date``
-``{{ next_execution_date }}``           the next execution date (if available) (`pendulum.DateTime`_)
-                                        if ``{{ execution_date }}`` is ``2018-01-01 00:00:00`` and
-                                        ``schedule_interval`` is ``@weekly``, ``{{ next_execution_date }}``
-                                        will be ``2018-01-08 00:00:00``

Review Comment:
   I don't think the intent was to remove the example?



-- 
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] dstandish merged pull request #25410: Consolidate to one `schedule` param

Posted by GitBox <gi...@apache.org>.
dstandish merged PR #25410:
URL: https://github.com/apache/airflow/pull/25410


-- 
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 #25410: Consolidate to one `schedule` param

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


##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.
+
+So if you previously used the ``@daily`` cron preset, your dag may have looked like this:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule_interval='@daily',
+    ) as dag:
+        ...
+
+Going forward, instead you should use the ``schedule`` param instead:

Review Comment:
   ```suggestion
   Going forward, you should use the ``schedule`` argument instead:
   ```



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.
+
+So if you previously used the ``@daily`` cron preset, your dag may have looked like this:

Review Comment:
   ```suggestion
   If you previously used the ``@daily`` cron preset, your DAG may have looked like this:
   ```



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.
+
+So if you previously used the ``@daily`` cron preset, your dag may have looked like this:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule_interval='@daily',
+    ) as dag:
+        ...
+
+Going forward, instead you should use the ``schedule`` param instead:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule='@daily',
+    ) as dag:
+        ...
+
+And the same is true if you used a custom timetable.  Previously you would have used the ``timetable`` param:

Review Comment:
   ```suggestion
   The same is true if you used a custom timetable.  Previously you would have used the ``timetable`` argument:
   ```



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.
+
+So if you previously used the ``@daily`` cron preset, your dag may have looked like this:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule_interval='@daily',
+    ) as dag:
+        ...
+
+Going forward, instead you should use the ``schedule`` param instead:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        schedule='@daily',
+    ) as dag:
+        ...
+
+And the same is true if you used a custom timetable.  Previously you would have used the ``timetable`` param:
+
+.. code-block:: python
+    with DAG(
+        dag_id='my_example',
+        start_date=datetime(2021, 1, 1),
+        timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]),
+    ) as dag:
+        ...
+
+Now you should use the ``schedule`` param:

Review Comment:
   ```suggestion
   Now you should use the ``schedule`` argument:
   ```



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params

Review Comment:
   ```suggestion
   Deprecation of ``schedule_interval`` and ``timetable`` arguments
   ```



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.

Review Comment:
   ```suggestion
   We added new DAG argument ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Arguments ``schedule_interval`` and ``timetable`` are deprecated.
   ```



-- 
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 #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/concepts/timetable.rst:
##########
@@ -19,16 +19,18 @@
 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>`, or by passing a ``timetable`` argument
-directly. The timetable also dictates the data interval and the logical time of each
-run created for the DAG.
+For dags with time-based schedules (as opposed to event-driven), the scheduling

Review Comment:
   ```suggestion
   For DAGs with time-based schedules (as opposed to event-driven), the scheduling
   ```



##########
docs/apache-airflow/concepts/datasets.rst:
##########
@@ -37,11 +37,11 @@ Then reference the dataset as a task outlet:
     :start-after: [START task_outlet]
     :end-before: [END task_outlet]
 
-Finally, define a DAG and reference this dataset in the DAG's ``schedule_on`` parameter:
+Finally, define a DAG and reference this dataset in the DAG's ``schedule`` parameter:
 
 .. exampleinclude:: /../../airflow/example_dags/example_datasets.py
     :language: python
     :start-after: [START dag_dep]
     :end-before: [END dag_dep]
 
-You can reference multiple datasets in the DAG's ``schedule_on`` param.  Once there has been an update to all of the upstream datasets, the DAG will be triggered.  This means that the DAG will run as frequently as its least-frequently-updated dataset.
+You can reference multiple datasets in the DAG's ``schedule`` param.  Once there has been an update to all of the upstream datasets, the DAG will be triggered.  This means that the DAG will run as frequently as its least-frequently-updated dataset.

Review Comment:
   ```suggestion
   You can reference multiple datasets in the DAG's ``schedule`` argument.  Once there has been an update to all of the upstream datasets, the DAG will be triggered.  This means that the DAG will run as frequently as its least-frequently-updated dataset.
   ```



##########
docs/apache-airflow/concepts/timetable.rst:
##########
@@ -19,16 +19,18 @@
 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>`, or by passing a ``timetable`` argument
-directly. The timetable also dictates the data interval and the logical time of each
-run created for the DAG.
+For dags with time-based schedules (as opposed to event-driven), the scheduling
+decisions are driven by its internal "timetable".  The timetable class also

Review Comment:
   ```suggestion
   decisions are driven by its internal "timetable".  The timetable also
   ```



##########
docs/apache-airflow/concepts/timetable.rst:
##########
@@ -19,16 +19,18 @@
 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>`, or by passing a ``timetable`` argument
-directly. The timetable also dictates the data interval and the logical time of each
-run created for the DAG.
+For dags with time-based schedules (as opposed to event-driven), the scheduling
+decisions are driven by its internal "timetable".  The timetable class also
+determines the data interval and the logical date of each run created for the DAG.
 
-Cron expressions and timedeltas are still supported (using
-`CronDataIntervalTimetable`_ and `DeltaDataIntervalTimetable`_ under the hood
-respectively), however, there are situations where they cannot properly express
-the schedule. Some examples are:
+Even for dags whose scheduling is defined by a cron expression or ``timedelta`` object,
+internally these expressions converted to a timetable.

Review Comment:
   ```suggestion
   DAGs scheduled with a cron expression or ``timedelta`` object are
   internally converted to always use a timetable.
   ```



##########
docs/apache-airflow/faq.rst:
##########
@@ -41,13 +41,13 @@ There are very many reasons why your task might not be getting scheduled. Here a
   "airflow" and "DAG" in order to prevent the DagBag parsing from importing all python
   files collocated with user's DAGs.
 
-- Is your ``start_date`` set properly? The Airflow scheduler triggers the
-  task soon after the ``start_date + schedule_interval`` is passed.
+- Is your ``start_date`` set properly? For time-based dags, the task won't be triggered until the

Review Comment:
   ```suggestion
   - Is your ``start_date`` set properly? For time-based DAGs, the task won't be triggered until the
   ```



##########
docs/apache-airflow/concepts/datasets.rst:
##########
@@ -37,11 +37,11 @@ Then reference the dataset as a task outlet:
     :start-after: [START task_outlet]
     :end-before: [END task_outlet]
 
-Finally, define a DAG and reference this dataset in the DAG's ``schedule_on`` parameter:
+Finally, define a DAG and reference this dataset in the DAG's ``schedule`` parameter:

Review Comment:
   ```suggestion
   Finally, define a DAG and reference this dataset in the DAG's ``schedule`` argument:
   ```



##########
docs/apache-airflow/concepts/timetable.rst:
##########
@@ -19,16 +19,18 @@
 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>`, or by passing a ``timetable`` argument
-directly. The timetable also dictates the data interval and the logical time of each
-run created for the DAG.
+For dags with time-based schedules (as opposed to event-driven), the scheduling
+decisions are driven by its internal "timetable".  The timetable class also
+determines the data interval and the logical date of each run created for the DAG.
 
-Cron expressions and timedeltas are still supported (using
-`CronDataIntervalTimetable`_ and `DeltaDataIntervalTimetable`_ under the hood
-respectively), however, there are situations where they cannot properly express
-the schedule. Some examples are:
+Even for dags whose scheduling is defined by a cron expression or ``timedelta`` object,
+internally these expressions converted to a timetable.
+
+If a cron expression or ``timedelta`` is sufficient for your use case, you don't need
+to worry about timetables.  But for more complicated scheduling requirements,
+you may create your own timetable class and pass that to the DAG ``schedule`` param.

Review Comment:
   ```suggestion
   If a cron expression or ``timedelta`` is sufficient for your use case, you don't need
   to worry about timetables.  But for more complicated scheduling requirements,
   you may create your own timetable class and pass that to the DAG's ``schedule`` argument instead.
   ```



##########
docs/apache-airflow/dag-run.rst:
##########
@@ -117,8 +116,8 @@ DAG run fails.
 Catchup
 -------
 
-An Airflow DAG with a ``start_date``, possibly an ``end_date``, and a ``schedule_interval`` defines a
-series of intervals which the scheduler turns into individual DAG Runs and executes. The scheduler, by default, will
+An Airflow DAG defined with a ``start_date``, possibly an ``end_date``, and either a cron expression or timetable, defines a series of intervals which the scheduler turns into individual DAG Runs and executes.

Review Comment:
   ```suggestion
   An Airflow DAG defined with a ``start_date``, possibly an ``end_date``, and a non-dataset schedule, defines a series of intervals which the scheduler turns into individual DAG runs and executes.
   ```



##########
docs/apache-airflow/faq.rst:
##########
@@ -41,13 +41,13 @@ There are very many reasons why your task might not be getting scheduled. Here a
   "airflow" and "DAG" in order to prevent the DagBag parsing from importing all python
   files collocated with user's DAGs.
 
-- Is your ``start_date`` set properly? The Airflow scheduler triggers the
-  task soon after the ``start_date + schedule_interval`` is passed.
+- Is your ``start_date`` set properly? For time-based dags, the task won't be triggered until the
+  the first schedule interval following the start date has passed.
 
-- Is your ``schedule_interval`` set properly? The default ``schedule_interval``
-  is one day (``datetime.timedelta(1)``). You must specify a different ``schedule_interval``
+- Is your ``schedule`` param set properly? The default

Review Comment:
   ```suggestion
   - Is your ``schedule`` argument set properly? The default
   ```



##########
docs/apache-airflow/templates-ref.rst:
##########
@@ -89,16 +89,11 @@ existing code to use other variables instead.
 Deprecated Variable                     Description
 =====================================   ====================================
 ``{{ execution_date }}``                the execution date (logical date), same as ``dag_run.logical_date``
-``{{ next_execution_date }}``           the next execution date (if available) (`pendulum.DateTime`_)
-                                        if ``{{ execution_date }}`` is ``2018-01-01 00:00:00`` and
-                                        ``schedule_interval`` is ``@weekly``, ``{{ next_execution_date }}``
-                                        will be ``2018-01-08 00:00:00``
+``{{ next_execution_date }}``           the logical date of the next scheduled run (if applicable)
+                                        You may be able to use ``data_interval_end`` instead.

Review Comment:
   ```suggestion
   ``{{ next_execution_date }}``           the logical date of the next scheduled run (if applicable);
                                           you may be able to use ``data_interval_end`` instead
   ```



##########
docs/apache-airflow/faq.rst:
##########
@@ -376,7 +376,7 @@ Why ``next_ds`` or ``prev_ds`` might not contain expected values?
 ------------------------------------------------------------------
 
 - When scheduling DAG, the ``next_ds`` ``next_ds_nodash`` ``prev_ds`` ``prev_ds_nodash`` are calculated using
-  ``execution_date`` and ``schedule_interval``. If you set ``schedule_interval`` as ``None`` or ``@once``,
+  ``logical_date`` and the dag's schedule interval (if applicable). If you set ``schedule`` as ``None`` or ``@once``,

Review Comment:
   ```suggestion
     ``logical_date`` and the DAG's schedule interval (if applicable). If you set ``schedule`` as ``None`` or ``@once``,
   ```



##########
docs/apache-airflow/dag-run.rst:
##########
@@ -43,10 +43,9 @@ There are two possible terminal states for the DAG Run:
 Cron Presets
 ''''''''''''
 
-Each DAG may or may not have a schedule, which informs how DAG Runs are
-created. ``schedule_interval`` is defined as a DAG argument, which can be passed a
-`cron expression <https://en.wikipedia.org/wiki/Cron#CRON_expression>`_ as
-a ``str``, a ``datetime.timedelta`` object, or one of the following cron "presets".
+You may set your DAG to run on a simple schedule by setting its ``schedule`` param with either a
+`cron expression <https://en.wikipedia.org/wiki/Cron#CRON_expression>`_, or a ``datetime.timedelta`` object,
+or one of the following cron "presets". For more elaborate scheduling requirements, you can implement a :doc:`custom timetable </concepts/timetable>`

Review Comment:
   ```suggestion
   You may set your DAG to run on a simple schedule by setting its ``schedule`` argument to either a
   `cron expression <https://en.wikipedia.org/wiki/Cron#CRON_expression>`_, a ``datetime.timedelta`` object,
   or one of the following cron "presets". For more elaborate scheduling requirements, you can implement a :doc:`custom timetable </concepts/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] uranusjr commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -236,11 +237,16 @@ class DAG(LoggingMixin):
     :param dag_id: The id of the DAG; must consist exclusively of alphanumeric
         characters, dashes, dots and underscores (all ASCII)
     :param description: The description for the DAG to e.g. be shown on the webserver
+    :param schedule: Defines the rules according to which DAG runs are scheduled. Can
+        accept cron string, timedelta object, Timetable, or list of Dataset objects.
+        See also :doc:`/howto/timetable`.
     :param schedule_interval: Defines how often that DAG runs, this
         timedelta object gets added to your latest task instance's
-        execution_date to figure out the next schedule
+        execution_date to figure out the next schedule.
+        Note: deprecated in Airflow 2.4; use `schedule` instead.
     :param timetable: Specify which timetable to use (in which case schedule_interval
         must not be set). See :doc:`/howto/timetable` for more information
+        Note: deprecated in Airflow 2.4; use `schedule` instead.

Review Comment:
   One approach used elsewhere is to document the existence of deprecated arguments in the new replacement (i.e. `schedule`) and say something like _hey if you see `schedule_interval` or `timetable` they are deprecated forms and synonym of this_. This is slightly more difficult here since `DAG` has so many arguments, but would probably still be good enough.



-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -236,11 +237,16 @@ class DAG(LoggingMixin):
     :param dag_id: The id of the DAG; must consist exclusively of alphanumeric
         characters, dashes, dots and underscores (all ASCII)
     :param description: The description for the DAG to e.g. be shown on the webserver
+    :param schedule: Defines the rules according to which DAG runs are scheduled. Can
+        accept cron string, timedelta object, Timetable, or list of Dataset objects.
+        See also :doc:`/howto/timetable`.
     :param schedule_interval: Defines how often that DAG runs, this
         timedelta object gets added to your latest task instance's
-        execution_date to figure out the next schedule
+        execution_date to figure out the next schedule.
+        Note: deprecated in Airflow 2.4; use `schedule` instead.
     :param timetable: Specify which timetable to use (in which case schedule_interval
         must not be set). See :doc:`/howto/timetable` for more information
+        Note: deprecated in Airflow 2.4; use `schedule` instead.

Review Comment:
   Maybe better than what i've done currently is at least to put the deprecation warning *before* the description... anyway... lmk



-- 
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 #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -460,18 +467,46 @@ def __init__(
             self.default_args['end_date'] = timezone.convert_to_utc(self.default_args['end_date'])
 
         # sort out DAG's scheduling behavior
-        scheduling_args = [schedule_interval, timetable, schedule_on]
+        scheduling_args = [schedule_interval, timetable, schedule]
         if not at_most_one(*scheduling_args):
-            raise ValueError(
-                "At most one allowed for args 'schedule_interval', 'timetable', and 'schedule_on'."
+            raise ValueError("At most one allowed for args 'schedule_interval', 'timetable', and 'schedule'.")
+        if schedule_interval is not NOTSET:
+            warnings.warn(
+                "Param `schedule_interval` is deprecated and will be removed in a future release. "
+                "Please use `schedule` instead. ",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if timetable is not None:
+            warnings.warn(
+                "Param `timetable` is deprecated and will be removed in a future release. "
+                "Please use `schedule` instead. ",
+                DeprecationWarning,
+                stacklevel=2,
             )
-
         self.timetable: Timetable
         self.schedule_interval: ScheduleInterval
-        self.schedule_on: Optional[List["Dataset"]] = list(schedule_on) if schedule_on else None
-        if schedule_on:
-            if not isinstance(schedule_on, Sequence):
-                raise ValueError("Param `schedule_on` must be Sequence[Dataset]")
+        self.dataset_triggers: Optional[List[Dataset]] = None
+
+        if schedule is not NOTSET:
+            if isinstance(schedule, List):
+                # if List, only support List[Dataset]
+                if any(isinstance(x, Dataset) for x in schedule):
+                    if not all(isinstance(x, Dataset) for x in schedule):
+                        raise ValueError(
+                            "If scheduling DAG with List[Dataset], all elements must be Dataset."
+                        )
+                    self.dataset_triggers = list(schedule)
+                else:
+                    raise ValueError(
+                        "Use of List object with `schedule` param is only supported for List[Dataset]."
+                    )
+            elif isinstance(schedule, Timetable):
+                timetable = schedule
+            else:  # assumed to be ScheduleIntervalArg
+                schedule_interval = schedule

Review Comment:
   Just simpler? The original implementation has some unnecessary nesting.



-- 
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 #25410: Consolidate to one `schedule` param

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

   Wow, 356 files changed. Would it be a good idea to split out the part that consolidates arguement handling, and change the examples and documentation in a later PR?


-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -236,11 +237,16 @@ class DAG(LoggingMixin):
     :param dag_id: The id of the DAG; must consist exclusively of alphanumeric
         characters, dashes, dots and underscores (all ASCII)
     :param description: The description for the DAG to e.g. be shown on the webserver
+    :param schedule: Defines the rules according to which DAG runs are scheduled. Can
+        accept cron string, timedelta object, Timetable, or list of Dataset objects.
+        See also :doc:`/howto/timetable`.
     :param schedule_interval: Defines how often that DAG runs, this
         timedelta object gets added to your latest task instance's
-        execution_date to figure out the next schedule
+        execution_date to figure out the next schedule.
+        Note: deprecated in Airflow 2.4; use `schedule` instead.
     :param timetable: Specify which timetable to use (in which case schedule_interval
         must not be set). See :doc:`/howto/timetable` for more information
+        Note: deprecated in Airflow 2.4; use `schedule` instead.

Review Comment:
   well i'll defer to your judgment....  i.e. if you say let's remove it, i'm happy to do... just... from my perspective it seems odd.



-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/faq.rst:
##########
@@ -41,13 +41,13 @@ There are very many reasons why your task might not be getting scheduled. Here a
   "airflow" and "DAG" in order to prevent the DagBag parsing from importing all python
   files collocated with user's DAGs.
 
-- Is your ``start_date`` set properly? The Airflow scheduler triggers the
-  task soon after the ``start_date + schedule_interval`` is passed.
+- Is your ``start_date`` set properly? The Airflow scheduler won't trigger the task until the
+  after the first schedule interval following the start date has passed.

Review Comment:
   reworded to clarify context



##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,3 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.

Review Comment:
   done



-- 
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 #25410: Consolidate to one `schedule` param

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


##########
newsfragments/25410.significant.rst:
##########
@@ -0,0 +1,43 @@
+Deprecation of ``schedule_interval`` and ``timetable`` params
+
+We add new DAG parameter ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Params ``schedule_interval`` and ``timetable`` are  deprecated.

Review Comment:
   ```suggestion
   We added new DAG argument ``schedule`` that can accept a cron expression, timedelta object, *timetable* object, or list of dataset objects. Arguments ``schedule_interval`` and ``timetable`` are deprecated.
   ```



-- 
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 #25410: Consolidate to one `schedule` param

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


##########
docs/apache-airflow/dag-run.rst:
##########
@@ -102,7 +101,7 @@ scheduled one interval after ``start_date``.
 
 .. tip::
 
-    If ``schedule_interval`` is not enough to express your DAG's schedule,
+    If a cron expression or `timedelta` object is not enough to express your DAG's schedule,

Review Comment:
   ```suggestion
       If a cron expression or timedelta object is not enough to express your DAG's schedule,
   ```



##########
docs/apache-airflow/faq.rst:
##########
@@ -376,7 +376,7 @@ Why ``next_ds`` or ``prev_ds`` might not contain expected values?
 ------------------------------------------------------------------
 
 - When scheduling DAG, the ``next_ds`` ``next_ds_nodash`` ``prev_ds`` ``prev_ds_nodash`` are calculated using
-  ``execution_date`` and ``schedule_interval``. If you set ``schedule_interval`` as ``None`` or ``@once``,
+  ``logical_date`` and the DAG's schedule interval (if applicable). If you set ``schedule`` as ``None`` or ``@once``,

Review Comment:
   ```suggestion
     ``logical_date`` and the DAG's schedule (if applicable). If you set ``schedule`` as ``None`` or ``@once``,
   ```



##########
docs/apache-airflow/faq.rst:
##########
@@ -224,15 +224,15 @@ an hour after now as ``now()`` moves along.
 
 
 Previously, we also recommended using rounded ``start_date`` in relation to your
-``schedule_interval``. This meant an ``@hourly`` would be at ``00:00``
+DAG ``schedule``. This meant an ``@hourly`` would be at ``00:00``

Review Comment:
   ```suggestion
   DAG's ``schedule``. This meant an ``@hourly`` would be at ``00:00``
   ```



-- 
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] dstandish commented on a diff in pull request #25410: Consolidate to one `schedule` param

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


##########
airflow/models/dag.py:
##########
@@ -236,11 +237,16 @@ class DAG(LoggingMixin):
     :param dag_id: The id of the DAG; must consist exclusively of alphanumeric
         characters, dashes, dots and underscores (all ASCII)
     :param description: The description for the DAG to e.g. be shown on the webserver
+    :param schedule: Defines the rules according to which DAG runs are scheduled. Can
+        accept cron string, timedelta object, Timetable, or list of Dataset objects.
+        See also :doc:`/howto/timetable`.
     :param schedule_interval: Defines how often that DAG runs, this
         timedelta object gets added to your latest task instance's
-        execution_date to figure out the next schedule
+        execution_date to figure out the next schedule.
+        Note: deprecated in Airflow 2.4; use `schedule` instead.
     :param timetable: Specify which timetable to use (in which case schedule_interval
         must not be set). See :doc:`/howto/timetable` for more information
+        Note: deprecated in Airflow 2.4; use `schedule` instead.

Review Comment:
   it is tempting. _however_, i think it 's better to leave like this because a user who encounters them might look to the docstring to understand what they are, and won't be able to easily find out that they are deprecated params.  they might go on a bit of a wild goose chase to find out.



##########
airflow/models/dag.py:
##########
@@ -236,11 +237,16 @@ class DAG(LoggingMixin):
     :param dag_id: The id of the DAG; must consist exclusively of alphanumeric
         characters, dashes, dots and underscores (all ASCII)
     :param description: The description for the DAG to e.g. be shown on the webserver
+    :param schedule: Defines the rules according to which DAG runs are scheduled. Can
+        accept cron string, timedelta object, Timetable, or list of Dataset objects.
+        See also :doc:`/howto/timetable`.
     :param schedule_interval: Defines how often that DAG runs, this
         timedelta object gets added to your latest task instance's
-        execution_date to figure out the next schedule
+        execution_date to figure out the next schedule.
+        Note: deprecated in Airflow 2.4; use `schedule` instead.
     :param timetable: Specify which timetable to use (in which case schedule_interval
         must not be set). See :doc:`/howto/timetable` for more information
+        Note: deprecated in Airflow 2.4; use `schedule` instead.

Review Comment:
   wdyt?



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