From c30cb1551487fa6c46b7801d41ec0a8556f41aa9 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 07:40:28 -0700 Subject: [PATCH 01/38] Rename DAG `schedule_on` parameter to `schedule` The intention is that `schedule` becomes the single DAG scheduling param, and that we will deprecate `schedule_interval` and `timetable`. --- airflow/example_dags/example_datasets.py | 8 ++--- airflow/models/dag.py | 35 +++++++++++-------- airflow/models/dagrun.py | 2 +- airflow/serialization/schema.json | 2 +- airflow/serialization/serialized_objects.py | 6 ++-- airflow/timetables/simple.py | 3 +- docs/apache-airflow/concepts/datasets.rst | 4 +-- .../endpoints/test_dag_run_endpoint.py | 4 +-- tests/dags/test_datasets.py | 4 +-- tests/models/test_dag.py | 10 +++--- tests/models/test_dagrun.py | 4 +-- 11 files changed, 43 insertions(+), 39 deletions(-) diff --git a/airflow/example_dags/example_datasets.py b/airflow/example_dags/example_datasets.py index 41351bad96562..8e5967e77e0da 100644 --- a/airflow/example_dags/example_datasets.py +++ b/airflow/example_dags/example_datasets.py @@ -71,7 +71,7 @@ dag_id='example_dataset_dag3_req_dag1', catchup=False, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_on=[dag1_dataset], + schedule=[dag1_dataset], tags=['downstream'], ) as dag3: # [END dag_dep] @@ -85,7 +85,7 @@ dag_id='example_dataset_dag4_req_dag1_dag2', catchup=False, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_on=[dag1_dataset, dag2_dataset], + schedule=[dag1_dataset, dag2_dataset], tags=['downstream'], ) as dag4: BashOperator( @@ -98,7 +98,7 @@ dag_id='example_dataset_dag5_req_dag1_D', catchup=False, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_on=[ + schedule=[ dag1_dataset, Dataset('s3://this-dataset-doesnt-get-triggered'), ], @@ -114,7 +114,7 @@ dag_id='example_dataset_dag6_req_DD', catchup=False, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_on=[ + schedule=[ Dataset('s3://unrelated/dataset3.txt'), Dataset('s3://unrelated/dataset_other_unknown.txt'), ], diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 664e680f1a22f..2ae20e77d5d34 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -115,6 +115,7 @@ # but Mypy cannot handle that right now. Track progress of PEP 661 for progress. # See also: https://discuss.python.org/t/9126/7 ScheduleIntervalArg = Union[ArgNotSet, ScheduleInterval] +ScheduleArg = Union[ArgNotSet, ScheduleInterval, Timetable, List["Dataset"]] SLAMissCallback = Callable[["DAG", str, str, List["SlaMiss"], List[TaskInstance]], None] @@ -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. :param start_date: The timestamp from which the scheduler will attempt to backfill :param end_date: A date beyond which your DAG won't run, leave to None @@ -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. :param owner_links: Dict of owners and their links, that will be clickable on the DAGs view UI. Can be used as an HTTP link (for example the link to your Slack channel), or a mailto link. e.g: {"dag_owner": "https://airflow.apache.org/"} @@ -349,6 +355,7 @@ def __init__( self, dag_id: str, description: Optional[str] = None, + schedule: ScheduleArg = NOTSET, schedule_interval: ScheduleIntervalArg = NOTSET, timetable: Optional[Timetable] = None, start_date: Optional[datetime] = None, @@ -376,9 +383,9 @@ def __init__( jinja_environment_kwargs: Optional[Dict] = None, render_template_as_native_obj: bool = False, tags: Optional[List[str]] = None, - schedule_on: Optional[Sequence["Dataset"]] = None, owner_links: Optional[Dict[str, str]] = None, ): + from airflow.models.dataset import Dataset from airflow.utils.task_group import TaskGroup if tags and any(len(tag) > TAG_MAX_LEN for tag in tags): @@ -460,18 +467,16 @@ 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'.") + dataset_triggers = [] + if schedule and isinstance(schedule, List) and isinstance(schedule[0], Dataset): + dataset_triggers = schedule 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]] = list(dataset_triggers) if dataset_triggers else None + if dataset_triggers: self.timetable = DatasetTriggeredTimetable() self.schedule_interval = self.timetable.summary elif timetable: @@ -2625,7 +2630,7 @@ class InletRef(NamedTuple): outlet_datasets = set() input_datasets = set() for dag in dags: - for dataset in dag.schedule_on or []: + for dataset in dag.dataset_triggers or []: dag_references.add(InletRef(dag.dag_id, dataset.uri)) input_datasets.add(dataset) for task in dag.tasks: @@ -3221,7 +3226,7 @@ def dag( jinja_environment_kwargs: Optional[Dict] = None, render_template_as_native_obj: bool = False, tags: Optional[List[str]] = None, - schedule_on: Optional[Sequence["Dataset"]] = None, + schedule: Optional[Sequence["Dataset"]] = None, owner_links: Optional[Dict[str, str]] = None, ) -> Callable[[Callable], Callable[..., DAG]]: """ @@ -3273,7 +3278,7 @@ def factory(*args, **kwargs): jinja_environment_kwargs=jinja_environment_kwargs, render_template_as_native_obj=render_template_as_native_obj, tags=tags, - schedule_on=schedule_on, + schedule=schedule, owner_links=owner_links, ) as dag_obj: # Set DAG documentation from function documentation. diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index c3ac204dfe804..abc02ef1852eb 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -701,7 +701,7 @@ def _process_dataset_dagrun_events(self, *, session=NEW_SESSION): self.log.warning("Could not find serialized DAG %s", target_dag_id) continue dag = row.dag - if dag.schedule_on: + if dag.dataset_triggers: dag.create_dagrun( run_type=DagRunType.DATASET_TRIGGERED, run_id=self.generate_run_id( diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 80d579f567831..7c7dc742f5626 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -118,7 +118,7 @@ { "$ref": "#/definitions/typed_relativedelta" } ] }, - "schedule_on":{ + "dataset_triggers":{ "anyOf": [ { "type": "null" }, { diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index ddbfcce868a99..2ec8db7daf697 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -589,7 +589,7 @@ def detect_task_dependencies(task: Operator) -> List['DagDependency']: @staticmethod def detect_dag_dependencies(dag: Optional[DAG]) -> List["DagDependency"]: """Detects dependencies set directly on the DAG object.""" - if dag and dag.schedule_on: + if dag and dag.dataset_triggers: return [ DagDependency( source="dataset", @@ -597,7 +597,7 @@ def detect_dag_dependencies(dag: Optional[DAG]) -> List["DagDependency"]: dependency_type="dataset", dependency_id=x.uri, ) - for x in dag.schedule_on + for x in dag.dataset_triggers ] else: return [] @@ -1111,7 +1111,7 @@ def deserialize_dag(cls, encoded_dag: Dict[str, Any]) -> 'SerializedDAG': v = cls._deserialize(v) elif k == "params": v = cls._deserialize_params_dict(v) - elif k == "schedule_on": + elif k == "upstream_datasets": v = cls._deserialize(v) # else use v as it is diff --git a/airflow/timetables/simple.py b/airflow/timetables/simple.py index db576b25a555d..1ab00403a3f45 100644 --- a/airflow/timetables/simple.py +++ b/airflow/timetables/simple.py @@ -103,8 +103,7 @@ def next_dagrun_info( class DatasetTriggeredTimetable(NullTimetable): """Timetable that never schedules anything. - This should not be directly used anywhere, but only set if a DAG specifies - ``schedule_on`` and is triggered by a dataset. + This should not be directly used anywhere, but only set if a DAG is triggered by datasets. :meta private: """ diff --git a/docs/apache-airflow/concepts/datasets.rst b/docs/apache-airflow/concepts/datasets.rst index 74735e36ee900..de24594991acc 100644 --- a/docs/apache-airflow/concepts/datasets.rst +++ b/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. diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index 969fd5748dfc3..6735dd1fdeb65 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -1499,7 +1499,7 @@ def test__get_upstream_dataset_events_no_prior(configured_app): session = settings.Session() dataset1a = Dataset(uri=f"s3://{unique_id}-1a") dataset1b = Dataset(uri=f"s3://{unique_id}-1b") - dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule_on=[dataset1a, dataset1b]) + dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule=[dataset1a, dataset1b]) DAG.bulk_write_to_db(dags=[dag2], session=session) session.add_all([dataset1a, dataset1b]) session.commit() @@ -1534,7 +1534,7 @@ def test__get_upstream_dataset_events_with_prior(configured_app): session = settings.Session() dataset1a = Dataset(uri=f"s3://{unique_id}-1a") dataset1b = Dataset(uri=f"s3://{unique_id}-1b") - dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule_on=[dataset1a, dataset1b]) + dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule=[dataset1a, dataset1b]) DAG.bulk_write_to_db(dags=[dag2], session=session) session.add_all([dataset1a, dataset1b]) session.commit() diff --git a/tests/dags/test_datasets.py b/tests/dags/test_datasets.py index f228270b912df..46d4fb8585c41 100644 --- a/tests/dags/test_datasets.py +++ b/tests/dags/test_datasets.py @@ -49,7 +49,7 @@ def raise_skip_exc(): dag_id='dag_that_follows_dag_with_skip', catchup=False, start_date=datetime(2020, 1, 1), - schedule_on=[skip_task_dag_dataset], + schedule=[skip_task_dag_dataset], tags=['downstream-skipped'], ) as dag_that_follows_dag_with_skip: BashOperator( @@ -80,7 +80,7 @@ def raise_fail_exc(): dag_id='dag_that_follows_dag_with_fail', catchup=False, start_date=datetime(2020, 1, 1), - schedule_on=[fail_task_dag_dataset], + schedule=[fail_task_dag_dataset], tags=['downstream-failed'], ) as dag_that_follows_dag_with_fail: BashOperator( diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 865e31c670633..1a2aa72c8b58b 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -824,7 +824,7 @@ def test_bulk_write_to_db_has_import_error(self): assert not model.has_import_errors session.close() - def test_bulk_write_to_db_datasets_schedule_on(self): + def test_bulk_write_to_db_datasets(self): """ Ensure that datasets referenced in a dag are correctly loaded into the database. """ @@ -836,7 +836,7 @@ def test_bulk_write_to_db_datasets_schedule_on(self): d1 = Dataset(uri1, extra={"not": "used"}) d2 = Dataset('s3://dataset2') d3 = Dataset('s3://dataset3') - dag1 = DAG(dag_id=dag_id1, start_date=DEFAULT_DATE, schedule_on=[d1]) + dag1 = DAG(dag_id=dag_id1, start_date=DEFAULT_DATE, schedule=[d1]) EmptyOperator(task_id=task_id, dag=dag1, outlets=[d2, d3]) dag2 = DAG(dag_id=dag_id2, start_date=DEFAULT_DATE) EmptyOperator(task_id=task_id, dag=dag2, outlets=[Dataset(uri1, extra={"should": "be used"})]) @@ -2703,9 +2703,9 @@ def test_get_dataset_triggered_next_run_info(session, reset_dataset): dataset1 = Dataset(uri=f"s3://{unique_id}-1") dataset2 = Dataset(uri=f"s3://{unique_id}-2") dataset3 = Dataset(uri=f"s3://{unique_id}-3") - dag1 = DAG(dag_id=f"datasets-{unique_id}-1", schedule_on=[dataset2]) - dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule_on=[dataset1, dataset2]) - dag3 = DAG(dag_id=f"datasets-{unique_id}-3", schedule_on=[dataset1, dataset2, dataset3]) + dag1 = DAG(dag_id=f"datasets-{unique_id}-1", schedule=[dataset2]) + dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule=[dataset1, dataset2]) + dag3 = DAG(dag_id=f"datasets-{unique_id}-3", schedule=[dataset1, dataset2, dataset3]) DAG.bulk_write_to_db(dags=[dag1, dag2, dag3], session=session) session.commit() diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 7f1bf8dfaac31..97bd1525fb259 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -1381,8 +1381,8 @@ def test_dataset_dagruns_triggered(session): dag1 = DAG(dag_id=f"datasets-{unique_id}-1", start_date=timezone.utcnow()) dataset1 = Dataset(uri=f"s3://{unique_id}-1") dataset2 = Dataset(uri=f"s3://{unique_id}-2") - dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule_on=[dataset1, dataset2]) - dag3 = DAG(dag_id=f"datasets-{unique_id}-3", schedule_on=[dataset1]) + dag2 = DAG(dag_id=f"datasets-{unique_id}-2", schedule=[dataset1, dataset2]) + dag3 = DAG(dag_id=f"datasets-{unique_id}-3", schedule=[dataset1]) task = BashOperator(task_id="task", bash_command="echo 1", dag=dag1, outlets=[dataset1]) # BashOperator(task_id="task", bash_command="echo 1", dag=dag2) # BashOperator(task_id="task", bash_command="echo 1", dag=dag3) From dba73a58f4408febe2dd37a22ea9c2fa610d36ef Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 12:29:27 -0700 Subject: [PATCH 02/38] add deprecation warning for old params --- airflow/models/dag.py | 31 ++++++++++++++++++++++++++----- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 2ae20e77d5d34..da701d7805229 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -470,13 +470,34 @@ def __init__( 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'.") - dataset_triggers = [] - if schedule and isinstance(schedule, List) and isinstance(schedule[0], Dataset): - dataset_triggers = 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, + ) + if timetable is not NOTSET: + warnings.warn( + "Param `timetable` is deprecated and will be removed in a future release. " + "Please use `schedule` instead. ", + DeprecationWarning, + ) self.timetable: Timetable self.schedule_interval: ScheduleInterval - self.dataset_triggers: Optional[List[Dataset]] = list(dataset_triggers) if dataset_triggers else None - if dataset_triggers: + self.dataset_triggers: Optional[List[Dataset]] = None + + if schedule and isinstance(schedule, Sequence): + # if Sequence, only support Sequence[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]." + ) + + if self.dataset_triggers: self.timetable = DatasetTriggeredTimetable() self.schedule_interval = self.timetable.summary elif timetable: From 1b38a3f6be650a78626fa609e5201bbedb78e2fa Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 12:44:53 -0700 Subject: [PATCH 03/38] harvest schedule interval args and timetables from `schedule` --- airflow/models/dag.py | 27 +++++++++++++++++---------- airflow/timetables/base.py | 3 ++- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index da701d7805229..2fd3bcf132583 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -486,16 +486,23 @@ def __init__( self.schedule_interval: ScheduleInterval self.dataset_triggers: Optional[List[Dataset]] = None - if schedule and isinstance(schedule, Sequence): - # if Sequence, only support Sequence[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]." - ) + if schedule: + if isinstance(schedule, Sequence): + # if Sequence, only support Sequence[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 if self.dataset_triggers: self.timetable = DatasetTriggeredTimetable() diff --git a/airflow/timetables/base.py b/airflow/timetables/base.py index 168f2af82d68d..c6705ee7a1c44 100644 --- a/airflow/timetables/base.py +++ b/airflow/timetables/base.py @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -from typing import Any, Dict, NamedTuple, Optional +from typing import Any, Dict, NamedTuple, Optional, runtime_checkable from pendulum import DateTime @@ -100,6 +100,7 @@ def logical_date(self: "DagRunInfo") -> DateTime: return self.data_interval.start +@runtime_checkable class Timetable(Protocol): """Protocol that all Timetable classes are expected to implement.""" From 269673330a20c7c51b37efcb8609e0c8da51fcad Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 12:56:35 -0700 Subject: [PATCH 04/38] fixup! harvest schedule interval args and timetables from `schedule` --- airflow/models/dag.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 2fd3bcf132583..e16e5224f9f64 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -115,7 +115,7 @@ # but Mypy cannot handle that right now. Track progress of PEP 661 for progress. # See also: https://discuss.python.org/t/9126/7 ScheduleIntervalArg = Union[ArgNotSet, ScheduleInterval] -ScheduleArg = Union[ArgNotSet, ScheduleInterval, Timetable, List["Dataset"]] +ScheduleArg = Union[ArgNotSet, ScheduleInterval, Timetable, Sequence["Dataset"]] SLAMissCallback = Callable[["DAG", str, str, List["SlaMiss"], List[TaskInstance]], None] From f24a55ace9ba530e98fa0e342846266893783ec3 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 13:20:42 -0700 Subject: [PATCH 05/38] fix runtime checkable import --- airflow/timetables/base.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/timetables/base.py b/airflow/timetables/base.py index c6705ee7a1c44..1eb144d48a5f7 100644 --- a/airflow/timetables/base.py +++ b/airflow/timetables/base.py @@ -15,11 +15,11 @@ # specific language governing permissions and limitations # under the License. -from typing import Any, Dict, NamedTuple, Optional, runtime_checkable +from typing import Any, Dict, NamedTuple, Optional from pendulum import DateTime -from airflow.typing_compat import Protocol +from airflow.typing_compat import Protocol, runtime_checkable class DataInterval(NamedTuple): From 4d0ef73d5f43cf042ad59ecc60573ff65491c551 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 13:21:53 -0700 Subject: [PATCH 06/38] rename most usages of schedule_interval= to schedule= --- airflow/example_dags/example_bash_operator.py | 2 +- .../example_branch_datetime_operator.py | 4 +- .../example_branch_day_of_week_operator.py | 2 +- airflow/example_dags/example_branch_labels.py | 2 +- .../example_dags/example_branch_operator.py | 2 +- .../example_branch_operator_decorator.py | 2 +- .../example_branch_python_dop_operator_3.py | 2 +- airflow/example_dags/example_complex.py | 2 +- airflow/example_dags/example_dag_decorator.py | 2 +- airflow/example_dags/example_datasets.py | 4 +- .../example_external_task_marker_dag.py | 4 +- .../example_kubernetes_executor.py | 2 +- airflow/example_dags/example_latest_only.py | 2 +- .../example_latest_only_with_trigger.py | 2 +- .../example_local_kubernetes_executor.py | 2 +- .../example_dags/example_nested_branch_dag.py | 2 +- ...example_passing_params_via_test_command.py | 2 +- .../example_dags/example_python_operator.py | 2 +- airflow/example_dags/example_sla_dag.py | 2 +- .../example_dags/example_subdag_operator.py | 2 +- .../example_time_delta_sensor_async.py | 2 +- .../example_trigger_controller_dag.py | 2 +- .../example_trigger_target_dag.py | 2 +- airflow/example_dags/example_xcom.py | 2 +- airflow/example_dags/example_xcomargs.py | 4 +- airflow/example_dags/subdags/subdag.py | 2 +- airflow/example_dags/tutorial.py | 2 +- airflow/example_dags/tutorial_etl_dag.py | 2 +- .../example_dags/tutorial_taskflow_api_etl.py | 2 +- .../tutorial_taskflow_api_etl_virtualenv.py | 2 +- airflow/models/dag.py | 4 +- .../aws/example_dags/example_appflow.py | 2 +- .../amazon/aws/example_dags/example_dms.py | 2 +- .../example_dags/example_dynamodb_to_s3.py | 2 +- .../amazon/aws/example_dags/example_ec2.py | 2 +- .../amazon/aws/example_dags/example_ecs.py | 2 +- .../aws/example_dags/example_eks_templated.py | 2 +- .../example_eks_with_fargate_in_one_step.py | 2 +- .../example_eks_with_fargate_profile.py | 2 +- .../example_eks_with_nodegroup_in_one_step.py | 2 +- .../example_eks_with_nodegroups.py | 2 +- .../amazon/aws/example_dags/example_emr.py | 2 +- .../aws/example_dags/example_emr_eks.py | 2 +- .../aws/example_dags/example_ftp_to_s3.py | 2 +- .../aws/example_dags/example_gcs_to_s3.py | 2 +- .../example_dags/example_glacier_to_gcs.py | 2 +- .../example_google_api_sheets_to_s3.py | 2 +- .../example_google_api_youtube_to_s3.py | 2 +- .../example_dags/example_hive_to_dynamodb.py | 2 +- .../example_imap_attachment_to_s3.py | 2 +- .../aws/example_dags/example_mongo_to_s3.py | 2 +- .../aws/example_dags/example_quicksight.py | 2 +- .../amazon/aws/example_dags/example_s3.py | 2 +- .../aws/example_dags/example_s3_to_ftp.py | 2 +- .../example_dags/example_s3_to_redshift.py | 2 +- .../aws/example_dags/example_s3_to_sftp.py | 2 +- .../example_dags/example_salesforce_to_s3.py | 2 +- .../aws/example_dags/example_sftp_to_s3.py | 2 +- .../aws/example_dags/example_sql_to_s3.py | 2 +- .../example_automl_nl_text_classification.py | 2 +- .../example_automl_nl_text_sentiment.py | 2 +- .../example_dags/example_automl_tables.py | 8 +-- .../example_automl_translation.py | 2 +- ...utoml_video_intelligence_classification.py | 2 +- ...mple_automl_video_intelligence_tracking.py | 2 +- .../example_automl_vision_object_detection.py | 2 +- .../example_dags/example_bigquery_dts.py | 2 +- .../cloud/example_dags/example_bigtable.py | 2 +- .../cloud/example_dags/example_cloud_build.py | 4 +- .../example_dags/example_cloud_composer.py | 4 +- .../example_dags/example_cloud_sql_query.py | 2 +- ...mple_cloud_storage_transfer_service_aws.py | 2 +- ...mple_cloud_storage_transfer_service_gcp.py | 2 +- .../cloud/example_dags/example_compute.py | 2 +- .../cloud/example_dags/example_compute_ssh.py | 2 +- .../cloud/example_dags/example_dataflow.py | 8 +-- .../example_dataflow_flex_template.py | 2 +- .../example_dags/example_dataflow_sql.py | 2 +- .../cloud/example_dags/example_datafusion.py | 2 +- .../cloud/example_dags/example_dataplex.py | 2 +- .../cloud/example_dags/example_dataprep.py | 2 +- .../example_dataproc_metastore.py | 2 +- .../google/cloud/example_dags/example_dlp.py | 8 +-- .../example_facebook_ads_to_gcs.py | 2 +- .../cloud/example_dags/example_functions.py | 2 +- .../example_dags/example_gdrive_to_gcs.py | 2 +- .../example_dags/example_gdrive_to_local.py | 2 +- .../cloud/example_dags/example_looker.py | 2 +- .../cloud/example_dags/example_mlengine.py | 2 +- .../example_dags/example_postgres_to_gcs.py | 2 +- .../example_dags/example_presto_to_gcs.py | 2 +- .../example_dags/example_salesforce_to_gcs.py | 2 +- .../cloud/example_dags/example_sftp_to_gcs.py | 2 +- .../cloud/example_dags/example_vertex_ai.py | 14 ++-- .../example_video_intelligence.py | 2 +- .../cloud/example_dags/example_vision.py | 6 +- .../google/cloud/transfers/gcs_to_sftp.py | 2 +- .../example_dags/example_display_video.py | 6 +- .../example_dags/example_local_to_drive.py | 2 +- airflow/timetables/base.py | 2 +- airflow/timetables/interval.py | 4 +- airflow/timetables/simple.py | 4 +- docs/apache-airflow/best-practices.rst | 8 +-- docs/apache-airflow/concepts/dags.rst | 18 ++--- docs/apache-airflow/concepts/operators.rst | 2 +- docs/apache-airflow/dag-run.rst | 4 +- docs/apache-airflow/executor/kubernetes.rst | 2 +- docs/apache-airflow/faq.rst | 4 +- docs/apache-airflow/howto/add-dag-tags.rst | 2 +- .../howto/dynamic-dag-generation.rst | 2 +- docs/apache-airflow/howto/timetable.rst | 2 +- docs/apache-airflow/index.rst | 2 +- docs/apache-airflow/lineage.rst | 2 +- .../logging-monitoring/callbacks.rst | 2 +- docs/apache-airflow/tutorial.rst | 2 +- .../extending/embedding-dags/test_dag.py | 2 +- .../endpoints/test_dag_endpoint.py | 8 +-- .../endpoints/test_dag_run_endpoint.py | 2 +- .../api_connexion/schemas/test_dag_schema.py | 2 +- tests/cli/commands/test_dag_command.py | 4 +- tests/core/test_core.py | 4 +- tests/core/test_sentry.py | 2 +- tests/dag_processing/test_manager.py | 2 +- tests/dags/subdir2/test_dont_ignore_this.py | 2 +- tests/dags/test_clear_subdag.py | 4 +- tests/dags/test_dag_with_no_tags.py | 2 +- tests/dags/test_datasets.py | 4 +- tests/dags/test_default_views.py | 4 +- tests/dags/test_example_bash_operator.py | 2 +- tests/dags/test_invalid_cron.py | 2 +- tests/dags/test_invalid_param.py | 2 +- tests/dags/test_logging_in_dag.py | 2 +- tests/dags/test_miscellaneous.py | 2 +- tests/dags/test_missing_owner.py | 2 +- tests/dags/test_multiple_dags.py | 2 +- tests/dags/test_only_empty_tasks.py | 2 +- tests/dags/test_retry_handling_job.py | 2 +- tests/dags/test_sensor.py | 2 +- tests/dags/test_subdag.py | 4 +- tests/dags/test_with_non_default_owner.py | 2 +- tests/dags_corrupted/test_nonstring_owner.py | 2 +- tests/jobs/test_backfill_job.py | 18 ++--- tests/jobs/test_scheduler_job.py | 44 ++++++------- tests/models/test_dag.py | 66 +++++++++---------- tests/models/test_dagbag.py | 2 +- tests/models/test_dagrun.py | 2 +- tests/models/test_taskinstance.py | 18 ++--- tests/operators/test_bash.py | 2 +- tests/operators/test_branch_operator.py | 2 +- tests/operators/test_datetime.py | 2 +- tests/operators/test_email.py | 2 +- tests/operators/test_latest_only_operator.py | 2 +- tests/operators/test_python.py | 8 +-- tests/operators/test_trigger_dagrun.py | 2 +- tests/operators/test_weekday.py | 2 +- .../amazon/aws/operators/test_appflow.py | 2 +- .../amazon/aws/operators/test_athena.py | 2 +- .../amazon/aws/operators/test_datasync.py | 2 +- .../aws/operators/test_dms_describe_tasks.py | 2 +- .../aws/sensors/test_s3_keys_unchanged.py | 4 +- .../amazon/aws/transfers/test_s3_to_sftp.py | 2 +- .../amazon/aws/transfers/test_sftp_to_s3.py | 2 +- .../common/sql/operators/test_sql.py | 2 +- .../google/cloud/operators/test_mlengine.py | 2 +- .../cloud/operators/test_mlengine_utils.py | 4 +- .../google/cloud/sensors/test_gcs.py | 6 +- .../operators/test_campaign_manager.py | 2 +- .../operators/test_display_video.py | 2 +- .../operators/test_search_ads.py | 2 +- .../microsoft/azure/operators/test_adx.py | 2 +- .../operators/test_yandexcloud_dataproc.py | 2 +- tests/sensors/test_external_task_sensor.py | 32 ++++----- .../airbyte/example_airbyte_trigger_job.py | 2 +- .../providers/amazon/aws/example_athena.py | 2 +- .../providers/amazon/aws/example_batch.py | 2 +- .../amazon/aws/example_cloudformation.py | 2 +- .../providers/amazon/aws/example_datasync.py | 2 +- .../amazon/aws/example_ecs_fargate.py | 2 +- .../providers/amazon/aws/example_glue.py | 2 +- .../providers/amazon/aws/example_lambda.py | 2 +- .../providers/amazon/aws/example_rds_event.py | 2 +- .../amazon/aws/example_rds_export.py | 2 +- .../amazon/aws/example_rds_snapshot.py | 2 +- .../amazon/aws/example_redshift_cluster.py | 2 +- .../aws/example_redshift_data_execute_sql.py | 2 +- .../amazon/aws/example_redshift_sql.py | 2 +- .../amazon/aws/example_redshift_to_s3.py | 2 +- .../providers/amazon/aws/example_sagemaker.py | 2 +- .../amazon/aws/example_sagemaker_endpoint.py | 2 +- .../providers/amazon/aws/example_sns.py | 2 +- .../providers/amazon/aws/example_sqs.py | 2 +- .../amazon/aws/example_step_functions.py | 2 +- .../amazon/aws/rds/example_rds_instance.py | 2 +- .../providers/apache/beam/example_beam.py | 2 +- .../apache/beam/example_beam_java_flink.py | 2 +- .../apache/beam/example_beam_java_spark.py | 2 +- .../providers/apache/beam/example_go.py | 2 +- .../apache/beam/example_go_dataflow.py | 2 +- .../apache/beam/example_java_dataflow.py | 2 +- .../providers/apache/beam/example_python.py | 2 +- .../apache/beam/example_python_dataflow.py | 2 +- .../apache/cassandra/example_cassandra_dag.py | 2 +- .../apache/drill/example_drill_dag.py | 2 +- .../apache/druid/example_druid_dag.py | 2 +- .../apache/hive/example_twitter_dag.py | 2 +- .../apache/kylin/example_kylin_dag.py | 2 +- .../providers/apache/livy/example_livy.py | 2 +- .../providers/apache/pig/example_pig.py | 2 +- .../apache/spark/example_spark_dag.py | 2 +- .../cncf/kubernetes/example_kubernetes.py | 2 +- .../kubernetes/example_spark_kubernetes.py | 2 +- .../sql/example_sql_column_table_check.py | 2 +- .../databricks/example_databricks.py | 2 +- .../databricks/example_databricks_repos.py | 2 +- .../databricks/example_databricks_sql.py | 2 +- .../providers/dbt/cloud/example_dbt_cloud.py | 2 +- .../providers/dingding/example_dingding.py | 2 +- .../system/providers/docker/example_docker.py | 2 +- .../docker/example_docker_copy_data.py | 2 +- .../providers/docker/example_docker_swarm.py | 2 +- ...mple_taskflow_api_etl_docker_virtualenv.py | 2 +- .../example_elasticsearch_query.py | 2 +- .../providers/google/ads/example_ads.py | 2 +- .../example_automl_nl_text_extraction.py | 2 +- .../example_automl_vision_classification.py | 2 +- .../azure/example_azure_fileshare_to_gcs.py | 2 +- .../bigquery/example_bigquery_dataset.py | 2 +- .../bigquery/example_bigquery_operations.py | 2 +- .../example_bigquery_operations_location.py | 2 +- .../bigquery/example_bigquery_queries.py | 2 +- .../bigquery/example_bigquery_sensors.py | 2 +- .../cloud/bigquery/example_bigquery_tables.py | 2 +- .../bigquery/example_bigquery_to_bigquery.py | 2 +- .../cloud/bigquery/example_bigquery_to_gcs.py | 2 +- .../bigquery/example_bigquery_to_mssql.py | 2 +- .../bigquery/example_bigquery_transfer.py | 2 +- .../example_cloud_memorystore_memcached.py | 2 +- .../example_cloud_memorystore_redis.py | 2 +- .../cloud/cloud_sql/example_cloud_sql.py | 2 +- .../cloud/compute_igm/example_compute_igm.py | 2 +- .../cloud/dataproc/example_dataproc_batch.py | 2 +- .../example_dataproc_batch_persistent.py | 2 +- .../example_dataproc_cluster_generator.py | 2 +- .../cloud/dataproc/example_dataproc_gke.py | 2 +- .../cloud/dataproc/example_dataproc_hadoop.py | 2 +- .../cloud/dataproc/example_dataproc_hive.py | 2 +- .../cloud/dataproc/example_dataproc_pig.py | 2 +- .../dataproc/example_dataproc_pyspark.py | 2 +- .../cloud/dataproc/example_dataproc_spark.py | 2 +- .../dataproc/example_dataproc_spark_async.py | 2 +- .../dataproc/example_dataproc_spark_sql.py | 2 +- .../cloud/dataproc/example_dataproc_sparkr.py | 2 +- .../cloud/dataproc/example_dataproc_update.py | 2 +- .../dataproc/example_dataproc_workflow.py | 2 +- .../datastore/example_datastore_commit.py | 2 +- .../example_datastore_export_import.py | 2 +- .../datastore/example_datastore_query.py | 2 +- .../datastore/example_datastore_rollback.py | 2 +- .../cloud/gcs/example_calendar_to_gcs.py | 2 +- .../google/cloud/gcs/example_firestore.py | 2 +- .../google/cloud/gcs/example_gcs_acl.py | 2 +- .../cloud/gcs/example_gcs_copy_delete.py | 2 +- .../google/cloud/gcs/example_gcs_sensor.py | 2 +- .../cloud/gcs/example_gcs_to_bigquery.py | 2 +- .../google/cloud/gcs/example_gcs_to_gcs.py | 2 +- .../google/cloud/gcs/example_gcs_to_gdrive.py | 2 +- .../google/cloud/gcs/example_gcs_to_sheets.py | 2 +- .../google/cloud/gcs/example_gcs_transform.py | 2 +- .../gcs/example_gcs_transform_timespan.py | 2 +- .../cloud/gcs/example_gcs_upload_download.py | 2 +- .../google/cloud/gcs/example_mssql_to_gcs.py | 2 +- .../google/cloud/gcs/example_mysql_to_gcs.py | 2 +- .../google/cloud/gcs/example_oracle_to_gcs.py | 2 +- .../google/cloud/gcs/example_s3_to_gcs.py | 2 +- .../google/cloud/gcs/example_sheets.py | 2 +- .../google/cloud/gcs/example_sheets_to_gcs.py | 2 +- .../google/cloud/gcs/example_trino_to_gcs.py | 2 +- .../example_kubernetes_engine.py | 2 +- .../life_sciences/example_life_sciences.py | 2 +- .../example_natural_language.py | 2 +- .../google/cloud/pubsub/example_pubsub.py | 2 +- .../google/cloud/spanner/example_spanner.py | 2 +- .../speech_to_text/example_speech_to_text.py | 2 +- .../sql_to_sheets/example_sql_to_sheets.py | 2 +- .../cloud/stackdriver/example_stackdriver.py | 2 +- .../google/cloud/tasks/example_queue.py | 2 +- .../google/cloud/tasks/example_tasks.py | 2 +- .../text_to_speech/example_text_to_speech.py | 2 +- .../cloud/transfers/example_gcs_to_sftp.py | 2 +- .../cloud/translate/example_translate.py | 2 +- .../example_translate_speech.py | 2 +- .../cloud/workflows/example_workflows.py | 2 +- .../example_datacatalog_entries.py | 2 +- .../example_datacatalog_search_catalog.py | 2 +- .../example_datacatalog_tag_templates.py | 2 +- .../datacatalog/example_datacatalog_tags.py | 2 +- .../google/leveldb/example_leveldb.py | 2 +- .../marketing_platform/example_analytics.py | 2 +- .../example_campaign_manager.py | 2 +- .../marketing_platform/example_search_ads.py | 2 +- .../providers/influxdb/example_influxdb.py | 2 +- .../providers/jdbc/example_jdbc_queries.py | 2 +- .../jenkins/example_jenkins_job_trigger.py | 2 +- .../azure/example_adf_run_pipeline.py | 2 +- .../microsoft/azure/example_adls_delete.py | 2 +- .../azure/example_azure_blob_to_gcs.py | 2 +- .../example_azure_container_instances.py | 2 +- .../azure/example_azure_service_bus.py | 2 +- .../microsoft/azure/example_fileshare.py | 2 +- .../microsoft/azure/example_local_to_adls.py | 2 +- .../microsoft/azure/example_local_to_wasb.py | 2 +- .../microsoft/azure/example_sftp_to_wasb.py | 2 +- .../microsoft/mssql/example_mssql.py | 2 +- .../microsoft/winrm/example_winrm.py | 2 +- .../opsgenie/example_opsgenie_alert.py | 2 +- .../providers/papermill/example_papermill.py | 2 +- .../papermill/example_papermill_verify.py | 2 +- .../system/providers/plexus/example_plexus.py | 2 +- .../providers/postgres/example_postgres.py | 2 +- .../providers/presto/example_gcs_to_presto.py | 2 +- .../system/providers/qubole/example_qubole.py | 2 +- .../qubole/example_qubole_sensors.py | 2 +- .../providers/salesforce/example_bulk.py | 2 +- .../example_salesforce_apex_rest.py | 2 +- .../singularity/example_singularity.py | 2 +- tests/system/providers/slack/example_slack.py | 2 +- .../providers/slack/example_sql_to_slack.py | 2 +- .../system/providers/sqlite/example_sqlite.py | 2 +- .../providers/tableau/example_tableau.py | 2 +- .../example_tableau_refresh_workbook.py | 2 +- .../providers/tabular/example_tabular.py | 2 +- .../providers/trino/example_gcs_to_trino.py | 2 +- tests/system/providers/trino/example_trino.py | 2 +- .../providers/yandex/example_yandexcloud.py | 2 +- .../yandex/example_yandexcloud_dataproc.py | 2 +- ...xample_yandexcloud_dataproc_lightweight.py | 2 +- .../zendesk/example_zendesk_custom_get.py | 2 +- tests/test_utils/perf/dags/elastic_dag.py | 4 +- tests/test_utils/perf/dags/perf_dag_1.py | 2 +- tests/test_utils/perf/dags/perf_dag_2.py | 2 +- tests/test_utils/perf/dags/sql_perf_dag.py | 2 +- .../deps/test_not_previously_skipped_dep.py | 10 +-- .../deps/test_runnable_exec_date_dep.py | 4 +- tests/utils/log/test_log_reader.py | 2 +- tests/utils/test_task_group.py | 2 +- tests/www/test_security.py | 4 +- 346 files changed, 491 insertions(+), 497 deletions(-) diff --git a/airflow/example_dags/example_bash_operator.py b/airflow/example_dags/example_bash_operator.py index 335f3ad961d3b..799c10101dc40 100644 --- a/airflow/example_dags/example_bash_operator.py +++ b/airflow/example_dags/example_bash_operator.py @@ -28,7 +28,7 @@ with DAG( dag_id='example_bash_operator', - schedule_interval='0 0 * * *', + schedule='0 0 * * *', start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, dagrun_timeout=datetime.timedelta(minutes=60), diff --git a/airflow/example_dags/example_branch_datetime_operator.py b/airflow/example_dags/example_branch_datetime_operator.py index e707514c868a0..1f0e603da3288 100644 --- a/airflow/example_dags/example_branch_datetime_operator.py +++ b/airflow/example_dags/example_branch_datetime_operator.py @@ -31,7 +31,7 @@ start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], - schedule_interval="@daily", + schedule="@daily", ) # [START howto_branch_datetime_operator] @@ -57,7 +57,7 @@ start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], - schedule_interval="@daily", + schedule="@daily", ) # [START howto_branch_datetime_operator_next_day] empty_task_12 = EmptyOperator(task_id='date_in_range', dag=dag2) diff --git a/airflow/example_dags/example_branch_day_of_week_operator.py b/airflow/example_dags/example_branch_day_of_week_operator.py index 879824ab1c876..62b0bc6ce9a11 100644 --- a/airflow/example_dags/example_branch_day_of_week_operator.py +++ b/airflow/example_dags/example_branch_day_of_week_operator.py @@ -30,7 +30,7 @@ start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], - schedule_interval="@daily", + schedule="@daily", ) as dag: # [START howto_operator_day_of_week_branch] empty_task_1 = EmptyOperator(task_id='branch_true') diff --git a/airflow/example_dags/example_branch_labels.py b/airflow/example_dags/example_branch_labels.py index 72337eb45c873..1645abd860123 100644 --- a/airflow/example_dags/example_branch_labels.py +++ b/airflow/example_dags/example_branch_labels.py @@ -27,7 +27,7 @@ with DAG( "example_branch_labels", - schedule_interval="@daily", + schedule="@daily", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, ) as dag: diff --git a/airflow/example_dags/example_branch_operator.py b/airflow/example_dags/example_branch_operator.py index 8721c78bcbc8e..e16d06eb13da0 100644 --- a/airflow/example_dags/example_branch_operator.py +++ b/airflow/example_dags/example_branch_operator.py @@ -32,7 +32,7 @@ dag_id='example_branch_operator', start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval="@daily", + schedule="@daily", tags=['example', 'example2'], ) as dag: run_this_first = EmptyOperator( diff --git a/airflow/example_dags/example_branch_operator_decorator.py b/airflow/example_dags/example_branch_operator_decorator.py index 864029349cb96..eecd2c07a0101 100644 --- a/airflow/example_dags/example_branch_operator_decorator.py +++ b/airflow/example_dags/example_branch_operator_decorator.py @@ -34,7 +34,7 @@ dag_id='example_branch_python_operator_decorator', start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval="@daily", + schedule="@daily", tags=['example', 'example2'], ) as dag: run_this_first = EmptyOperator(task_id='run_this_first') diff --git a/airflow/example_dags/example_branch_python_dop_operator_3.py b/airflow/example_dags/example_branch_python_dop_operator_3.py index acb957f72eacb..e991e1634045c 100644 --- a/airflow/example_dags/example_branch_python_dop_operator_3.py +++ b/airflow/example_dags/example_branch_python_dop_operator_3.py @@ -47,7 +47,7 @@ def should_run(**kwargs): with DAG( dag_id='example_branch_dop_operator_v3', - schedule_interval='*/1 * * * *', + schedule='*/1 * * * *', start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, default_args={'depends_on_past': True}, diff --git a/airflow/example_dags/example_complex.py b/airflow/example_dags/example_complex.py index 22e1906c042dd..dffaddd8ab2b7 100644 --- a/airflow/example_dags/example_complex.py +++ b/airflow/example_dags/example_complex.py @@ -27,7 +27,7 @@ with models.DAG( dag_id="example_complex", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example', 'example2', 'example3'], diff --git a/airflow/example_dags/example_dag_decorator.py b/airflow/example_dags/example_dag_decorator.py index a47766f8a7a4d..b43b7f7193241 100644 --- a/airflow/example_dags/example_dag_decorator.py +++ b/airflow/example_dags/example_dag_decorator.py @@ -39,7 +39,7 @@ def execute(self, context: Context): # [START dag_decorator_usage] @dag( - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example'], diff --git a/airflow/example_dags/example_datasets.py b/airflow/example_dags/example_datasets.py index 8e5967e77e0da..461038f5086c3 100644 --- a/airflow/example_dags/example_datasets.py +++ b/airflow/example_dags/example_datasets.py @@ -50,7 +50,7 @@ dag_id='example_dataset_dag1', catchup=False, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval='@daily', + schedule='@daily', tags=['upstream'], ) as dag1: # [START task_outlet] @@ -61,7 +61,7 @@ dag_id='example_dataset_dag2', catchup=False, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval=None, + schedule=None, tags=['upstream'], ) as dag2: BashOperator(outlets=[dag2_dataset], task_id='upstream_task_2', bash_command="sleep 5") diff --git a/airflow/example_dags/example_external_task_marker_dag.py b/airflow/example_dags/example_external_task_marker_dag.py index 733b732756633..88dae819de2b5 100644 --- a/airflow/example_dags/example_external_task_marker_dag.py +++ b/airflow/example_dags/example_external_task_marker_dag.py @@ -51,7 +51,7 @@ dag_id="example_external_task_marker_parent", start_date=start_date, catchup=False, - schedule_interval=None, + schedule=None, tags=['example2'], ) as parent_dag: # [START howto_operator_external_task_marker] @@ -65,7 +65,7 @@ with DAG( dag_id="example_external_task_marker_child", start_date=start_date, - schedule_interval=None, + schedule=None, catchup=False, tags=['example2'], ) as child_dag: diff --git a/airflow/example_dags/example_kubernetes_executor.py b/airflow/example_dags/example_kubernetes_executor.py index f278eb379b1e8..f14e54e284b3f 100644 --- a/airflow/example_dags/example_kubernetes_executor.py +++ b/airflow/example_dags/example_kubernetes_executor.py @@ -46,7 +46,7 @@ if k8s: with DAG( dag_id='example_kubernetes_executor', - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example3'], diff --git a/airflow/example_dags/example_latest_only.py b/airflow/example_dags/example_latest_only.py index 92ec1436a6951..cecb1dfc187ec 100644 --- a/airflow/example_dags/example_latest_only.py +++ b/airflow/example_dags/example_latest_only.py @@ -26,7 +26,7 @@ with DAG( dag_id='latest_only', - schedule_interval=dt.timedelta(hours=4), + schedule=dt.timedelta(hours=4), start_date=dt.datetime(2021, 1, 1), catchup=False, tags=['example2', 'example3'], diff --git a/airflow/example_dags/example_latest_only_with_trigger.py b/airflow/example_dags/example_latest_only_with_trigger.py index 56d6a24462217..eb452e1adbdbe 100644 --- a/airflow/example_dags/example_latest_only_with_trigger.py +++ b/airflow/example_dags/example_latest_only_with_trigger.py @@ -31,7 +31,7 @@ with DAG( dag_id='latest_only_with_trigger', - schedule_interval=datetime.timedelta(hours=4), + schedule=datetime.timedelta(hours=4), start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example3'], diff --git a/airflow/example_dags/example_local_kubernetes_executor.py b/airflow/example_dags/example_local_kubernetes_executor.py index e586cafda9416..3a3c6607c7caa 100644 --- a/airflow/example_dags/example_local_kubernetes_executor.py +++ b/airflow/example_dags/example_local_kubernetes_executor.py @@ -41,7 +41,7 @@ if k8s: with DAG( dag_id='example_local_kubernetes_executor', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example3'], diff --git a/airflow/example_dags/example_nested_branch_dag.py b/airflow/example_dags/example_nested_branch_dag.py index 1eb41d3fdabf3..27f758c67b81f 100644 --- a/airflow/example_dags/example_nested_branch_dag.py +++ b/airflow/example_dags/example_nested_branch_dag.py @@ -32,7 +32,7 @@ dag_id="example_nested_branch_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval="@daily", + schedule="@daily", tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_passing_params_via_test_command.py b/airflow/example_dags/example_passing_params_via_test_command.py index 8057d5fd54a13..f851d12349da1 100644 --- a/airflow/example_dags/example_passing_params_via_test_command.py +++ b/airflow/example_dags/example_passing_params_via_test_command.py @@ -59,7 +59,7 @@ def print_env_vars(test_mode=None): with DAG( "example_passing_params_via_test_command", - schedule_interval='*/1 * * * *', + schedule='*/1 * * * *', start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, dagrun_timeout=datetime.timedelta(minutes=4), diff --git a/airflow/example_dags/example_python_operator.py b/airflow/example_dags/example_python_operator.py index 0f9a7fc476acb..9b4bc52079faa 100644 --- a/airflow/example_dags/example_python_operator.py +++ b/airflow/example_dags/example_python_operator.py @@ -34,7 +34,7 @@ with DAG( dag_id='example_python_operator', - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example'], diff --git a/airflow/example_dags/example_sla_dag.py b/airflow/example_dags/example_sla_dag.py index 985b2549d517c..692b2df735020 100644 --- a/airflow/example_dags/example_sla_dag.py +++ b/airflow/example_dags/example_sla_dag.py @@ -40,7 +40,7 @@ def sla_callback(dag, task_list, blocking_task_list, slas, blocking_tis): @dag( - schedule_interval="*/2 * * * *", + schedule="*/2 * * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, sla_miss_callback=sla_callback, diff --git a/airflow/example_dags/example_subdag_operator.py b/airflow/example_dags/example_subdag_operator.py index 79d369d638d6a..ea45786dfab73 100644 --- a/airflow/example_dags/example_subdag_operator.py +++ b/airflow/example_dags/example_subdag_operator.py @@ -32,7 +32,7 @@ dag_id=DAG_NAME, default_args={"retries": 2}, start_date=datetime.datetime(2022, 1, 1), - schedule_interval="@once", + schedule="@once", tags=['example'], ) as dag: diff --git a/airflow/example_dags/example_time_delta_sensor_async.py b/airflow/example_dags/example_time_delta_sensor_async.py index a2aa3cb66edee..9742833cd6765 100644 --- a/airflow/example_dags/example_time_delta_sensor_async.py +++ b/airflow/example_dags/example_time_delta_sensor_async.py @@ -31,7 +31,7 @@ with DAG( dag_id="example_time_delta_sensor_async", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/example_trigger_controller_dag.py b/airflow/example_dags/example_trigger_controller_dag.py index a017c9a5b4176..af6e407a3ec1e 100644 --- a/airflow/example_dags/example_trigger_controller_dag.py +++ b/airflow/example_dags/example_trigger_controller_dag.py @@ -30,7 +30,7 @@ dag_id="example_trigger_controller_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval="@once", + schedule="@once", tags=['example'], ) as dag: trigger = TriggerDagRunOperator( diff --git a/airflow/example_dags/example_trigger_target_dag.py b/airflow/example_dags/example_trigger_target_dag.py index 20932338c8dd8..20f73cf4fc314 100644 --- a/airflow/example_dags/example_trigger_target_dag.py +++ b/airflow/example_dags/example_trigger_target_dag.py @@ -42,7 +42,7 @@ def run_this_func(dag_run=None): dag_id="example_trigger_target_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval=None, + schedule=None, tags=['example'], ) as dag: run_this = run_this_func() diff --git a/airflow/example_dags/example_xcom.py b/airflow/example_dags/example_xcom.py index b55d4e5d667cd..fa9cb685caa12 100644 --- a/airflow/example_dags/example_xcom.py +++ b/airflow/example_dags/example_xcom.py @@ -63,7 +63,7 @@ def pull_value_from_bash_push(ti=None): with DAG( 'example_xcom', - schedule_interval="@once", + schedule="@once", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example'], diff --git a/airflow/example_dags/example_xcomargs.py b/airflow/example_dags/example_xcomargs.py index 8312aca8c9d25..4101ca58ae0c1 100644 --- a/airflow/example_dags/example_xcomargs.py +++ b/airflow/example_dags/example_xcomargs.py @@ -44,7 +44,7 @@ def print_value(value, ts=None): dag_id='example_xcom_args', start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval=None, + schedule=None, tags=['example'], ) as dag: print_value(generate_value()) @@ -53,7 +53,7 @@ def print_value(value, ts=None): "example_xcom_args_with_operators", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval=None, + schedule=None, tags=['example'], ) as dag2: bash_op1 = BashOperator(task_id="c", bash_command="echo c") diff --git a/airflow/example_dags/subdags/subdag.py b/airflow/example_dags/subdags/subdag.py index 2fcab731092fb..0686507942914 100644 --- a/airflow/example_dags/subdags/subdag.py +++ b/airflow/example_dags/subdags/subdag.py @@ -40,7 +40,7 @@ def subdag(parent_dag_name, child_dag_name, args): default_args=args, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, - schedule_interval="@daily", + schedule="@daily", ) for i in range(5): diff --git a/airflow/example_dags/tutorial.py b/airflow/example_dags/tutorial.py index b69f350c88fd8..11fc3ce1e00ae 100644 --- a/airflow/example_dags/tutorial.py +++ b/airflow/example_dags/tutorial.py @@ -63,7 +63,7 @@ }, # [END default_args] description='A simple tutorial DAG', - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/example_dags/tutorial_etl_dag.py b/airflow/example_dags/tutorial_etl_dag.py index d039a73488c18..6b0271ee27723 100644 --- a/airflow/example_dags/tutorial_etl_dag.py +++ b/airflow/example_dags/tutorial_etl_dag.py @@ -45,7 +45,7 @@ default_args={'retries': 2}, # [END default_args] description='ETL DAG tutorial', - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example'], diff --git a/airflow/example_dags/tutorial_taskflow_api_etl.py b/airflow/example_dags/tutorial_taskflow_api_etl.py index f6af78f0a5a2c..d6068dd1d6052 100644 --- a/airflow/example_dags/tutorial_taskflow_api_etl.py +++ b/airflow/example_dags/tutorial_taskflow_api_etl.py @@ -30,7 +30,7 @@ # [START instantiate_dag] @dag( - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example'], diff --git a/airflow/example_dags/tutorial_taskflow_api_etl_virtualenv.py b/airflow/example_dags/tutorial_taskflow_api_etl_virtualenv.py index ac280956b7f45..e8e4610e19c36 100644 --- a/airflow/example_dags/tutorial_taskflow_api_etl_virtualenv.py +++ b/airflow/example_dags/tutorial_taskflow_api_etl_virtualenv.py @@ -31,7 +31,7 @@ ) else: - @dag(schedule_interval=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example']) + @dag(schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example']) def tutorial_taskflow_api_etl_virtualenv(): """ ### TaskFlow API example using virtualenv diff --git a/airflow/models/dag.py b/airflow/models/dag.py index e16e5224f9f64..46cd12df8cddd 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -611,7 +611,7 @@ def _check_schedule_interval_matches_timetable(self) -> bool: dag1 = DAG("d1", timetable=MyTimetable()) dag1.schedule_interval = "@once" - dag2 = DAG("d2", schedule_interval="@once") + dag2 = DAG("d2", schedule="@once") dag2.timetable = MyTimetable() Validation is done by creating a timetable and check its summary matches @@ -3355,7 +3355,7 @@ class DagContext: with DAG( dag_id="example_dag", default_args=default_args, - schedule_interval="0 0 * * *", + schedule="0 0 * * *", dagrun_timeout=timedelta(minutes=60), ) as dag: ... diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index 06903c73d9a2e..ff4b7924e9712 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -34,7 +34,7 @@ with DAG( "example_appflow", - schedule_interval=None, + schedule=None, start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/amazon/aws/example_dags/example_dms.py b/airflow/providers/amazon/aws/example_dags/example_dms.py index 46e97d92a50d1..27444a5917157 100644 --- a/airflow/providers/amazon/aws/example_dags/example_dms.py +++ b/airflow/providers/amazon/aws/example_dags/example_dms.py @@ -227,7 +227,7 @@ def delete_dms_assets(): with DAG( dag_id='example_dms', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py index 66334fc996522..0630193495bc0 100644 --- a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py @@ -27,7 +27,7 @@ with DAG( dag_id='example_dynamodb_to_s3', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ec2.py b/airflow/providers/amazon/aws/example_dags/example_ec2.py index 7d12aef36746e..50d8a97daeab8 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ec2.py +++ b/airflow/providers/amazon/aws/example_dags/example_ec2.py @@ -26,7 +26,7 @@ with DAG( dag_id='example_ec2', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ecs.py b/airflow/providers/amazon/aws/example_dags/example_ecs.py index 45154a617d9a2..5cd9fb541aee7 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ecs.py +++ b/airflow/providers/amazon/aws/example_dags/example_ecs.py @@ -53,7 +53,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_templated.py b/airflow/providers/amazon/aws/example_dags/example_eks_templated.py index 87d6d9ef44e68..bef8b220f043f 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_templated.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_templated.py @@ -49,7 +49,7 @@ with DAG( dag_id='example_eks_templated', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example', 'templated'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py index e08e6525e6fc8..35c5eecf35b0d 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py @@ -44,7 +44,7 @@ with DAG( dag_id='example_eks_with_fargate_in_one_step', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py index 3ca3b2eb87728..6715698ac6a05 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py @@ -47,7 +47,7 @@ with DAG( dag_id='example_eks_with_fargate_profile', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py index 38d1bd1ad4c2f..f6c4cbaaf9cb8 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py @@ -43,7 +43,7 @@ with DAG( dag_id='example_eks_with_nodegroup_in_one_step', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py index efeeb14e04bfb..670d136ddfbd4 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py @@ -46,7 +46,7 @@ with DAG( dag_id='example_eks_with_nodegroups', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_emr.py b/airflow/providers/amazon/aws/example_dags/example_emr.py index 786b4a0ad4f71..51fa6e0d0ef23 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr.py @@ -68,7 +68,7 @@ with DAG( dag_id='example_emr', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_eks.py b/airflow/providers/amazon/aws/example_dags/example_emr_eks.py index 467db6aadba7c..02bdd4dcce48e 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr_eks.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr_eks.py @@ -54,7 +54,7 @@ with DAG( dag_id='example_emr_eks', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py index d01ca38810729..97372de06b527 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py @@ -27,7 +27,7 @@ with models.DAG( "example_ftp_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py index d9d04c73ffa31..37869b75ca9bf 100644 --- a/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py @@ -26,7 +26,7 @@ with DAG( dag_id="example_gcs_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=["example"], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py index 2df40a7d0c37b..4a8a5ab288ff0 100644 --- a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py +++ b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py @@ -28,7 +28,7 @@ with DAG( "example_glacier_to_gcs", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), # Override to match your needs catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py index 7b6e4b291a66c..32a85b4649114 100644 --- a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py @@ -31,7 +31,7 @@ with DAG( dag_id="example_google_api_sheets_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py index 8909adbf41598..5865857a8919f 100644 --- a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py @@ -66,7 +66,7 @@ def transform_video_ids(**kwargs): with DAG( dag_id="example_google_api_youtube_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py b/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py index 6fccd64c6d596..e21482d3e6935 100644 --- a/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py +++ b/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py @@ -105,7 +105,7 @@ def configure_hive_connection(): with DAG( dag_id='example_hive_to_dynamodb', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py index 357d92a6f5694..917e4d0960ccb 100644 --- a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py @@ -35,7 +35,7 @@ with DAG( dag_id="example_imap_attachment_to_s3", start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, tags=['example'], ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py index e95964b59cee1..788ad68cf3c0f 100644 --- a/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py @@ -29,7 +29,7 @@ with models.DAG( "example_mongo_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_quicksight.py b/airflow/providers/amazon/aws/example_dags/example_quicksight.py index 5c50a54492617..ee8d2589d9e79 100644 --- a/airflow/providers/amazon/aws/example_dags/example_quicksight.py +++ b/airflow/providers/amazon/aws/example_dags/example_quicksight.py @@ -28,7 +28,7 @@ with DAG( dag_id="example_quicksight", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=["example"], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_s3.py b/airflow/providers/amazon/aws/example_dags/example_s3.py index 4be800a9933ea..6144af57a7a25 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3.py @@ -57,7 +57,7 @@ with DAG( dag_id='example_s3', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py index 6ebe2501c613b..e2e79782991de 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py @@ -27,7 +27,7 @@ with models.DAG( "example_s3_to_ftp", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py index 82ae0660053f1..483b75cb1e987 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py @@ -46,7 +46,7 @@ def task_remove_sample_data_from_s3(): with DAG( dag_id="example_s3_to_redshift", start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, tags=['example'], ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py index d7983265e7697..812815ed52f33 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py @@ -26,7 +26,7 @@ with models.DAG( "example_s3_to_sftp", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py index 735cda4d3af14..c5677ea34ccae 100644 --- a/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py @@ -32,7 +32,7 @@ with DAG( dag_id="example_salesforce_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 7, 8), catchup=False, tags=["example"], diff --git a/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py index 0e2407a7d3546..6b72886dce037 100644 --- a/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py @@ -27,7 +27,7 @@ with models.DAG( "example_sftp_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py index df2abee0f3052..2f7aa7b057a5d 100644 --- a/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py @@ -28,7 +28,7 @@ with models.DAG( "example_sql_to_s3", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py index 28935e7b286f5..e3350d069253f 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py @@ -59,7 +59,7 @@ # Example DAG for AutoML Natural Language Text Classification with models.DAG( "example_automl_text_cls", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py index 47a4a7695a98a..14b8c9f8fd8d2 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py @@ -59,7 +59,7 @@ # Example DAG for AutoML Natural Language Text Sentiment with models.DAG( "example_automl_text_sentiment", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_tables.py b/airflow/providers/google/cloud/example_dags/example_automl_tables.py index c13de99fa8512..cb06224d53b54 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_tables.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_tables.py @@ -86,7 +86,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: # Example DAG to create dataset, train model_id and deploy it. with models.DAG( "example_create_and_deploy", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=START_DATE, catchup=False, user_defined_macros={ @@ -199,7 +199,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: # Example DAG for AutoML datasets operations with models.DAG( "example_automl_dataset", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=START_DATE, catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, @@ -268,7 +268,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: with models.DAG( "example_gcp_get_deploy", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=START_DATE, catchup=False, tags=["example"], @@ -294,7 +294,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: with models.DAG( "example_gcp_predict", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=START_DATE, catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_automl_translation.py b/airflow/providers/google/cloud/example_dags/example_automl_translation.py index ae90458b5869a..6ed845928134a 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_translation.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_translation.py @@ -65,7 +65,7 @@ # Example DAG for AutoML Translation with models.DAG( "example_automl_translation", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py index 2ecaebf871f95..c9bda04122d5e 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py @@ -62,7 +62,7 @@ # Example DAG for AutoML Video Intelligence Classification with models.DAG( "example_automl_video", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py index f8ba82da98823..ccd4e8df1b906 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py @@ -63,7 +63,7 @@ # Example DAG for AutoML Video Intelligence Object Tracking with models.DAG( "example_automl_video_tracking", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py index 1d897f2d6a9e4..bc19d91cf7aaa 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py @@ -62,7 +62,7 @@ # Example DAG for AutoML Vision Object Detection with models.DAG( "example_automl_vision_detection", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py index ac584eb197496..2e5d0be0170d1 100644 --- a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py +++ b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py @@ -64,7 +64,7 @@ with models.DAG( "example_gcp_bigquery_dts", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_bigtable.py b/airflow/providers/google/cloud/example_dags/example_bigtable.py index 2bfc145f877ba..70044541e2663 100644 --- a/airflow/providers/google/cloud/example_dags/example_bigtable.py +++ b/airflow/providers/google/cloud/example_dags/example_bigtable.py @@ -80,7 +80,7 @@ with models.DAG( 'example_gcp_bigtable_operators', - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_build.py b/airflow/providers/google/cloud/example_dags/example_cloud_build.py index f78e1eb47891a..06777ff619a3d 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_build.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_build.py @@ -116,7 +116,7 @@ with models.DAG( "example_gcp_cloud_build", - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, tags=["example"], @@ -205,7 +205,7 @@ with models.DAG( "example_gcp_cloud_build_trigger", - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_composer.py b/airflow/providers/google/cloud/example_dags/example_cloud_composer.py index eda597023e3fb..6b8bd6b08e2bb 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_composer.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_composer.py @@ -55,7 +55,7 @@ with models.DAG( "composer_dag1", - schedule_interval="@once", # Override to match your needs + schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], @@ -118,7 +118,7 @@ with models.DAG( "composer_dag_deferrable1", - schedule_interval="@once", # Override to match your needs + schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py index 307de77ff3ed2..a956a9c4a52cc 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py @@ -270,7 +270,7 @@ def get_absolute_path(path): with models.DAG( dag_id='example_gcp_sql_query', - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py index bf73959d4ff72..d6a88e4e63308 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py @@ -106,7 +106,7 @@ with models.DAG( 'example_gcp_transfer_aws', - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py index cdf38e0ca1fae..499c5098bd6d7 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py @@ -101,7 +101,7 @@ with models.DAG( "example_gcp_transfer", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_compute.py b/airflow/providers/google/cloud/example_dags/example_compute.py index 6d81e3a232a02..181062a81ce21 100644 --- a/airflow/providers/google/cloud/example_dags/example_compute.py +++ b/airflow/providers/google/cloud/example_dags/example_compute.py @@ -52,7 +52,7 @@ with models.DAG( 'example_gcp_compute', - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_compute_ssh.py b/airflow/providers/google/cloud/example_dags/example_compute_ssh.py index c20743725342c..d1353b05feab5 100644 --- a/airflow/providers/google/cloud/example_dags/example_compute_ssh.py +++ b/airflow/providers/google/cloud/example_dags/example_compute_ssh.py @@ -30,7 +30,7 @@ with models.DAG( 'example_compute_ssh', - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow.py b/airflow/providers/google/cloud/example_dags/example_dataflow.py index 8b1d01ff62735..e131ec66e5a07 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow.py @@ -64,7 +64,7 @@ with models.DAG( "example_gcp_dataflow_native_java", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=START_DATE, catchup=False, tags=['example'], @@ -115,7 +115,7 @@ default_args=default_args, start_date=START_DATE, catchup=False, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs tags=['example'], ) as dag_native_python: @@ -151,7 +151,7 @@ default_args=default_args, start_date=START_DATE, catchup=False, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs tags=['example'], ) as dag_native_python_async: # [START howto_operator_start_python_job_async] @@ -253,7 +253,7 @@ def check_autoscaling_event(autoscaling_events: List[dict]) -> bool: default_args=default_args, start_date=START_DATE, catchup=False, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs tags=['example'], ) as dag_template: # [START howto_operator_start_template_job] diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py b/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py index 43d9914850973..a5a6bd84c1c12 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py @@ -47,7 +47,7 @@ dag_id="example_gcp_dataflow_flex_template_java", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs ) as dag_flex_template: # [START howto_operator_start_template_job] start_flex_template = DataflowStartFlexTemplateOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py b/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py index a74f5dedc1254..538ad559a9c6f 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py @@ -38,7 +38,7 @@ dag_id="example_gcp_dataflow_sql", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs tags=['example'], ) as dag_sql: # [START howto_operator_start_sql_job] diff --git a/airflow/providers/google/cloud/example_dags/example_datafusion.py b/airflow/providers/google/cloud/example_dags/example_datafusion.py index e442164fd686a..5a43d2b4b571b 100644 --- a/airflow/providers/google/cloud/example_dags/example_datafusion.py +++ b/airflow/providers/google/cloud/example_dags/example_datafusion.py @@ -147,7 +147,7 @@ with models.DAG( "example_data_fusion", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_dataplex.py b/airflow/providers/google/cloud/example_dags/example_dataplex.py index aabe17aed69a6..dc47dbe3c59f0 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataplex.py +++ b/airflow/providers/google/cloud/example_dags/example_dataplex.py @@ -51,7 +51,7 @@ # [END howto_dataplex_configuration] with models.DAG( - "example_dataplex", start_date=datetime.datetime(2021, 1, 1), schedule_interval="@once", catchup=False + "example_dataplex", start_date=datetime.datetime(2021, 1, 1), schedule="@once", catchup=False ) as dag: # [START howto_dataplex_create_task_operator] create_dataplex_task = DataplexCreateTaskOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dataprep.py b/airflow/providers/google/cloud/example_dags/example_dataprep.py index 1bd460a370476..5f663824a668b 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataprep.py +++ b/airflow/providers/google/cloud/example_dags/example_dataprep.py @@ -52,7 +52,7 @@ with models.DAG( "example_dataprep", - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), # Override to match your needs catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py b/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py index 91b067f8c7304..0db1d33825f73 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py +++ b/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py @@ -94,7 +94,7 @@ with models.DAG( - "example_gcp_dataproc_metastore", start_date=datetime.datetime(2021, 1, 1), schedule_interval="@once" + "example_gcp_dataproc_metastore", start_date=datetime.datetime(2021, 1, 1), schedule="@once" ) as dag: # [START how_to_cloud_dataproc_metastore_create_service_operator] create_service = DataprocMetastoreCreateServiceOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dlp.py b/airflow/providers/google/cloud/example_dags/example_dlp.py index 480fda1f0d984..cbdbb89aa0a50 100644 --- a/airflow/providers/google/cloud/example_dags/example_dlp.py +++ b/airflow/providers/google/cloud/example_dags/example_dlp.py @@ -63,7 +63,7 @@ with models.DAG( "example_gcp_dlp", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=START_DATE, catchup=False, tags=['example'], @@ -113,7 +113,7 @@ with models.DAG( "example_gcp_dlp_info_types", - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, tags=["example", "dlp", "info-types"], @@ -157,7 +157,7 @@ with models.DAG( "example_gcp_dlp_job", - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, tags=["example", "dlp_job"], @@ -203,7 +203,7 @@ with models.DAG( "example_gcp_dlp_deidentify_content", - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, tags=["example", "dlp", "deidentify"], diff --git a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py index bd80091e48dc5..4028bdd5838f6 100644 --- a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py @@ -57,7 +57,7 @@ with models.DAG( "example_facebook_ads_to_gcs", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_functions.py b/airflow/providers/google/cloud/example_dags/example_functions.py index 5ce7f5e218206..d2999e78a0275 100644 --- a/airflow/providers/google/cloud/example_dags/example_functions.py +++ b/airflow/providers/google/cloud/example_dags/example_functions.py @@ -94,7 +94,7 @@ with models.DAG( 'example_gcp_function', default_args=default_args, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py index bb656340f1fba..1597796a40646 100644 --- a/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py @@ -32,7 +32,7 @@ "example_gdrive_to_gcs_with_gdrive_sensor", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs tags=["example"], ) as dag: # [START detect_file] diff --git a/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py b/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py index 2ba38ea81311a..92deb49139ebd 100644 --- a/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py +++ b/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py @@ -31,7 +31,7 @@ "example_gdrive_to_local_with_gdrive_sensor", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs tags=["example"], ) as dag: # [START detect_file] diff --git a/airflow/providers/google/cloud/example_dags/example_looker.py b/airflow/providers/google/cloud/example_dags/example_looker.py index ece60f3823398..28e350206ab1b 100644 --- a/airflow/providers/google/cloud/example_dags/example_looker.py +++ b/airflow/providers/google/cloud/example_dags/example_looker.py @@ -28,7 +28,7 @@ with models.DAG( dag_id='example_gcp_looker', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_mlengine.py b/airflow/providers/google/cloud/example_dags/example_mlengine.py index 7db060a08319a..7fc42b44fb816 100644 --- a/airflow/providers/google/cloud/example_dags/example_mlengine.py +++ b/airflow/providers/google/cloud/example_dags/example_mlengine.py @@ -59,7 +59,7 @@ with models.DAG( "example_gcp_mlengine", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py index 96ac71193998a..cc727b0b06794 100644 --- a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py @@ -31,7 +31,7 @@ with models.DAG( dag_id='example_postgres_to_gcs', - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py index 6ac82c5b97995..9a61fb988f98c 100644 --- a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py @@ -48,7 +48,7 @@ def safe_name(s: str) -> str: with models.DAG( dag_id="example_presto_to_gcs", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py index 4cf449f538cf0..eb16de6fbff14 100644 --- a/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py @@ -44,7 +44,7 @@ with models.DAG( "example_salesforce_to_gcs", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py index 46870f48ff5f8..afb908bf39f72 100644 --- a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py @@ -38,7 +38,7 @@ with models.DAG( "example_sftp_to_gcs", - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py index a421c31a7ee2c..84482061d41ba 100644 --- a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py +++ b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py @@ -238,7 +238,7 @@ with models.DAG( "example_gcp_vertex_ai_custom_jobs", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as custom_jobs_dag: @@ -328,7 +328,7 @@ with models.DAG( "example_gcp_vertex_ai_dataset", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as dataset_dag: @@ -431,7 +431,7 @@ with models.DAG( "example_gcp_vertex_ai_auto_ml", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as auto_ml_dag: @@ -548,7 +548,7 @@ with models.DAG( "example_gcp_vertex_ai_batch_prediction_job", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as batch_prediction_job_dag: @@ -588,7 +588,7 @@ with models.DAG( "example_gcp_vertex_ai_endpoint", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as endpoint_dag: @@ -644,7 +644,7 @@ with models.DAG( "example_gcp_vertex_ai_hyperparameter_tuning_job", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as hyperparameter_tuning_job_dag: @@ -717,7 +717,7 @@ with models.DAG( "example_gcp_vertex_ai_model_service", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as model_service_dag: diff --git a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py index 7280cf3e085c1..4692060a2a4b6 100644 --- a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py +++ b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py @@ -49,7 +49,7 @@ with models.DAG( "example_gcp_video_intelligence", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_vision.py b/airflow/providers/google/cloud/example_dags/example_vision.py index eb3703d3598ed..4da627a06b03d 100644 --- a/airflow/providers/google/cloud/example_dags/example_vision.py +++ b/airflow/providers/google/cloud/example_dags/example_vision.py @@ -118,7 +118,7 @@ with models.DAG( 'example_gcp_vision_autogenerated_id', - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, ) as dag_autogenerated_id: @@ -276,7 +276,7 @@ with models.DAG( 'example_gcp_vision_explicit_id', - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, ) as dag_explicit_id: @@ -444,7 +444,7 @@ with models.DAG( 'example_gcp_vision_annotate_image', - schedule_interval='@once', + schedule='@once', start_date=START_DATE, catchup=False, ) as dag_annotate_image: diff --git a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py index 1ad74cc3c1d01..964b365fc4002 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py @@ -40,7 +40,7 @@ class GCSToSFTPOperator(BaseOperator): with models.DAG( "example_gcs_to_sftp", start_date=datetime(2020, 6, 19), - schedule_interval=None, + schedule=None, ) as dag: # downloads file to /tmp/sftp/folder/subfolder/file.txt copy_file_from_gcs_to_sftp = GCSToSFTPOperator( diff --git a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py index d2e2d07b002af..915c79ff5aa89 100644 --- a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py +++ b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py @@ -86,7 +86,7 @@ with models.DAG( "example_display_video", - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=START_DATE, catchup=False, ) as dag1: @@ -129,7 +129,7 @@ with models.DAG( "example_display_video_misc", - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=START_DATE, catchup=False, ) as dag2: @@ -163,7 +163,7 @@ with models.DAG( "example_display_video_sdf", - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=START_DATE, catchup=False, ) as dag3: diff --git a/airflow/providers/google/suite/example_dags/example_local_to_drive.py b/airflow/providers/google/suite/example_dags/example_local_to_drive.py index 6e985b796a60b..7a34e45a6cba4 100644 --- a/airflow/providers/google/suite/example_dags/example_local_to_drive.py +++ b/airflow/providers/google/suite/example_dags/example_local_to_drive.py @@ -31,7 +31,7 @@ with models.DAG( "example_local_to_drive", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/timetables/base.py b/airflow/timetables/base.py index 1eb144d48a5f7..779534116bb78 100644 --- a/airflow/timetables/base.py +++ b/airflow/timetables/base.py @@ -115,7 +115,7 @@ class Timetable(Protocol): """Whether this timetable runs periodically. This defaults to and should generally be *True*, but some special setups - like ``schedule_interval=None`` and ``"@once"`` set it to *False*. + like ``schedule=None`` and ``"@once"`` set it to *False*. """ can_run: bool = True diff --git a/airflow/timetables/interval.py b/airflow/timetables/interval.py index 9e8386ac81cbb..4e162dc15b9a1 100644 --- a/airflow/timetables/interval.py +++ b/airflow/timetables/interval.py @@ -96,7 +96,7 @@ def next_dagrun_info( class CronDataIntervalTimetable(CronMixin, _DataIntervalTimetable): """Timetable that schedules data intervals with a cron expression. - This corresponds to ``schedule_interval=``, where ```` is either + This corresponds to ``schedule=``, where ```` is either a five/six-segment representation, or one of ``cron_presets``. The implementation extends on croniter to add timezone awareness. This is @@ -151,7 +151,7 @@ def infer_manual_data_interval(self, *, run_after: DateTime) -> DataInterval: class DeltaDataIntervalTimetable(_DataIntervalTimetable): """Timetable that schedules data intervals with a time delta. - This corresponds to ``schedule_interval=``, where ```` is + This corresponds to ``schedule=``, where ```` is either a ``datetime.timedelta`` or ``dateutil.relativedelta.relativedelta`` instance. """ diff --git a/airflow/timetables/simple.py b/airflow/timetables/simple.py index 1ab00403a3f45..80e2232108295 100644 --- a/airflow/timetables/simple.py +++ b/airflow/timetables/simple.py @@ -51,7 +51,7 @@ def infer_manual_data_interval(self, *, run_after: DateTime) -> DataInterval: class NullTimetable(_TrivialTimetable): """Timetable that never schedules anything. - This corresponds to ``schedule_interval=None``. + This corresponds to ``schedule=None``. """ description: str = "Never, external triggers only" @@ -72,7 +72,7 @@ def next_dagrun_info( class OnceTimetable(_TrivialTimetable): """Timetable that schedules the execution once as soon as possible. - This corresponds to ``schedule_interval="@once"``. + This corresponds to ``schedule="@once"``. """ description: str = "Once, as soon as possible" diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index fe4dab0614258..3383dc7e6e810 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -130,7 +130,7 @@ Bad example: with DAG( dag_id="example_python_operator", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], @@ -158,7 +158,7 @@ Good example: with DAG( dag_id="example_python_operator", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], @@ -289,7 +289,7 @@ It's easier to grab the concept with an example. Let's say that we have the foll with DAG( dag_id="watcher_example", - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as dag: @@ -499,7 +499,7 @@ This is an example test want to verify the structure of a code-generated DAG aga def dag(): with DAG( dag_id=TEST_DAG_ID, - schedule_interval="@daily", + schedule="@daily", start_date=DATA_INTERVAL_START, ) as dag: MyCustomOperator( diff --git a/docs/apache-airflow/concepts/dags.rst b/docs/apache-airflow/concepts/dags.rst index 14fb36524dc64..881082965ab38 100644 --- a/docs/apache-airflow/concepts/dags.rst +++ b/docs/apache-airflow/concepts/dags.rst @@ -39,7 +39,7 @@ which will add the DAG to anything inside it implicitly:: with DAG( "my_dag_name", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval="@daily", catchup=False + schedule="@daily", catchup=False ) as dag: op = EmptyOperator(task_id="task") @@ -47,13 +47,13 @@ Or, you can use a standard constructor, passing the dag into any operators you use:: my_dag = DAG("my_dag_name", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval="@daily", catchup=False) + schedule="@daily", catchup=False) op = EmptyOperator(task_id="task", dag=my_dag) Or, you can use the ``@dag`` decorator to :ref:`turn a function into a DAG generator `:: @dag(start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval="@daily", catchup=False) + schedule="@daily", catchup=False) def generate_dag(): op = EmptyOperator(task_id="task") @@ -147,12 +147,12 @@ DAGs will run in one of two ways: 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"): + with DAG("my_daily_dag", schedule="@daily"): ... The ``schedule_interval`` argument takes any value that is a valid `Crontab `_ schedule value, so you could also do:: - with DAG("my_daily_dag", schedule_interval="0 * * * *"): + with DAG("my_daily_dag", schedule="0 * * * *"): ... .. tip:: @@ -222,7 +222,7 @@ Often, many Operators inside a DAG need the same set of default arguments (such with DAG( dag_id='my_dag', start_date=pendulum.datetime(2016, 1, 1, tz="UTC"), - schedule_interval='@daily', + schedule='@daily', catchup=False, default_args={'retries': 2}, ) as dag: @@ -408,7 +408,7 @@ You can also combine this with the :ref:`concepts:depends-on-past` functionality dag = DAG( dag_id="branch_without_trigger", - schedule_interval="@once", + schedule="@once", start_date=pendulum.datetime(2019, 2, 28, tz="UTC"), ) @@ -496,7 +496,7 @@ TaskGroup also supports ``default_args`` like DAG, it will overwrite the ``defau with DAG( dag_id='dag1', start_date=pendulum.datetime(2016, 1, 1, tz="UTC"), - schedule_interval="@daily", + schedule="@daily", catchup=False, default_args={'retries': 1}, ): @@ -579,7 +579,7 @@ This is especially useful if your tasks are built dynamically from configuration dag = DAG( "my_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval="@daily", + schedule="@daily", catchup=False, ) dag.doc_md = __doc__ diff --git a/docs/apache-airflow/concepts/operators.rst b/docs/apache-airflow/concepts/operators.rst index 935ae9e0ebeae..944ff64a52ee1 100644 --- a/docs/apache-airflow/concepts/operators.rst +++ b/docs/apache-airflow/concepts/operators.rst @@ -176,7 +176,7 @@ you can pass ``render_template_as_native_obj=True`` to the DAG as follows: dag = DAG( dag_id="example_template_as_python_object", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, render_template_as_native_obj=True, diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/dag-run.rst index 57cba84f89752..2e8d70b1ec3dd 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/docs/apache-airflow/dag-run.rst @@ -146,7 +146,7 @@ in the configuration file. When turned off, the scheduler creates a DAG run only }, start_date=pendulum.datetime(2015, 12, 1, tz="UTC"), description="A simple tutorial DAG", - schedule_interval="@daily", + schedule="@daily", catchup=False, ) @@ -253,7 +253,7 @@ Example of a parameterized DAG: dag = DAG( "example_parameterized_dag", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, ) diff --git a/docs/apache-airflow/executor/kubernetes.rst b/docs/apache-airflow/executor/kubernetes.rst index 3dcd388261718..e24582c7dfc5b 100644 --- a/docs/apache-airflow/executor/kubernetes.rst +++ b/docs/apache-airflow/executor/kubernetes.rst @@ -166,7 +166,7 @@ Here is an example of a task with both features: with DAG( dag_id="example_pod_template_file", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example3"], diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index c2568926ba63a..4958851972d91 100644 --- a/docs/apache-airflow/faq.rst +++ b/docs/apache-airflow/faq.rst @@ -189,7 +189,7 @@ until ``min_file_process_interval`` is reached since DAG Parser will look for mo dag = DAG( dag_id, - schedule_interval=schedule, + schedule=schedule, default_args=default_args, pendulum.datetime(2021, 9, 13, tz="UTC"), ) @@ -422,7 +422,7 @@ upstream task. pass - @dag(schedule_interval="@once", start_date=pendulum.datetime(2021, 1, 1, tz="UTC")) + @dag(schedule="@once", start_date=pendulum.datetime(2021, 1, 1, tz="UTC")) def my_dag(): a = a_func() b = b_func() diff --git a/docs/apache-airflow/howto/add-dag-tags.rst b/docs/apache-airflow/howto/add-dag-tags.rst index 236414a1edc7c..53a0197a5ce09 100644 --- a/docs/apache-airflow/howto/add-dag-tags.rst +++ b/docs/apache-airflow/howto/add-dag-tags.rst @@ -32,7 +32,7 @@ In your Dag file, pass a list of tags you want to add to DAG object: .. code-block:: python - dag = DAG(dag_id="example_dag_tag", schedule_interval="0 0 * * *", tags=["example"]) + dag = DAG(dag_id="example_dag_tag", schedule="0 0 * * *", tags=["example"]) **Screenshot**: diff --git a/docs/apache-airflow/howto/dynamic-dag-generation.rst b/docs/apache-airflow/howto/dynamic-dag-generation.rst index e627e224baaaa..6fd71b66224bb 100644 --- a/docs/apache-airflow/howto/dynamic-dag-generation.rst +++ b/docs/apache-airflow/howto/dynamic-dag-generation.rst @@ -71,7 +71,7 @@ Then you can import and use the ``ALL_TASKS`` constant in all your DAGs like tha with DAG( dag_id="my_dag", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/docs/apache-airflow/howto/timetable.rst b/docs/apache-airflow/howto/timetable.rst index 0748e137da66e..4dfb2d485b134 100644 --- a/docs/apache-airflow/howto/timetable.rst +++ b/docs/apache-airflow/howto/timetable.rst @@ -21,7 +21,7 @@ Customizing DAG Scheduling with Timetables For our example, let's say a company wants to run a job after each weekday to process data collected during the work day. The first intuitive answer to this -would be ``schedule_interval="0 0 * * 1-5"`` (midnight on Monday to Friday), but +would be ``schedule="0 0 * * 1-5"`` (midnight on Monday to Friday), but this means data collected on Friday will *not* be processed right after Friday ends, but on the next Monday, and that run's interval would be from midnight Friday to midnight *Monday*. What we want is: diff --git a/docs/apache-airflow/index.rst b/docs/apache-airflow/index.rst index 66ac7f9d45133..0743f5d7a686c 100644 --- a/docs/apache-airflow/index.rst +++ b/docs/apache-airflow/index.rst @@ -44,7 +44,7 @@ Take a look at the following snippet of code: from airflow.operators.python import PythonOperator # A DAG represents a workflow, a collection of tasks - with DAG(dag_id="demo", start_date=datetime(2022, 1, 1), schedule_interval="0 0 * * *") as dag: + with DAG(dag_id="demo", start_date=datetime(2022, 1, 1), schedule="0 0 * * *") as dag: # Tasks are represented as operators hello = BashOperator(task_id="hello", bash_command="echo hello") diff --git a/docs/apache-airflow/lineage.rst b/docs/apache-airflow/lineage.rst index 8029d94ceeb4a..d717f59e248bf 100644 --- a/docs/apache-airflow/lineage.rst +++ b/docs/apache-airflow/lineage.rst @@ -44,7 +44,7 @@ works. dag = DAG( dag_id="example_lineage", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - schedule_interval="0 0 * * *", + schedule="0 0 * * *", catchup=False, dagrun_timeout=datetime.timedelta(minutes=60), ) diff --git a/docs/apache-airflow/logging-monitoring/callbacks.rst b/docs/apache-airflow/logging-monitoring/callbacks.rst index da8c4a0b8ccf7..f0f014cf6f46f 100644 --- a/docs/apache-airflow/logging-monitoring/callbacks.rst +++ b/docs/apache-airflow/logging-monitoring/callbacks.rst @@ -69,7 +69,7 @@ In the following example, failures in any task call the ``task_failure_alert`` f with DAG( dag_id="example_callback", - schedule_interval=None, + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), dagrun_timeout=datetime.timedelta(minutes=60), catchup=False, diff --git a/docs/apache-airflow/tutorial.rst b/docs/apache-airflow/tutorial.rst index c7734942e87eb..1409adcbc1957 100644 --- a/docs/apache-airflow/tutorial.rst +++ b/docs/apache-airflow/tutorial.rst @@ -587,7 +587,7 @@ Putting all of the pieces together, we have our completed DAG. @dag( - schedule_interval="0 0 * * *", + schedule="0 0 * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, dagrun_timeout=datetime.timedelta(minutes=60), diff --git a/docs/docker-stack/docker-examples/extending/embedding-dags/test_dag.py b/docs/docker-stack/docker-examples/extending/embedding-dags/test_dag.py index 686158161567d..a26328c56795f 100644 --- a/docs/docker-stack/docker-examples/extending/embedding-dags/test_dag.py +++ b/docs/docker-stack/docker-examples/extending/embedding-dags/test_dag.py @@ -31,7 +31,7 @@ dag = DAG( DAG_NAME, - schedule_interval='*/10 * * * *', + schedule='*/10 * * * *', default_args={'depends_on_past': True}, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, diff --git a/tests/api_connexion/endpoints/test_dag_endpoint.py b/tests/api_connexion/endpoints/test_dag_endpoint.py index 09ef3c28ae050..cd14a7f7f8ac1 100644 --- a/tests/api_connexion/endpoints/test_dag_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_endpoint.py @@ -122,7 +122,7 @@ def _create_dag_models(self, count, session=None): dag_model = DagModel( dag_id=f"TEST_DAG_{num}", fileloc=f"/tmp/dag_{num}.py", - schedule_interval="2 2 * * *", + schedule="2 2 * * *", is_active=True, is_paused=False, ) @@ -133,7 +133,7 @@ def _create_deactivated_dag(self, session=None): dag_model = DagModel( dag_id="TEST_DAG_DELETED_1", fileloc="/tmp/dag_del_1.py", - schedule_interval="2 2 * * *", + schedule="2 2 * * *", is_active=False, ) session.add(dag_model) @@ -179,7 +179,7 @@ def test_should_respond_200_with_schedule_interval_none(self, session): dag_model = DagModel( dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", - schedule_interval=None, + schedule=None, is_paused=False, ) session.add(dag_model) @@ -901,7 +901,7 @@ def test_should_respond_404(self): @provide_session def _create_dag_model(self, session=None): dag_model = DagModel( - dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", schedule_interval="2 2 * * *", is_paused=True + dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", schedule="2 2 * * *", is_paused=True ) session.add(dag_model) return dag_model diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index 6735dd1fdeb65..e9d5cea0ef2b6 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -121,7 +121,7 @@ def _create_dag(self, dag_id): dag_instance = DagModel(dag_id=dag_id) with create_session() as session: session.add(dag_instance) - dag = DAG(dag_id=dag_id, schedule_interval=None) + dag = DAG(dag_id=dag_id, schedule=None) self.app.dag_bag.bag_dag(dag, root_dag=dag) return dag_instance diff --git a/tests/api_connexion/schemas/test_dag_schema.py b/tests/api_connexion/schemas/test_dag_schema.py index 040ce41f1d066..4f59b2108d2e9 100644 --- a/tests/api_connexion/schemas/test_dag_schema.py +++ b/tests/api_connexion/schemas/test_dag_schema.py @@ -37,7 +37,7 @@ def test_serialize_test_dag_schema(url_safe_serializer): fileloc="/root/airflow/dags/my_dag.py", owners="airflow1,airflow2", description="The description", - schedule_interval="5 4 * * *", + schedule="5 4 * * *", tags=[DagTag(name="tag-1"), DagTag(name="tag-2")], ) serialized_dag = DAGSchema().dump(dag_model) diff --git a/tests/cli/commands/test_dag_command.py b/tests/cli/commands/test_dag_command.py index ae2baef2f1634..d175df2ed6e64 100644 --- a/tests/cli/commands/test_dag_command.py +++ b/tests/cli/commands/test_dag_command.py @@ -354,9 +354,9 @@ def test_next_execution(self): dag_ids = [ 'example_bash_operator', # schedule_interval is '0 0 * * *' 'latest_only', # schedule_interval is timedelta(hours=4) - 'example_python_operator', # schedule_interval=None + 'example_python_operator', # schedule=None 'example_xcom', - ] # schedule_interval="@once" + ] # schedule="@once" # Delete DagRuns with create_session() as session: diff --git a/tests/core/test_core.py b/tests/core/test_core.py index e63ed955bb324..80b47ee3a1fe9 100644 --- a/tests/core/test_core.py +++ b/tests/core/test_core.py @@ -123,7 +123,7 @@ def test_externally_triggered_dagrun(self, dag_maker): execution_ds = execution_date.strftime('%Y-%m-%d') execution_ds_nodash = execution_ds.replace('-', '') - with dag_maker(schedule_interval=timedelta(weeks=1)): + with dag_maker(schedule=timedelta(weeks=1)): task = EmptyOperator(task_id='test_externally_triggered_dag_context') dag_maker.create_dagrun( run_type=DagRunType.SCHEDULED, @@ -150,7 +150,7 @@ def test_dag_params_and_task_params(self, dag_maker): TI = TaskInstance with dag_maker( - schedule_interval=timedelta(weeks=1), + schedule=timedelta(weeks=1), params={'key_1': 'value_1', 'key_2': 'value_2_old'}, ): task1 = EmptyOperator( diff --git a/tests/core/test_sentry.py b/tests/core/test_sentry.py index d68e37cdc2580..da8f332b9e7ca 100644 --- a/tests/core/test_sentry.py +++ b/tests/core/test_sentry.py @@ -72,7 +72,7 @@ class TestSentryHook: @pytest.fixture def task_instance(self, dag_maker): # Mock the Dag - with dag_maker(DAG_ID, schedule_interval=SCHEDULE_INTERVAL): + with dag_maker(DAG_ID, schedule=SCHEDULE_INTERVAL): task = PythonOperator(task_id=TASK_ID, python_callable=int) dr = dag_maker.create_dagrun(data_interval=DATA_INTERVAL, execution_date=EXECUTION_DATE) diff --git a/tests/dag_processing/test_manager.py b/tests/dag_processing/test_manager.py index bcca03d5598e5..f88c27dafd3fe 100644 --- a/tests/dag_processing/test_manager.py +++ b/tests/dag_processing/test_manager.py @@ -690,7 +690,7 @@ def test_send_file_processing_statsd_timing(self, statsd_timing_mock, tmpdir): dag_code = dedent( """ from airflow import DAG - dag = DAG(dag_id='temp_dag', schedule_interval='0 0 * * *') + dag = DAG(dag_id='temp_dag', schedule='0 0 * * *') """ ) with open(filename_to_parse, 'w') as file_to_parse: diff --git a/tests/dags/subdir2/test_dont_ignore_this.py b/tests/dags/subdir2/test_dont_ignore_this.py index 6af1f19a239ec..2d4c150771882 100644 --- a/tests/dags/subdir2/test_dont_ignore_this.py +++ b/tests/dags/subdir2/test_dont_ignore_this.py @@ -22,5 +22,5 @@ DEFAULT_DATE = datetime(2019, 12, 1) -dag = DAG(dag_id='test_dag_under_subdir2', start_date=DEFAULT_DATE, schedule_interval=None) +dag = DAG(dag_id='test_dag_under_subdir2', start_date=DEFAULT_DATE, schedule=None) task = BashOperator(task_id='task1', bash_command='echo "test dag under sub directory subdir2"', dag=dag) diff --git a/tests/dags/test_clear_subdag.py b/tests/dags/test_clear_subdag.py index ef551a43cb6de..be3c2874ea428 100644 --- a/tests/dags/test_clear_subdag.py +++ b/tests/dags/test_clear_subdag.py @@ -30,7 +30,7 @@ def create_subdag_opt(main_dag): subdag = DAG( dag_id='.'.join([dag_name, subdag_name]), start_date=start_date, - schedule_interval=None, + schedule=None, max_active_tasks=2, ) BashOperator(bash_command="echo 1", task_id="daily_job_subdag_task", dag=subdag) @@ -46,7 +46,7 @@ def create_subdag_opt(main_dag): start_date = datetime.datetime(2016, 1, 1) -dag = DAG(dag_id=dag_name, max_active_tasks=3, start_date=start_date, schedule_interval="0 0 * * *") +dag = DAG(dag_id=dag_name, max_active_tasks=3, start_date=start_date, schedule="0 0 * * *") daily_job_irrelevant = BashOperator( bash_command="echo 1", diff --git a/tests/dags/test_dag_with_no_tags.py b/tests/dags/test_dag_with_no_tags.py index eb2761381f6ff..85a7eab0f164d 100644 --- a/tests/dags/test_dag_with_no_tags.py +++ b/tests/dags/test_dag_with_no_tags.py @@ -27,5 +27,5 @@ "start_date": DEFAULT_DATE, } -with DAG(dag_id="test_dag_with_no_tags", default_args=default_args, schedule_interval='@once') as dag: +with DAG(dag_id="test_dag_with_no_tags", default_args=default_args, schedule='@once') as dag: task_a = EmptyOperator(task_id="test_task_a") diff --git a/tests/dags/test_datasets.py b/tests/dags/test_datasets.py index 46d4fb8585c41..0a464522c2376 100644 --- a/tests/dags/test_datasets.py +++ b/tests/dags/test_datasets.py @@ -35,7 +35,7 @@ def raise_skip_exc(): dag_id='dag_with_skip_task', catchup=False, start_date=datetime(2020, 1, 1), - schedule_interval='@daily', + schedule='@daily', tags=['upstream-skipping'], ) PythonOperator( @@ -66,7 +66,7 @@ def raise_fail_exc(): dag_id='dag_with_fail_task', catchup=False, start_date=datetime(2020, 1, 1), - schedule_interval='@daily', + schedule='@daily', tags=['upstream-skipping'], ) PythonOperator( diff --git a/tests/dags/test_default_views.py b/tests/dags/test_default_views.py index 6a1aefb4ee28e..fa9aecd482496 100644 --- a/tests/dags/test_default_views.py +++ b/tests/dags/test_default_views.py @@ -25,13 +25,13 @@ tree_dag = DAG( dag_id='test_tree_view', default_args=args, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', default_view='grid', ) graph_dag = DAG( dag_id='test_graph_view', default_args=args, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', default_view='graph', ) diff --git a/tests/dags/test_example_bash_operator.py b/tests/dags/test_example_bash_operator.py index 288667ad06b7b..eedf0beb77c13 100644 --- a/tests/dags/test_example_bash_operator.py +++ b/tests/dags/test_example_bash_operator.py @@ -24,7 +24,7 @@ dag = DAG( dag_id='test_example_bash_operator', default_args={'owner': 'airflow', 'retries': 3, 'start_date': datetime.datetime(2022, 1, 1)}, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', dagrun_timeout=datetime.timedelta(minutes=60), ) diff --git a/tests/dags/test_invalid_cron.py b/tests/dags/test_invalid_cron.py index 4e7eda9e12f59..48c4f36a44064 100644 --- a/tests/dags/test_invalid_cron.py +++ b/tests/dags/test_invalid_cron.py @@ -24,5 +24,5 @@ # Cron expression. This invalid DAG will be used to # test whether dagbag.process_file() can identify # invalid Cron expression. -dag1 = DAG(dag_id='test_invalid_cron', start_date=datetime(2015, 1, 1), schedule_interval="0 100 * * *") +dag1 = DAG(dag_id='test_invalid_cron', start_date=datetime(2015, 1, 1), schedule="0 100 * * *") dag1_task1 = EmptyOperator(task_id='task1', dag=dag1, owner='airflow') diff --git a/tests/dags/test_invalid_param.py b/tests/dags/test_invalid_param.py index ebbe7dbef1473..c8c779c322770 100644 --- a/tests/dags/test_invalid_param.py +++ b/tests/dags/test_invalid_param.py @@ -24,7 +24,7 @@ with DAG( "test_invalid_param", start_date=datetime(2021, 1, 1), - schedule_interval="@once", + schedule="@once", params={ # a mandatory str param "str_param": Param(type="string", minLength=2, maxLength=4), diff --git a/tests/dags/test_logging_in_dag.py b/tests/dags/test_logging_in_dag.py index 8b0c629de750a..35434f6c4cd0a 100644 --- a/tests/dags/test_logging_in_dag.py +++ b/tests/dags/test_logging_in_dag.py @@ -35,7 +35,7 @@ def test_logging_fn(**kwargs): print("Log from Print statement") -dag = DAG(dag_id='test_logging_dag', schedule_interval=None, start_date=datetime(2016, 1, 1)) +dag = DAG(dag_id='test_logging_dag', schedule=None, start_date=datetime(2016, 1, 1)) PythonOperator( task_id='test_task', diff --git a/tests/dags/test_miscellaneous.py b/tests/dags/test_miscellaneous.py index 2174a614782c5..2f37c4b96a560 100644 --- a/tests/dags/test_miscellaneous.py +++ b/tests/dags/test_miscellaneous.py @@ -31,7 +31,7 @@ dag = DAG( dag_id='miscellaneous_test_dag', default_args=args, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', start_date=datetime.datetime(2022, 1, 1), dagrun_timeout=datetime.timedelta(minutes=60), tags=['example', 'example2'], diff --git a/tests/dags/test_missing_owner.py b/tests/dags/test_missing_owner.py index cc20db08c63e2..d76925148d4d8 100644 --- a/tests/dags/test_missing_owner.py +++ b/tests/dags/test_missing_owner.py @@ -23,7 +23,7 @@ with DAG( dag_id="test_missing_owner", - schedule_interval="0 0 * * *", + schedule="0 0 * * *", start_date=datetime.datetime(2022, 1, 1), dagrun_timeout=datetime.timedelta(minutes=60), tags=["example"], diff --git a/tests/dags/test_multiple_dags.py b/tests/dags/test_multiple_dags.py index 67ae544123150..009a805c948ea 100644 --- a/tests/dags/test_multiple_dags.py +++ b/tests/dags/test_multiple_dags.py @@ -27,7 +27,7 @@ def create_dag(suffix): dag = DAG( dag_id=f'test_multiple_dags__{suffix}', default_args=args, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', dagrun_timeout=datetime.timedelta(minutes=60), ) diff --git a/tests/dags/test_only_empty_tasks.py b/tests/dags/test_only_empty_tasks.py index 2bf70c6319a70..72f5031a4ef2e 100644 --- a/tests/dags/test_only_empty_tasks.py +++ b/tests/dags/test_only_empty_tasks.py @@ -28,7 +28,7 @@ "start_date": DEFAULT_DATE, } -dag = DAG(dag_id="test_only_empty_tasks", default_args=default_args, schedule_interval='@once') +dag = DAG(dag_id="test_only_empty_tasks", default_args=default_args, schedule='@once') class MyEmptyOperator(EmptyOperator): diff --git a/tests/dags/test_retry_handling_job.py b/tests/dags/test_retry_handling_job.py index 42bf9a47331e1..0826b1ee4f839 100644 --- a/tests/dags/test_retry_handling_job.py +++ b/tests/dags/test_retry_handling_job.py @@ -32,6 +32,6 @@ 'retry_delay': timedelta(seconds=0), } -dag = DAG('test_retry_handling_job', default_args=default_args, schedule_interval='@once') +dag = DAG('test_retry_handling_job', default_args=default_args, schedule='@once') task1 = BashOperator(task_id='test_retry_handling_op', bash_command='exit 1', dag=dag) diff --git a/tests/dags/test_sensor.py b/tests/dags/test_sensor.py index 760300ef00cd7..e886d1a58f120 100644 --- a/tests/dags/test_sensor.py +++ b/tests/dags/test_sensor.py @@ -23,7 +23,7 @@ from airflow.utils import timezone with DAG( - dag_id='test_sensor', start_date=datetime.datetime(2022, 1, 1), catchup=False, schedule_interval='@once' + dag_id='test_sensor', start_date=datetime.datetime(2022, 1, 1), catchup=False, schedule='@once' ) as dag: @task diff --git a/tests/dags/test_subdag.py b/tests/dags/test_subdag.py index 28e294054f78b..9046a43519405 100644 --- a/tests/dags/test_subdag.py +++ b/tests/dags/test_subdag.py @@ -44,7 +44,7 @@ def subdag(parent_dag_name, child_dag_name, args): dag_subdag = DAG( dag_id=f'{parent_dag_name}.{child_dag_name}', default_args=args, - schedule_interval="@daily", + schedule="@daily", ) for i in range(2): @@ -62,7 +62,7 @@ def subdag(parent_dag_name, child_dag_name, args): start_date=datetime(2019, 1, 1), max_active_runs=1, default_args=DEFAULT_TASK_ARGS, - schedule_interval=timedelta(minutes=1), + schedule=timedelta(minutes=1), ) as dag: start = EmptyOperator( diff --git a/tests/dags/test_with_non_default_owner.py b/tests/dags/test_with_non_default_owner.py index 43a2953f9c1f5..2445691eb290c 100644 --- a/tests/dags/test_with_non_default_owner.py +++ b/tests/dags/test_with_non_default_owner.py @@ -23,7 +23,7 @@ with DAG( dag_id="test_with_non_default_owner", - schedule_interval="0 0 * * *", + schedule="0 0 * * *", start_date=datetime.datetime(2022, 1, 1), dagrun_timeout=datetime.timedelta(minutes=60), tags=["example"], diff --git a/tests/dags_corrupted/test_nonstring_owner.py b/tests/dags_corrupted/test_nonstring_owner.py index ace1e4ce0a8f1..6a8a50d37051e 100644 --- a/tests/dags_corrupted/test_nonstring_owner.py +++ b/tests/dags_corrupted/test_nonstring_owner.py @@ -23,7 +23,7 @@ with DAG( dag_id="test_nonstring_owner", - schedule_interval="0 0 * * *", + schedule="0 0 * * *", start_date=datetime(2022, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=["example"], diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index bf6032595b737..309334d8ab0e1 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -94,7 +94,7 @@ def _get_dummy_dag( task_id='op', **kwargs, ): - with dag_maker_fixture(dag_id=dag_id, schedule_interval='@daily', **kwargs) as dag: + with dag_maker_fixture(dag_id=dag_id, schedule='@daily', **kwargs) as dag: EmptyOperator(task_id=task_id, pool=pool, max_active_tis_per_dag=max_active_tis_per_dag) return dag @@ -677,7 +677,7 @@ def test_backfill_rerun_failed_tasks(self, dag_maker): def test_backfill_rerun_upstream_failed_tasks(self, dag_maker): - with dag_maker(dag_id='test_backfill_rerun_upstream_failed', schedule_interval='@daily') as dag: + with dag_maker(dag_id='test_backfill_rerun_upstream_failed', schedule='@daily') as dag: op1 = EmptyOperator(task_id='test_backfill_rerun_upstream_failed_task-1') op2 = EmptyOperator(task_id='test_backfill_rerun_upstream_failed_task-2') op1.set_upstream(op2) @@ -743,7 +743,7 @@ def test_backfill_rerun_failed_tasks_without_flag(self, dag_maker): def test_backfill_retry_intermittent_failed_task(self, dag_maker): with dag_maker( dag_id='test_intermittent_failure_job', - schedule_interval="@daily", + schedule="@daily", default_args={ 'retries': 2, 'retry_delay': datetime.timedelta(seconds=0), @@ -770,7 +770,7 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): def test_backfill_retry_always_failed_task(self, dag_maker): with dag_maker( dag_id='test_always_failure_job', - schedule_interval="@daily", + schedule="@daily", default_args={ 'retries': 1, 'retry_delay': datetime.timedelta(seconds=0), @@ -797,7 +797,7 @@ def test_backfill_ordered_concurrent_execute(self, dag_maker): with dag_maker( dag_id='test_backfill_ordered_concurrent_execute', - schedule_interval="@daily", + schedule="@daily", ) as dag: op1 = EmptyOperator(task_id='leave1') op2 = EmptyOperator(task_id='leave2') @@ -942,7 +942,7 @@ def _get_dag_test_max_active_limits( ): with dag_maker_fixture( dag_id=dag_id, - schedule_interval="@hourly", + schedule="@hourly", max_active_runs=max_active_runs, **kwargs, ) as dag: @@ -1418,7 +1418,7 @@ def test_update_counters(self, dag_maker, session): def test_dag_dagrun_infos_between(self, dag_maker): with dag_maker( - dag_id='dagrun_infos_between', start_date=DEFAULT_DATE, schedule_interval="@hourly" + dag_id='dagrun_infos_between', start_date=DEFAULT_DATE, schedule="@hourly" ) as test_dag: EmptyOperator( task_id='dummy', @@ -1545,7 +1545,7 @@ def test_reset_orphaned_tasks_specified_dagrun(self, session, dag_maker): with dag_maker( dag_id=dag_id, start_date=DEFAULT_DATE, - schedule_interval='@daily', + schedule='@daily', session=session, ) as dag: EmptyOperator(task_id=task_id, dag=dag) @@ -1574,7 +1574,7 @@ def test_reset_orphaned_tasks_specified_dagrun(self, session, dag_maker): def test_job_id_is_assigned_to_dag_run(self, dag_maker): dag_id = 'test_job_id_is_assigned_to_dag_run' - with dag_maker(dag_id=dag_id, start_date=DEFAULT_DATE, schedule_interval='@daily') as dag: + with dag_maker(dag_id=dag_id, start_date=DEFAULT_DATE, schedule='@daily') as dag: EmptyOperator(task_id="dummy_task", dag=dag) job = BackfillJob( diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index ba055b295cfe5..8eac1addf061a 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -1928,7 +1928,7 @@ def test_do_not_schedule_removed_task(self, dag_maker): schedule_interval = datetime.timedelta(days=1) with dag_maker( dag_id='test_scheduler_do_not_schedule_removed_task', - schedule_interval=schedule_interval, + schedule=schedule_interval, ): EmptyOperator(task_id='dummy') @@ -1942,7 +1942,7 @@ def test_do_not_schedule_removed_task(self, dag_maker): session.query(DagModel).delete() with dag_maker( dag_id='test_scheduler_do_not_schedule_removed_task', - schedule_interval=schedule_interval, + schedule=schedule_interval, start_date=DEFAULT_DATE + schedule_interval, ): pass @@ -2620,7 +2620,7 @@ def test_retry_still_in_executor(self, dag_maker): with create_session() as session: with dag_maker( dag_id='test_retry_still_in_executor', - schedule_interval="@once", + schedule="@once", session=session, ): dag_task1 = BashOperator( @@ -2721,9 +2721,7 @@ def test_dag_get_active_runs(self, dag_maker): dag_name1 = 'get_active_runs_test' default_args = {'depends_on_past': False, 'start_date': start_date} - with dag_maker( - dag_name1, schedule_interval='* * * * *', max_active_runs=1, default_args=default_args - ) as dag1: + with dag_maker(dag_name1, schedule='* * * * *', max_active_runs=1, default_args=default_args) as dag1: run_this_1 = EmptyOperator(task_id='run_this_1') run_this_2 = EmptyOperator(task_id='run_this_2') @@ -2795,7 +2793,7 @@ def test_adopt_or_reset_orphaned_tasks_nothing(self): def test_adopt_or_reset_orphaned_tasks_external_triggered_dag(self, dag_maker): dag_id = 'test_reset_orphaned_tasks_external_triggered_dag' - with dag_maker(dag_id=dag_id, schedule_interval='@daily'): + with dag_maker(dag_id=dag_id, schedule='@daily'): task_id = dag_id + '_task' EmptyOperator(task_id=task_id) @@ -2814,7 +2812,7 @@ def test_adopt_or_reset_orphaned_tasks_external_triggered_dag(self, dag_maker): def test_adopt_or_reset_orphaned_tasks_backfill_dag(self, dag_maker): dag_id = 'test_adopt_or_reset_orphaned_tasks_backfill_dag' - with dag_maker(dag_id=dag_id, schedule_interval='@daily'): + with dag_maker(dag_id=dag_id, schedule='@daily'): task_id = dag_id + '_task' EmptyOperator(task_id=task_id) @@ -2837,7 +2835,7 @@ def test_adopt_or_reset_orphaned_tasks_backfill_dag(self, dag_maker): def test_reset_orphaned_tasks_no_orphans(self, dag_maker): dag_id = 'test_reset_orphaned_tasks_no_orphans' - with dag_maker(dag_id=dag_id, schedule_interval='@daily'): + with dag_maker(dag_id=dag_id, schedule='@daily'): task_id = dag_id + '_task' EmptyOperator(task_id=task_id) @@ -2861,7 +2859,7 @@ def test_reset_orphaned_tasks_no_orphans(self, dag_maker): def test_reset_orphaned_tasks_non_running_dagruns(self, dag_maker): """Ensure orphaned tasks with non-running dagruns are not reset.""" dag_id = 'test_reset_orphaned_tasks_non_running_dagruns' - with dag_maker(dag_id=dag_id, schedule_interval='@daily'): + with dag_maker(dag_id=dag_id, schedule='@daily'): task_id = dag_id + '_task' EmptyOperator(task_id=task_id) @@ -2884,7 +2882,7 @@ def test_reset_orphaned_tasks_non_running_dagruns(self, dag_maker): def test_adopt_or_reset_orphaned_tasks_stale_scheduler_jobs(self, dag_maker): dag_id = 'test_adopt_or_reset_orphaned_tasks_stale_scheduler_jobs' - with dag_maker(dag_id=dag_id, schedule_interval='@daily'): + with dag_maker(dag_id=dag_id, schedule='@daily'): EmptyOperator(task_id='task1') EmptyOperator(task_id='task2') @@ -2963,7 +2961,7 @@ def test_adopt_or_reset_orphaned_tasks_only_fails_scheduler_jobs(self, caplog): def test_send_sla_callbacks_to_processor_sla_disabled(self, dag_maker): """Test SLA Callbacks are not sent when check_slas is False""" dag_id = 'test_send_sla_callbacks_to_processor_sla_disabled' - with dag_maker(dag_id=dag_id, schedule_interval='@daily') as dag: + with dag_maker(dag_id=dag_id, schedule='@daily') as dag: EmptyOperator(task_id='task1') with patch.object(settings, "CHECK_SLAS", False): @@ -2976,7 +2974,7 @@ def test_send_sla_callbacks_to_processor_sla_disabled(self, dag_maker): def test_send_sla_callbacks_to_processor_sla_no_task_slas(self, dag_maker): """Test SLA Callbacks are not sent when no task SLAs are defined""" dag_id = 'test_send_sla_callbacks_to_processor_sla_no_task_slas' - with dag_maker(dag_id=dag_id, schedule_interval='@daily') as dag: + with dag_maker(dag_id=dag_id, schedule='@daily') as dag: EmptyOperator(task_id='task1') with patch.object(settings, "CHECK_SLAS", True): @@ -2989,7 +2987,7 @@ def test_send_sla_callbacks_to_processor_sla_no_task_slas(self, dag_maker): def test_send_sla_callbacks_to_processor_sla_with_task_slas(self, dag_maker): """Test SLA Callbacks are sent to the DAG Processor when SLAs are defined on tasks""" dag_id = 'test_send_sla_callbacks_to_processor_sla_with_task_slas' - with dag_maker(dag_id=dag_id, schedule_interval='@daily') as dag: + with dag_maker(dag_id=dag_id, schedule='@daily') as dag: EmptyOperator(task_id='task1', sla=timedelta(seconds=60)) with patch.object(settings, "CHECK_SLAS", True): @@ -3118,7 +3116,7 @@ def test_bulk_write_to_db_external_trigger_dont_skip_scheduled_run(self, dag_mak """ with dag_maker( dag_id='test_bulk_write_to_db_external_trigger_dont_skip_scheduled_run', - schedule_interval="*/1 * * * *", + schedule="*/1 * * * *", max_active_runs=5, catchup=True, ) as dag: @@ -3179,7 +3177,7 @@ def test_scheduler_create_dag_runs_check_existing_run(self, dag_maker): """ with dag_maker( dag_id='test_scheduler_create_dag_runs_check_existing_run', - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), ) as dag: EmptyOperator( task_id='dummy', @@ -3224,7 +3222,7 @@ def test_do_schedule_max_active_runs_dag_timed_out(self, dag_maker): with dag_maker( dag_id='test_max_active_run_with_dag_timed_out', - schedule_interval='@once', + schedule='@once', max_active_runs=1, catchup=True, dagrun_timeout=datetime.timedelta(seconds=1), @@ -3282,7 +3280,7 @@ def test_do_schedule_max_active_runs_task_removed(self, session, dag_maker): with dag_maker( dag_id='test_do_schedule_max_active_runs_task_removed', start_date=DEFAULT_DATE, - schedule_interval='@once', + schedule='@once', max_active_runs=1, session=session, ): @@ -3415,7 +3413,7 @@ def test_do_schedule_max_active_runs_and_manual_trigger(self, dag_maker): with dag_maker( dag_id='test_max_active_run_plus_manual_trigger', - schedule_interval='@once', + schedule='@once', max_active_runs=1, ) as dag: # Can't use EmptyOperator as that goes straight to success @@ -3469,7 +3467,7 @@ def test_max_active_runs_in_a_dag_doesnt_stop_running_dagruns_in_otherdags(self, with dag_maker( 'test_dag1', start_date=DEFAULT_DATE, - schedule_interval=timedelta(hours=1), + schedule=timedelta(hours=1), max_active_runs=1, ): EmptyOperator(task_id='mytask') @@ -3481,7 +3479,7 @@ def test_max_active_runs_in_a_dag_doesnt_stop_running_dagruns_in_otherdags(self, with dag_maker( 'test_dag2', start_date=timezone.datetime(2020, 1, 1), - schedule_interval=timedelta(hours=1), + schedule=timedelta(hours=1), ): EmptyOperator(task_id='mytask') @@ -3829,7 +3827,7 @@ def test_timeout_triggers(self, dag_maker): with dag_maker( dag_id='test_timeout_triggers', start_date=DEFAULT_DATE, - schedule_interval='@once', + schedule='@once', max_active_runs=1, session=session, ): @@ -4150,7 +4148,7 @@ def test_catchup_works_correctly(self, dag_maker): session = settings.Session() with dag_maker( dag_id='test_catchup_schedule_dag', - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), start_date=DEFAULT_DATE, catchup=True, max_active_runs=1, diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 1a2aa72c8b58b..915a0e3fcc757 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -231,7 +231,7 @@ def test_dag_as_context_manager(self): def test_dag_topological_sort_include_subdag_tasks(self): child_dag = DAG( 'parent_dag.child_dag', - schedule_interval='@daily', + schedule='@daily', start_date=DEFAULT_DATE, ) @@ -241,7 +241,7 @@ def test_dag_topological_sort_include_subdag_tasks(self): parent_dag = DAG( 'parent_dag', - schedule_interval='@daily', + schedule='@daily', start_date=DEFAULT_DATE, ) @@ -539,7 +539,7 @@ def test_following_previous_schedule(self): utc = timezone.convert_to_utc(start) assert utc.isoformat() == "2018-10-28T00:55:00+00:00", "Pre-condition: correct DST->UTC conversion" - dag = DAG('tz_dag', start_date=start, schedule_interval='*/5 * * * *') + dag = DAG('tz_dag', start_date=start, schedule='*/5 * * * *') _next = dag.following_schedule(utc) next_local = local_tz.convert(_next) @@ -566,7 +566,7 @@ def test_following_previous_schedule_daily_dag_cest_to_cet(self): utc = timezone.convert_to_utc(start) - dag = DAG('tz_dag', start_date=start, schedule_interval='0 3 * * *') + dag = DAG('tz_dag', start_date=start, schedule='0 3 * * *') prev = dag.previous_schedule(utc) prev_local = local_tz.convert(prev) @@ -595,7 +595,7 @@ def test_following_previous_schedule_daily_dag_cet_to_cest(self): utc = timezone.convert_to_utc(start) - dag = DAG('tz_dag', start_date=start, schedule_interval='0 3 * * *') + dag = DAG('tz_dag', start_date=start, schedule='0 3 * * *') prev = dag.previous_schedule(utc) prev_local = local_tz.convert(prev) @@ -621,7 +621,7 @@ def test_following_schedule_relativedelta(self): """ dag_id = "test_schedule_dag_relativedelta" delta = relativedelta(hours=+1) - dag = DAG(dag_id=dag_id, schedule_interval=delta) + dag = DAG(dag_id=dag_id, schedule=delta) dag.add_task(BaseOperator(task_id="faketastic", owner='Also fake', start_date=TEST_DATE)) _next = dag.following_schedule(TEST_DATE) @@ -634,7 +634,7 @@ def test_previous_schedule_datetime_timezone(self): # Check that we don't get an AttributeError 'name' for self.timezone start = datetime.datetime(2018, 3, 25, 2, tzinfo=datetime.timezone.utc) - dag = DAG('tz_dag', start_date=start, schedule_interval='@hourly') + dag = DAG('tz_dag', start_date=start, schedule='@hourly') when = dag.previous_schedule(start) assert when.isoformat() == "2018-03-25T01:00:00+00:00" @@ -642,7 +642,7 @@ def test_following_schedule_datetime_timezone(self): # Check that we don't get an AttributeError 'name' for self.timezone start = datetime.datetime(2018, 3, 25, 2, tzinfo=datetime.timezone.utc) - dag = DAG('tz_dag', start_date=start, schedule_interval='@hourly') + dag = DAG('tz_dag', start_date=start, schedule='@hourly') when = dag.following_schedule(start) assert when.isoformat() == "2018-03-25T03:00:00+00:00" @@ -666,7 +666,7 @@ def tzname(self, dt): return self.__class__._name start = datetime.datetime(2018, 3, 25, 10, tzinfo=UTC0530()) - dag = DAG('tz_dag', start_date=start, schedule_interval='@hourly') + dag = DAG('tz_dag', start_date=start, schedule='@hourly') when = dag.following_schedule(start) assert when.isoformat() == "2018-03-25T05:30:00+00:00" @@ -1201,7 +1201,7 @@ def test_next_dagrun_after_fake_scheduled_previous(self): """ delta = datetime.timedelta(hours=1) dag_id = "test_schedule_dag_fake_scheduled_previous" - dag = DAG(dag_id=dag_id, schedule_interval=delta, start_date=DEFAULT_DATE) + dag = DAG(dag_id=dag_id, schedule=delta, start_date=DEFAULT_DATE) dag.add_task(BaseOperator(task_id="faketastic", owner='Also fake', start_date=DEFAULT_DATE)) dag.create_dagrun( @@ -1227,7 +1227,7 @@ def test_schedule_dag_once(self): it is called, and not scheduled the second. """ dag_id = "test_schedule_dag_once" - dag = DAG(dag_id=dag_id, schedule_interval="@once") + dag = DAG(dag_id=dag_id, schedule="@once") assert isinstance(dag.timetable, OnceTimetable) dag.add_task(BaseOperator(task_id="faketastic", owner='Also fake', start_date=TEST_DATE)) @@ -1250,7 +1250,7 @@ def test_fractional_seconds(self): Tests if fractional seconds are stored in the database """ dag_id = "test_fractional_seconds" - dag = DAG(dag_id=dag_id, schedule_interval="@once") + dag = DAG(dag_id=dag_id, schedule="@once") dag.add_task(BaseOperator(task_id="faketastic", owner='Also fake', start_date=TEST_DATE)) start_date = timezone.utcnow() @@ -1354,7 +1354,7 @@ def test_get_paused_dag_ids(self): def test_timetable_and_description_from_schedule_interval( self, schedule_interval, expected_timetable, interval_description ): - dag = DAG("test_schedule_interval", schedule_interval=schedule_interval) + dag = DAG("test_schedule_interval", schedule=schedule_interval) assert dag.timetable == expected_timetable assert dag.schedule_interval == schedule_interval assert dag.timetable.description == interval_description @@ -1678,9 +1678,7 @@ def test_clear_dag(self, ti_state_begin, ti_state_end: Optional[str]): self._clean_up(dag_id) def test_next_dagrun_info_once(self): - dag = DAG( - 'test_scheduler_dagrun_once', start_date=timezone.datetime(2015, 1, 1), schedule_interval="@once" - ) + dag = DAG('test_scheduler_dagrun_once', start_date=timezone.datetime(2015, 1, 1), schedule="@once") next_info = dag.next_dagrun_info(None) assert next_info and next_info.logical_date == timezone.datetime(2015, 1, 1) @@ -1698,7 +1696,7 @@ def test_next_dagrun_info_start_end_dates(self): start_date = DEFAULT_DATE end_date = start_date + (runs - 1) * delta dag_id = "test_schedule_dag_start_end_dates" - dag = DAG(dag_id=dag_id, start_date=start_date, end_date=end_date, schedule_interval=delta) + dag = DAG(dag_id=dag_id, start_date=start_date, end_date=end_date, schedule=delta) dag.add_task(BaseOperator(task_id='faketastic', owner='Also fake')) # Create and schedule the dag runs @@ -1728,7 +1726,7 @@ def make_dag(dag_id, schedule_interval, start_date, catchup): } dag = DAG( dag_id, - schedule_interval=schedule_interval, + schedule=schedule_interval, start_date=start_date, catchup=catchup, default_args=default_args, @@ -1750,7 +1748,7 @@ def make_dag(dag_id, schedule_interval, start_date, catchup): dag1 = make_dag( dag_id='dag_without_catchup_ten_minute', - schedule_interval='*/10 * * * *', + schedule='*/10 * * * *', start_date=six_hours_ago_to_the_hour, catchup=False, ) @@ -1761,7 +1759,7 @@ def make_dag(dag_id, schedule_interval, start_date, catchup): dag2 = make_dag( dag_id='dag_without_catchup_hourly', - schedule_interval='@hourly', + schedule='@hourly', start_date=six_hours_ago_to_the_hour, catchup=False, ) @@ -1774,7 +1772,7 @@ def make_dag(dag_id, schedule_interval, start_date, catchup): dag3 = make_dag( dag_id='dag_without_catchup_once', - schedule_interval='@once', + schedule='@once', start_date=six_hours_ago_to_the_hour, catchup=False, ) @@ -1792,7 +1790,7 @@ def test_next_dagrun_info_timedelta_schedule_and_catchup_false(self): dag = DAG( 'test_scheduler_dagrun_once_with_timedelta_and_catchup_false', start_date=timezone.datetime(2015, 1, 1), - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), catchup=False, ) @@ -1812,7 +1810,7 @@ def test_next_dagrun_info_timedelta_schedule_and_catchup_true(self): dag = DAG( 'test_scheduler_dagrun_once_with_timedelta_and_catchup_true', start_date=timezone.datetime(2020, 5, 1), - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), catchup=True, ) @@ -1873,7 +1871,7 @@ def test_next_dagrun_after_auto_align(self): dag = DAG( dag_id='test_scheduler_auto_align_1', start_date=timezone.datetime(2016, 1, 1, 10, 10, 0), - schedule_interval="4 5 * * *", + schedule="4 5 * * *", ) EmptyOperator(task_id='dummy', dag=dag, owner='airflow') @@ -1883,7 +1881,7 @@ def test_next_dagrun_after_auto_align(self): dag = DAG( dag_id='test_scheduler_auto_align_2', start_date=timezone.datetime(2016, 1, 1, 10, 10, 0), - schedule_interval="10 10 * * *", + schedule="10 10 * * *", ) EmptyOperator(task_id='dummy', dag=dag, owner='airflow') @@ -1902,7 +1900,7 @@ def subdag(parent_dag_name, child_dag_name, args): """ dag_subdag = DAG( dag_id=f'{parent_dag_name}.{child_dag_name}', - schedule_interval="@daily", + schedule="@daily", default_args=args, ) @@ -1915,7 +1913,7 @@ def subdag(parent_dag_name, child_dag_name, args): dag_id='test_subdag_operator', start_date=datetime.datetime(2019, 1, 1), max_active_runs=1, - schedule_interval=timedelta(minutes=1), + schedule=timedelta(minutes=1), ) as dag: section_1 = SubDagOperator( task_id='section-1', @@ -1951,7 +1949,7 @@ def test_replace_outdated_access_control_actions(self): assert dag.access_control == updated_permissions def test_validate_params_on_trigger_dag(self): - dag = models.DAG('dummy-dag', schedule_interval=None, params={'param1': Param(type="string")}) + dag = models.DAG('dummy-dag', schedule=None, params={'param1': Param(type="string")}) with pytest.raises(ParamValidationError, match="No value passed and Param has no default value"): dag.create_dagrun( run_id="test_dagrun_missing_param", @@ -1959,7 +1957,7 @@ def test_validate_params_on_trigger_dag(self): execution_date=TEST_DATE, ) - dag = models.DAG('dummy-dag', schedule_interval=None, params={'param1': Param(type="string")}) + dag = models.DAG('dummy-dag', schedule=None, params={'param1': Param(type="string")}) with pytest.raises( ParamValidationError, match="Invalid input for param param1: None is not of type 'string'" ): @@ -1970,7 +1968,7 @@ def test_validate_params_on_trigger_dag(self): conf={"param1": None}, ) - dag = models.DAG('dummy-dag', schedule_interval=None, params={'param1': Param(type="string")}) + dag = models.DAG('dummy-dag', schedule=None, params={'param1': Param(type="string")}) dag.create_dagrun( run_id="test_dagrun_missing_param", state=State.RUNNING, @@ -1982,7 +1980,7 @@ def test_return_date_range_with_num_method(self): start_date = TEST_DATE delta = timedelta(days=1) - dag = models.DAG('dummy-dag', schedule_interval=delta) + dag = models.DAG('dummy-dag', schedule=delta) dag_dates = dag.date_range(start_date=start_date, num=3) assert dag_dates == [ @@ -2378,7 +2376,7 @@ def test_dag_timetable_match_schedule_interval(timetable): @pytest.mark.parametrize("schedule_interval", [None, "@once", "@daily", timedelta(days=1)]) def test_dag_schedule_interval_match_timetable(schedule_interval): - dag = DAG("my-dag", schedule_interval=schedule_interval) + dag = DAG("my-dag", schedule=schedule_interval) assert dag._check_schedule_interval_matches_timetable() @@ -2566,7 +2564,7 @@ def consumer(value): ids=["in-dag-restriction", "out-of-dag-restriction"], ) def test_iter_dagrun_infos_between(start_date, expected_infos): - dag = DAG(dag_id='test_get_dates', start_date=DEFAULT_DATE, schedule_interval="@hourly") + dag = DAG(dag_id='test_get_dates', start_date=DEFAULT_DATE, schedule="@hourly") EmptyOperator(task_id='dummy', dag=dag) iterator = dag.iter_dagrun_infos_between( @@ -2636,7 +2634,7 @@ def test_get_next_data_interval( data_interval_end, expected_data_interval, ): - dag = DAG(dag_id="test_get_next_data_interval", schedule_interval="@daily") + dag = DAG(dag_id="test_get_next_data_interval", schedule="@daily") dag_model = DagModel( dag_id="test_get_next_data_interval", next_dagrun=logical_date, diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index 9a619919d37f9..d9425f78432d8 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -398,7 +398,7 @@ def test_dag_removed_if_serialized_dag_is_removed(self, dag_maker): with dag_maker( dag_id="test_dag_removed_if_serialized_dag_is_removed", - schedule_interval=None, + schedule=None, start_date=tz.datetime(2021, 10, 12), ) as dag: EmptyOperator(task_id="task_1") diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 97bd1525fb259..25a24d6f9abe2 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -853,7 +853,7 @@ def test_emit_scheduling_delay(self, session, schedule_interval, expected): Tests that dag scheduling delay stat is set properly once running scheduled dag. dag_run.update_state() invokes the _emit_true_scheduling_delay_stats_for_finished_state method. """ - dag = DAG(dag_id='test_emit_dag_stats', start_date=DEFAULT_DATE, schedule_interval=schedule_interval) + dag = DAG(dag_id='test_emit_dag_stats', start_date=DEFAULT_DATE, schedule=schedule_interval) dag_task = EmptyOperator(task_id='dummy', dag=dag, owner='airflow') try: diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index d60ab901cac88..0ef15d3018b3d 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -1104,7 +1104,7 @@ def test_xcom_pull_after_success(self, create_task_instance): ti = create_task_instance( dag_id='test_xcom', - schedule_interval='@monthly', + schedule='@monthly', task_id='test_xcom', pool='test_xcom', ) @@ -1135,7 +1135,7 @@ def test_xcom_pull_after_deferral(self, create_task_instance, session): ti = create_task_instance( dag_id='test_xcom', - schedule_interval='@monthly', + schedule='@monthly', task_id='test_xcom', pool='test_xcom', ) @@ -1160,7 +1160,7 @@ def test_xcom_pull_different_execution_date(self, create_task_instance): ti = create_task_instance( dag_id='test_xcom', - schedule_interval='@monthly', + schedule='@monthly', task_id='test_xcom', pool='test_xcom', ) @@ -1637,7 +1637,7 @@ def _test_previous_dates_setup( dag_maker, ) -> list: dag_id = 'test_previous_dates' - with dag_maker(dag_id=dag_id, schedule_interval=schedule_interval, catchup=catchup): + with dag_maker(dag_id=dag_id, schedule=schedule_interval, catchup=catchup): task = EmptyOperator(task_id='task') def get_test_ti(execution_date: pendulum.DateTime, state: str) -> TI: @@ -1731,7 +1731,7 @@ def test_get_previous_start_date_none(self, dag_maker): """ Test that get_previous_start_date() can handle TaskInstance with no start_date. """ - with dag_maker("test_get_previous_start_date_none", schedule_interval=None) as dag: + with dag_maker("test_get_previous_start_date_none", schedule=None) as dag: task = EmptyOperator(task_id="op") day_1 = DEFAULT_DATE @@ -1764,7 +1764,7 @@ def test_pendulum_template_dates(self, create_task_instance): ti = create_task_instance( dag_id='test_pendulum_template_dates', task_id='test_pendulum_template_dates_task', - schedule_interval='0 12 * * *', + schedule='0 12 * * *', ) template_context = ti.get_template_context() @@ -1776,7 +1776,7 @@ def test_template_render(self, create_task_instance): ti = create_task_instance( dag_id="test_template_render", task_id="test_template_render_task", - schedule_interval="0 12 * * *", + schedule="0 12 * * *", ) template_context = ti.get_template_context() result = ti.task.render_template("Task: {{ dag.dag_id }} -> {{ task.task_id }}", template_context) @@ -1786,7 +1786,7 @@ def test_template_render_deprecated(self, create_task_instance): ti = create_task_instance( dag_id="test_template_render", task_id="test_template_render_task", - schedule_interval="0 12 * * *", + schedule="0 12 * * *", ) template_context = ti.get_template_context() with pytest.deprecated_call(): @@ -2008,7 +2008,7 @@ def test_handle_failure(self, create_dummy_dag, session=None): mock_on_retry_1 = mock.MagicMock() dag, task1 = create_dummy_dag( dag_id="test_handle_failure", - schedule_interval=None, + schedule=None, start_date=start_date, task_id="test_handle_failure_on_failure", with_dagrun_type=DagRunType.MANUAL, diff --git a/tests/operators/test_bash.py b/tests/operators/test_bash.py index 84a0fdd9061bc..3c79c43736ebc 100644 --- a/tests/operators/test_bash.py +++ b/tests/operators/test_bash.py @@ -62,7 +62,7 @@ def test_echo_env_variables(self, append_env, user_defined_env, expected_airflow dag = DAG( dag_id='bash_op_test', default_args={'owner': 'airflow', 'retries': 100, 'start_date': DEFAULT_DATE}, - schedule_interval='@daily', + schedule='@daily', dagrun_timeout=timedelta(minutes=60), ) diff --git a/tests/operators/test_branch_operator.py b/tests/operators/test_branch_operator.py index 8227ed9579a83..664adec8b99bb 100644 --- a/tests/operators/test_branch_operator.py +++ b/tests/operators/test_branch_operator.py @@ -54,7 +54,7 @@ def setUp(self): self.dag = DAG( 'branch_operator_test', default_args={'owner': 'airflow', 'start_date': DEFAULT_DATE}, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.branch_1 = EmptyOperator(task_id='branch_1', dag=self.dag) diff --git a/tests/operators/test_datetime.py b/tests/operators/test_datetime.py index bb1f9282a465d..58bed61ff9661 100644 --- a/tests/operators/test_datetime.py +++ b/tests/operators/test_datetime.py @@ -55,7 +55,7 @@ def setUp(self): self.dag = DAG( 'branch_datetime_operator_test', default_args={'owner': 'airflow', 'start_date': DEFAULT_DATE}, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.branch_1 = EmptyOperator(task_id='branch_1', dag=self.dag) diff --git a/tests/operators/test_email.py b/tests/operators/test_email.py index 7d6195214251f..74b4b35f690e7 100644 --- a/tests/operators/test_email.py +++ b/tests/operators/test_email.py @@ -39,7 +39,7 @@ def setUp(self): self.dag = DAG( 'test_dag', default_args={'owner': 'airflow', 'start_date': DEFAULT_DATE}, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.addCleanup(self.dag.clear) diff --git a/tests/operators/test_latest_only_operator.py b/tests/operators/test_latest_only_operator.py index 8ca3ab5c72ada..86cd6e429ba42 100644 --- a/tests/operators/test_latest_only_operator.py +++ b/tests/operators/test_latest_only_operator.py @@ -61,7 +61,7 @@ def setup_method(self): self.dag = DAG( 'test_dag', default_args={'owner': 'airflow', 'start_date': DEFAULT_DATE}, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.freezer = freeze_time(FROZEN_NOW) self.freezer.start() diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py index a925e52fa986e..ad6ec3fb7fdf9 100644 --- a/tests/operators/test_python.py +++ b/tests/operators/test_python.py @@ -388,7 +388,7 @@ def setUp(self): self.dag = DAG( 'branch_operator_test', default_args={'owner': 'airflow', 'start_date': DEFAULT_DATE}, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.branch_1 = EmptyOperator(task_id='branch_1', dag=self.dag) @@ -596,7 +596,7 @@ def setup(self): self.dag = DAG( "short_circuit_op_test", start_date=DEFAULT_DATE, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) with self.dag: @@ -843,7 +843,7 @@ def setUp(self): 'test_dag', default_args={'owner': 'airflow', 'start_date': DEFAULT_DATE}, template_searchpath=TEMPLATE_SEARCHPATH, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.dag.create_dagrun( run_type=DagRunType.MANUAL, @@ -1303,7 +1303,7 @@ def test_virtualenv_serializable_context_fields(create_task_instance): ti = create_task_instance( dag_id="test_virtualenv_serializable_context_fields", task_id="test_virtualenv_serializable_context_fields_task", - schedule_interval=None, + schedule=None, ) context = ti.get_template_context() diff --git a/tests/operators/test_trigger_dagrun.py b/tests/operators/test_trigger_dagrun.py index fb1d66b3d8686..7639fa6ef2aac 100644 --- a/tests/operators/test_trigger_dagrun.py +++ b/tests/operators/test_trigger_dagrun.py @@ -42,7 +42,7 @@ "dag = DAG(\n" 'dag_id="{dag_id}", \n' 'default_args={{"start_date": datetime(2019, 1, 1)}}, \n' - "schedule_interval=None,\n" + "schedule=None,\n" ")\n" 'task = EmptyOperator(task_id="test", dag=dag)' ).format(dag_id=TRIGGERED_DAG_ID) diff --git a/tests/operators/test_weekday.py b/tests/operators/test_weekday.py index 5185e1728cf5b..aaa4845b385d6 100644 --- a/tests/operators/test_weekday.py +++ b/tests/operators/test_weekday.py @@ -51,7 +51,7 @@ def setUp(self): self.dag = DAG( "branch_day_of_week_operator_test", start_date=DEFAULT_DATE, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.branch_1 = EmptyOperator(task_id="branch_1", dag=self.dag) self.branch_2 = EmptyOperator(task_id="branch_2", dag=self.dag) diff --git a/tests/providers/amazon/aws/operators/test_appflow.py b/tests/providers/amazon/aws/operators/test_appflow.py index 7bbd515693541..4cceb7395f122 100644 --- a/tests/providers/amazon/aws/operators/test_appflow.py +++ b/tests/providers/amazon/aws/operators/test_appflow.py @@ -48,7 +48,7 @@ def ctx(create_task_instance): ti = create_task_instance( dag_id=DAG_ID, task_id=TASK_ID, - schedule_interval="0 12 * * *", + schedule="0 12 * * *", ) yield {"task_instance": ti} diff --git a/tests/providers/amazon/aws/operators/test_athena.py b/tests/providers/amazon/aws/operators/test_athena.py index 2ec96495e6537..9da242121bc71 100644 --- a/tests/providers/amazon/aws/operators/test_athena.py +++ b/tests/providers/amazon/aws/operators/test_athena.py @@ -50,7 +50,7 @@ def setUp(self): 'start_date': DEFAULT_DATE, } - self.dag = DAG(f'{TEST_DAG_ID}test_schedule_dag_once', default_args=args, schedule_interval='@once') + self.dag = DAG(f'{TEST_DAG_ID}test_schedule_dag_once', default_args=args, schedule='@once') self.athena = AthenaOperator( task_id='test_athena_operator', diff --git a/tests/providers/amazon/aws/operators/test_datasync.py b/tests/providers/amazon/aws/operators/test_datasync.py index 30a8666cfe106..f1047bb201be6 100644 --- a/tests/providers/amazon/aws/operators/test_datasync.py +++ b/tests/providers/amazon/aws/operators/test_datasync.py @@ -82,7 +82,7 @@ def setUp(self): self.dag = DAG( TEST_DAG_ID + "test_schedule_dag_once", default_args=args, - schedule_interval="@once", + schedule="@once", ) self.client = boto3.client("datasync", region_name="us-east-1") diff --git a/tests/providers/amazon/aws/operators/test_dms_describe_tasks.py b/tests/providers/amazon/aws/operators/test_dms_describe_tasks.py index 3c7c587d46153..07d357a0af5cf 100644 --- a/tests/providers/amazon/aws/operators/test_dms_describe_tasks.py +++ b/tests/providers/amazon/aws/operators/test_dms_describe_tasks.py @@ -57,7 +57,7 @@ def setUp(self): "start_date": DEFAULT_DATE, } - self.dag = DAG(f"{TEST_DAG_ID}test_schedule_dag_once", default_args=args, schedule_interval="@once") + self.dag = DAG(f"{TEST_DAG_ID}test_schedule_dag_once", default_args=args, schedule="@once") def test_init(self): dms_operator = DmsDescribeTasksOperator( diff --git a/tests/providers/amazon/aws/sensors/test_s3_keys_unchanged.py b/tests/providers/amazon/aws/sensors/test_s3_keys_unchanged.py index f7767f81e0aeb..8eb6064295a6d 100644 --- a/tests/providers/amazon/aws/sensors/test_s3_keys_unchanged.py +++ b/tests/providers/amazon/aws/sensors/test_s3_keys_unchanged.py @@ -32,9 +32,7 @@ class TestS3KeysUnchangedSensor(TestCase): def setUp(self): - self.dag = DAG( - f'{TEST_DAG_ID}test_schedule_dag_once', start_date=DEFAULT_DATE, schedule_interval="@once" - ) + self.dag = DAG(f'{TEST_DAG_ID}test_schedule_dag_once', start_date=DEFAULT_DATE, schedule="@once") self.sensor = S3KeysUnchangedSensor( task_id='sensor_1', diff --git a/tests/providers/amazon/aws/transfers/test_s3_to_sftp.py b/tests/providers/amazon/aws/transfers/test_s3_to_sftp.py index f7e8b5f4f4971..3bb2de77b7fcd 100644 --- a/tests/providers/amazon/aws/transfers/test_s3_to_sftp.py +++ b/tests/providers/amazon/aws/transfers/test_s3_to_sftp.py @@ -52,7 +52,7 @@ def setUp(self): dag = DAG( f'{TEST_DAG_ID}test_schedule_dag_once', start_date=DEFAULT_DATE, - schedule_interval='@once', + schedule='@once', ) self.hook = hook diff --git a/tests/providers/amazon/aws/transfers/test_sftp_to_s3.py b/tests/providers/amazon/aws/transfers/test_sftp_to_s3.py index e5dcbc7254c80..5763c629b9ebd 100644 --- a/tests/providers/amazon/aws/transfers/test_sftp_to_s3.py +++ b/tests/providers/amazon/aws/transfers/test_sftp_to_s3.py @@ -52,7 +52,7 @@ def setUp(self): hook.no_host_key_check = True dag = DAG( f'{TEST_DAG_ID}test_schedule_dag_once', - schedule_interval="@once", + schedule="@once", start_date=DEFAULT_DATE, ) diff --git a/tests/providers/common/sql/operators/test_sql.py b/tests/providers/common/sql/operators/test_sql.py index c6dc3d075772f..37876689ce6aa 100644 --- a/tests/providers/common/sql/operators/test_sql.py +++ b/tests/providers/common/sql/operators/test_sql.py @@ -517,7 +517,7 @@ def setUp(self): self.dag = DAG( "sql_branch_operator_test", default_args={"owner": "airflow", "start_date": DEFAULT_DATE}, - schedule_interval=INTERVAL, + schedule=INTERVAL, ) self.branch_1 = EmptyOperator(task_id="branch_1", dag=self.dag) self.branch_2 = EmptyOperator(task_id="branch_2", dag=self.dag) diff --git a/tests/providers/google/cloud/operators/test_mlengine.py b/tests/providers/google/cloud/operators/test_mlengine.py index 36cd5818f9ee8..687b932e3a101 100644 --- a/tests/providers/google/cloud/operators/test_mlengine.py +++ b/tests/providers/google/cloud/operators/test_mlengine.py @@ -100,7 +100,7 @@ def setUp(self): 'start_date': DEFAULT_DATE, 'end_date': DEFAULT_DATE, }, - schedule_interval='@daily', + schedule='@daily', ) @patch('airflow.providers.google.cloud.operators.mlengine.MLEngineHook') diff --git a/tests/providers/google/cloud/operators/test_mlengine_utils.py b/tests/providers/google/cloud/operators/test_mlengine_utils.py index bd0c268a80038..e802b6a9f59d1 100644 --- a/tests/providers/google/cloud/operators/test_mlengine_utils.py +++ b/tests/providers/google/cloud/operators/test_mlengine_utils.py @@ -63,7 +63,7 @@ def setUp(self): 'model_name': 'test_model', 'version_name': 'test_version', }, - schedule_interval='@daily', + schedule='@daily', ) self.metric_fn = lambda x: (0.1,) self.metric_fn_encoded = mlengine_operator_utils.base64.b64encode( @@ -145,7 +145,7 @@ def create_test_dag(dag_id): 'project_id': 'test-project', 'region': 'us-east1', }, - schedule_interval='@daily', + schedule='@daily', ) return dag diff --git a/tests/providers/google/cloud/sensors/test_gcs.py b/tests/providers/google/cloud/sensors/test_gcs.py index 47ac70b828851..3ffd01a722fdd 100644 --- a/tests/providers/google/cloud/sensors/test_gcs.py +++ b/tests/providers/google/cloud/sensors/test_gcs.py @@ -88,14 +88,14 @@ def test_should_pass_argument_to_hook(self, mock_hook): class TestTsFunction(TestCase): def test_should_support_datetime(self): context = { - 'dag': DAG(dag_id=TEST_DAG_ID, schedule_interval=timedelta(days=5)), + 'dag': DAG(dag_id=TEST_DAG_ID, schedule=timedelta(days=5)), 'execution_date': datetime(2019, 2, 14, 0, 0), } result = ts_function(context) assert datetime(2019, 2, 19, 0, 0, tzinfo=timezone.utc) == result def test_should_support_cron(self): - dag = DAG(dag_id=TEST_DAG_ID, start_date=datetime(2019, 2, 19, 0, 0), schedule_interval='@weekly') + dag = DAG(dag_id=TEST_DAG_ID, start_date=datetime(2019, 2, 19, 0, 0), schedule='@weekly') context = { 'dag': dag, @@ -203,7 +203,7 @@ class TestGCSUploadSessionCompleteSensor(TestCase): def setUp(self): self.dag = DAG( TEST_DAG_ID + 'test_schedule_dag_once', - schedule_interval="@once", + schedule="@once", start_date=DEFAULT_DATE, ) diff --git a/tests/providers/google/marketing_platform/operators/test_campaign_manager.py b/tests/providers/google/marketing_platform/operators/test_campaign_manager.py index 5f57567f967b3..283f68b827b1e 100644 --- a/tests/providers/google/marketing_platform/operators/test_campaign_manager.py +++ b/tests/providers/google/marketing_platform/operators/test_campaign_manager.py @@ -175,7 +175,7 @@ def test_set_bucket_name( dag = DAG( dag_id="test_set_bucket_name", start_date=DEFAULT_DATE, - schedule_interval=None, + schedule=None, catchup=False, ) diff --git a/tests/providers/google/marketing_platform/operators/test_display_video.py b/tests/providers/google/marketing_platform/operators/test_display_video.py index 0481f2d2f5ee7..659b06d53dbbc 100644 --- a/tests/providers/google/marketing_platform/operators/test_display_video.py +++ b/tests/providers/google/marketing_platform/operators/test_display_video.py @@ -199,7 +199,7 @@ def test_set_bucket_name( dag = DAG( dag_id="test_set_bucket_name", start_date=DEFAULT_DATE, - schedule_interval=None, + schedule=None, catchup=False, ) diff --git a/tests/providers/google/marketing_platform/operators/test_search_ads.py b/tests/providers/google/marketing_platform/operators/test_search_ads.py index 068d2b0dcccd5..77db574bb4661 100644 --- a/tests/providers/google/marketing_platform/operators/test_search_ads.py +++ b/tests/providers/google/marketing_platform/operators/test_search_ads.py @@ -138,7 +138,7 @@ def test_set_bucket_name(self, test_bucket_name, hook_mock, gcs_hook_mock, tempf dag = DAG( dag_id="test_set_bucket_name", start_date=DEFAULT_DATE, - schedule_interval=None, + schedule=None, catchup=False, ) diff --git a/tests/providers/microsoft/azure/operators/test_adx.py b/tests/providers/microsoft/azure/operators/test_adx.py index e5bb2b8e495b0..9b8fd8cb549bc 100644 --- a/tests/providers/microsoft/azure/operators/test_adx.py +++ b/tests/providers/microsoft/azure/operators/test_adx.py @@ -64,7 +64,7 @@ class TestAzureDataExplorerQueryOperator(unittest.TestCase): def setUp(self): args = {'owner': 'airflow', 'start_date': DEFAULT_DATE, 'provide_context': True} - self.dag = DAG(TEST_DAG_ID + 'test_schedule_dag_once', default_args=args, schedule_interval='@once') + self.dag = DAG(TEST_DAG_ID + 'test_schedule_dag_once', default_args=args, schedule='@once') self.operator = AzureDataExplorerQueryOperator(dag=self.dag, **MOCK_DATA) def test_init(self): diff --git a/tests/providers/yandex/operators/test_yandexcloud_dataproc.py b/tests/providers/yandex/operators/test_yandexcloud_dataproc.py index 23cda00e4a887..0cb95a2dcbe4a 100644 --- a/tests/providers/yandex/operators/test_yandexcloud_dataproc.py +++ b/tests/providers/yandex/operators/test_yandexcloud_dataproc.py @@ -74,7 +74,7 @@ def setUp(self): 'start_date': datetime.datetime.today(), 'end_date': datetime.datetime.today() + datetime.timedelta(days=1), }, - schedule_interval='@daily', + schedule='@daily', ) @patch('airflow.providers.yandex.hooks.yandex.YandexCloudBaseHook._get_credentials') diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index 4dbc07ebc2df4..c1505dc13d646 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -218,7 +218,7 @@ def test_external_task_sensor_failed_states_as_success_mulitple_task_ids(self): ) def test_external_dag_sensor(self): - other_dag = DAG('other_dag', default_args=self.args, end_date=DEFAULT_DATE, schedule_interval='@once') + other_dag = DAG('other_dag', default_args=self.args, end_date=DEFAULT_DATE, schedule='@once') other_dag.create_dagrun( run_id='test', start_date=DEFAULT_DATE, execution_date=DEFAULT_DATE, state=State.SUCCESS ) @@ -231,7 +231,7 @@ def test_external_dag_sensor(self): op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) def test_external_dag_sensor_soft_fail_as_skipped(self): - other_dag = DAG('other_dag', default_args=self.args, end_date=DEFAULT_DATE, schedule_interval='@once') + other_dag = DAG('other_dag', default_args=self.args, end_date=DEFAULT_DATE, schedule='@once') other_dag.create_dagrun( run_id='test', start_date=DEFAULT_DATE, execution_date=DEFAULT_DATE, state=State.SUCCESS ) @@ -266,7 +266,7 @@ def test_external_task_sensor_fn_multiple_execution_dates(self): exit 0 """ dag_external_id = TEST_DAG_ID + '_external' - dag_external = DAG(dag_external_id, default_args=self.args, schedule_interval=timedelta(seconds=1)) + dag_external = DAG(dag_external_id, default_args=self.args, schedule=timedelta(seconds=1)) task_external_with_failure = BashOperator( task_id="task_external_with_failure", bash_command=bash_command_code, retries=0, dag=dag_external ) @@ -303,7 +303,7 @@ def test_external_task_sensor_fn_multiple_execution_dates(self): raise e dag_id = TEST_DAG_ID - dag = DAG(dag_id, default_args=self.args, schedule_interval=timedelta(minutes=1)) + dag = DAG(dag_id, default_args=self.args, schedule=timedelta(minutes=1)) task_without_failure = ExternalTaskSensor( task_id='task_without_failure', external_dag_id=dag_external_id, @@ -555,14 +555,14 @@ def dag_bag_ext(): dag_bag = DagBag(dag_folder=DEV_NULL, include_examples=False) - dag_0 = DAG("dag_0", start_date=DEFAULT_DATE, schedule_interval=None) + dag_0 = DAG("dag_0", start_date=DEFAULT_DATE, schedule=None) task_a_0 = EmptyOperator(task_id="task_a_0", dag=dag_0) task_b_0 = ExternalTaskMarker( task_id="task_b_0", external_dag_id="dag_1", external_task_id="task_a_1", recursion_depth=3, dag=dag_0 ) task_a_0 >> task_b_0 - dag_1 = DAG("dag_1", start_date=DEFAULT_DATE, schedule_interval=None) + dag_1 = DAG("dag_1", start_date=DEFAULT_DATE, schedule=None) task_a_1 = ExternalTaskSensor( task_id="task_a_1", external_dag_id=dag_0.dag_id, external_task_id=task_b_0.task_id, dag=dag_1 ) @@ -571,7 +571,7 @@ def dag_bag_ext(): ) task_a_1 >> task_b_1 - dag_2 = DAG("dag_2", start_date=DEFAULT_DATE, schedule_interval=None) + dag_2 = DAG("dag_2", start_date=DEFAULT_DATE, schedule=None) task_a_2 = ExternalTaskSensor( task_id="task_a_2", external_dag_id=dag_1.dag_id, external_task_id=task_b_1.task_id, dag=dag_2 ) @@ -580,7 +580,7 @@ def dag_bag_ext(): ) task_a_2 >> task_b_2 - dag_3 = DAG("dag_3", start_date=DEFAULT_DATE, schedule_interval=None) + dag_3 = DAG("dag_3", start_date=DEFAULT_DATE, schedule=None) task_a_3 = ExternalTaskSensor( task_id="task_a_3", external_dag_id=dag_2.dag_id, external_task_id=task_b_2.task_id, dag=dag_3 ) @@ -618,7 +618,7 @@ def dag_bag_parent_child(): day_1 = DEFAULT_DATE - with DAG("parent_dag_0", start_date=day_1, schedule_interval=None) as dag_0: + with DAG("parent_dag_0", start_date=day_1, schedule=None) as dag_0: task_0 = ExternalTaskMarker( task_id="task_0", external_dag_id="child_dag_1", @@ -627,7 +627,7 @@ def dag_bag_parent_child(): recursion_depth=3, ) - with DAG("child_dag_1", start_date=day_1, schedule_interval=None) as dag_1: + with DAG("child_dag_1", start_date=day_1, schedule=None) as dag_1: ExternalTaskSensor( task_id="task_1", external_dag_id=dag_0.dag_id, @@ -817,7 +817,7 @@ def _factory(depth: int) -> DagBag: dags = [] - with DAG("dag_0", start_date=DEFAULT_DATE, schedule_interval=None) as dag: + with DAG("dag_0", start_date=DEFAULT_DATE, schedule=None) as dag: dags.append(dag) task_a_0 = EmptyOperator(task_id="task_a_0") task_b_0 = ExternalTaskMarker( @@ -826,7 +826,7 @@ def _factory(depth: int) -> DagBag: task_a_0 >> task_b_0 for n in range(1, depth): - with DAG(f"dag_{n}", start_date=DEFAULT_DATE, schedule_interval=None) as dag: + with DAG(f"dag_{n}", start_date=DEFAULT_DATE, schedule=None) as dag: dags.append(dag) task_a = ExternalTaskSensor( task_id=f"task_a_{n}", @@ -842,7 +842,7 @@ def _factory(depth: int) -> DagBag: task_a >> task_b # Create the last dag which loops back - with DAG(f"dag_{depth}", start_date=DEFAULT_DATE, schedule_interval=None) as dag: + with DAG(f"dag_{depth}", start_date=DEFAULT_DATE, schedule=None) as dag: dags.append(dag) task_a = ExternalTaskSensor( task_id=f"task_a_{depth}", @@ -906,8 +906,8 @@ def dag_bag_multiple(): Create a DagBag containing two DAGs, linked by multiple ExternalTaskMarker. """ dag_bag = DagBag(dag_folder=DEV_NULL, include_examples=False) - daily_dag = DAG("daily_dag", start_date=DEFAULT_DATE, schedule_interval="@daily") - agg_dag = DAG("agg_dag", start_date=DEFAULT_DATE, schedule_interval="@daily") + daily_dag = DAG("daily_dag", start_date=DEFAULT_DATE, schedule="@daily") + agg_dag = DAG("agg_dag", start_date=DEFAULT_DATE, schedule="@daily") dag_bag.bag_dag(dag=daily_dag, root_dag=daily_dag) dag_bag.bag_dag(dag=agg_dag, root_dag=agg_dag) @@ -962,7 +962,7 @@ def dag_bag_head_tail(): """ dag_bag = DagBag(dag_folder=DEV_NULL, include_examples=False) - with DAG("head_tail", start_date=DEFAULT_DATE, schedule_interval="@daily") as dag: + with DAG("head_tail", start_date=DEFAULT_DATE, schedule="@daily") as dag: head = ExternalTaskSensor( task_id='head', external_dag_id=dag.dag_id, diff --git a/tests/system/providers/airbyte/example_airbyte_trigger_job.py b/tests/system/providers/airbyte/example_airbyte_trigger_job.py index f7b9bdf7002e8..e57c573ebe5b5 100644 --- a/tests/system/providers/airbyte/example_airbyte_trigger_job.py +++ b/tests/system/providers/airbyte/example_airbyte_trigger_job.py @@ -31,7 +31,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=['example'], diff --git a/tests/system/providers/amazon/aws/example_athena.py b/tests/system/providers/amazon/aws/example_athena.py index 2a9a4d16a5267..1890c2176a103 100644 --- a/tests/system/providers/amazon/aws/example_athena.py +++ b/tests/system/providers/amazon/aws/example_athena.py @@ -75,7 +75,7 @@ def read_results_from_s3(query_execution_id): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_batch.py b/tests/system/providers/amazon/aws/example_batch.py index 9cf91b347536b..0f381ff1c1dc9 100644 --- a/tests/system/providers/amazon/aws/example_batch.py +++ b/tests/system/providers/amazon/aws/example_batch.py @@ -134,7 +134,7 @@ def delete_job_queue(job_queue_name): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_cloudformation.py b/tests/system/providers/amazon/aws/example_cloudformation.py index 5a17e79246eec..c7368cc42df13 100644 --- a/tests/system/providers/amazon/aws/example_cloudformation.py +++ b/tests/system/providers/amazon/aws/example_cloudformation.py @@ -52,7 +52,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_datasync.py b/tests/system/providers/amazon/aws/example_datasync.py index ac1e791f069f9..40fe217d71ce8 100644 --- a/tests/system/providers/amazon/aws/example_datasync.py +++ b/tests/system/providers/amazon/aws/example_datasync.py @@ -117,7 +117,7 @@ def delete_locations(locations): with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/amazon/aws/example_ecs_fargate.py b/tests/system/providers/amazon/aws/example_ecs_fargate.py index e365c934ad745..a0ac686a85301 100644 --- a/tests/system/providers/amazon/aws/example_ecs_fargate.py +++ b/tests/system/providers/amazon/aws/example_ecs_fargate.py @@ -83,7 +83,7 @@ def delete_cluster(cluster_name: str) -> None: with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_glue.py b/tests/system/providers/amazon/aws/example_glue.py index cf6ad57d8c1ae..50ed371bd5cbb 100644 --- a/tests/system/providers/amazon/aws/example_glue.py +++ b/tests/system/providers/amazon/aws/example_glue.py @@ -118,7 +118,7 @@ def set_up(env_id, role_arn): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_lambda.py b/tests/system/providers/amazon/aws/example_lambda.py index 0b8f2805f0e0c..795a6ed69ca8a 100644 --- a/tests/system/providers/amazon/aws/example_lambda.py +++ b/tests/system/providers/amazon/aws/example_lambda.py @@ -94,7 +94,7 @@ def delete_logs(function_name: str) -> None: with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_rds_event.py b/tests/system/providers/amazon/aws/example_rds_event.py index cb19cce09b755..bbb24037a207f 100644 --- a/tests/system/providers/amazon/aws/example_rds_event.py +++ b/tests/system/providers/amazon/aws/example_rds_event.py @@ -77,7 +77,7 @@ def delete_sns_topic(topic_arn) -> None: with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_rds_export.py b/tests/system/providers/amazon/aws/example_rds_export.py index baed5a4bce25a..b3a23567e8a95 100644 --- a/tests/system/providers/amazon/aws/example_rds_export.py +++ b/tests/system/providers/amazon/aws/example_rds_export.py @@ -81,7 +81,7 @@ def delete_rds_instance(instance_name) -> None: with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_rds_snapshot.py b/tests/system/providers/amazon/aws/example_rds_snapshot.py index 2bde55287b979..eb634a7099136 100644 --- a/tests/system/providers/amazon/aws/example_rds_snapshot.py +++ b/tests/system/providers/amazon/aws/example_rds_snapshot.py @@ -67,7 +67,7 @@ def delete_rds_instance(instance_name) -> None: with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_redshift_cluster.py b/tests/system/providers/amazon/aws/example_redshift_cluster.py index ed0e2219afe26..90d2092e8b41a 100644 --- a/tests/system/providers/amazon/aws/example_redshift_cluster.py +++ b/tests/system/providers/amazon/aws/example_redshift_cluster.py @@ -38,7 +38,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/amazon/aws/example_redshift_data_execute_sql.py b/tests/system/providers/amazon/aws/example_redshift_data_execute_sql.py index d5ddb6e21312f..b2a97aef05edf 100644 --- a/tests/system/providers/amazon/aws/example_redshift_data_execute_sql.py +++ b/tests/system/providers/amazon/aws/example_redshift_data_execute_sql.py @@ -55,7 +55,7 @@ def output_query_results(statement_id): with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/amazon/aws/example_redshift_sql.py b/tests/system/providers/amazon/aws/example_redshift_sql.py index 17e386c7de27c..ec95c840eff54 100644 --- a/tests/system/providers/amazon/aws/example_redshift_sql.py +++ b/tests/system/providers/amazon/aws/example_redshift_sql.py @@ -30,7 +30,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/amazon/aws/example_redshift_to_s3.py b/tests/system/providers/amazon/aws/example_redshift_to_s3.py index a7f50e57712c6..30accd1c53b96 100644 --- a/tests/system/providers/amazon/aws/example_redshift_to_s3.py +++ b/tests/system/providers/amazon/aws/example_redshift_to_s3.py @@ -31,7 +31,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/amazon/aws/example_sagemaker.py b/tests/system/providers/amazon/aws/example_sagemaker.py index 1b729e7a24734..22d786043fc9f 100644 --- a/tests/system/providers/amazon/aws/example_sagemaker.py +++ b/tests/system/providers/amazon/aws/example_sagemaker.py @@ -389,7 +389,7 @@ def delete_logs(env_id): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py b/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py index e0b4f5b499ce3..5733a87a92e76 100644 --- a/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py +++ b/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py @@ -182,7 +182,7 @@ def set_up(env_id, knn_image_uri, role_arn, ti=None): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_sns.py b/tests/system/providers/amazon/aws/example_sns.py index 3231184e12298..b296520f9fb6e 100644 --- a/tests/system/providers/amazon/aws/example_sns.py +++ b/tests/system/providers/amazon/aws/example_sns.py @@ -43,7 +43,7 @@ def delete_topic(topic_arn) -> None: with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_sqs.py b/tests/system/providers/amazon/aws/example_sqs.py index 0aabe9e118608..6a640809ee84a 100644 --- a/tests/system/providers/amazon/aws/example_sqs.py +++ b/tests/system/providers/amazon/aws/example_sqs.py @@ -43,7 +43,7 @@ def delete_queue(queue_url): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/example_step_functions.py b/tests/system/providers/amazon/aws/example_step_functions.py index 98d6b7a7436b4..fc0270b5afab1 100644 --- a/tests/system/providers/amazon/aws/example_step_functions.py +++ b/tests/system/providers/amazon/aws/example_step_functions.py @@ -63,7 +63,7 @@ def delete_state_machine(state_machine_arn): with DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/amazon/aws/rds/example_rds_instance.py b/tests/system/providers/amazon/aws/rds/example_rds_instance.py index da157dbdf914b..1d22e66f3bb7b 100644 --- a/tests/system/providers/amazon/aws/rds/example_rds_instance.py +++ b/tests/system/providers/amazon/aws/rds/example_rds_instance.py @@ -37,7 +37,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/apache/beam/example_beam.py b/tests/system/providers/apache/beam/example_beam.py index 94d34f1140c14..90496075629de 100644 --- a/tests/system/providers/apache/beam/example_beam.py +++ b/tests/system/providers/apache/beam/example_beam.py @@ -32,7 +32,7 @@ with models.DAG( "example_beam_native_java_direct_runner", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=START_DATE, catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/beam/example_beam_java_flink.py b/tests/system/providers/apache/beam/example_beam_java_flink.py index a82331e6b553f..a1f785136169c 100644 --- a/tests/system/providers/apache/beam/example_beam_java_flink.py +++ b/tests/system/providers/apache/beam/example_beam_java_flink.py @@ -32,7 +32,7 @@ with models.DAG( "example_beam_native_java_flink_runner", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=START_DATE, catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/beam/example_beam_java_spark.py b/tests/system/providers/apache/beam/example_beam_java_spark.py index ca602b30c4bd1..69c41f20c89cd 100644 --- a/tests/system/providers/apache/beam/example_beam_java_spark.py +++ b/tests/system/providers/apache/beam/example_beam_java_spark.py @@ -32,7 +32,7 @@ with models.DAG( "example_beam_native_java_spark_runner", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=START_DATE, catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/beam/example_go.py b/tests/system/providers/apache/beam/example_go.py index 768136560a5b5..36e699b491b90 100644 --- a/tests/system/providers/apache/beam/example_go.py +++ b/tests/system/providers/apache/beam/example_go.py @@ -36,7 +36,7 @@ with models.DAG( "example_beam_native_go", start_date=START_DATE, - schedule_interval="@once", + schedule="@once", catchup=False, default_args=DEFAULT_ARGS, tags=['example'], diff --git a/tests/system/providers/apache/beam/example_go_dataflow.py b/tests/system/providers/apache/beam/example_go_dataflow.py index 590318f25a41b..44aa5e6b59af1 100644 --- a/tests/system/providers/apache/beam/example_go_dataflow.py +++ b/tests/system/providers/apache/beam/example_go_dataflow.py @@ -39,7 +39,7 @@ "example_beam_native_go_dataflow_async", default_args=DEFAULT_ARGS, start_date=START_DATE, - schedule_interval="@once", + schedule="@once", catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/apache/beam/example_java_dataflow.py b/tests/system/providers/apache/beam/example_java_dataflow.py index 092dff4aafc2a..3559a6f69268f 100644 --- a/tests/system/providers/apache/beam/example_java_dataflow.py +++ b/tests/system/providers/apache/beam/example_java_dataflow.py @@ -34,7 +34,7 @@ with models.DAG( "example_beam_native_java_dataflow_runner", - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs start_date=START_DATE, catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/beam/example_python.py b/tests/system/providers/apache/beam/example_python.py index 3349e30565ed6..c0a33901be9e4 100644 --- a/tests/system/providers/apache/beam/example_python.py +++ b/tests/system/providers/apache/beam/example_python.py @@ -36,7 +36,7 @@ with models.DAG( "example_beam_native_python", start_date=START_DATE, - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs catchup=False, default_args=DEFAULT_ARGS, tags=['example'], diff --git a/tests/system/providers/apache/beam/example_python_dataflow.py b/tests/system/providers/apache/beam/example_python_dataflow.py index f119e31a66e40..88931e068a4fd 100644 --- a/tests/system/providers/apache/beam/example_python_dataflow.py +++ b/tests/system/providers/apache/beam/example_python_dataflow.py @@ -39,7 +39,7 @@ "example_beam_native_python_dataflow_async", default_args=DEFAULT_ARGS, start_date=START_DATE, - schedule_interval=None, # Override to match your needs + schedule=None, # Override to match your needs catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/apache/cassandra/example_cassandra_dag.py b/tests/system/providers/apache/cassandra/example_cassandra_dag.py index d68b5fbe03b7d..a6487b9ec2fc5 100644 --- a/tests/system/providers/apache/cassandra/example_cassandra_dag.py +++ b/tests/system/providers/apache/cassandra/example_cassandra_dag.py @@ -36,7 +36,7 @@ # [START howto_operator_cassandra_sensors] with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), default_args={'table': 'keyspace_name.table_name'}, catchup=False, diff --git a/tests/system/providers/apache/drill/example_drill_dag.py b/tests/system/providers/apache/drill/example_drill_dag.py index 9c721703013ff..8ea47e48d1aee 100644 --- a/tests/system/providers/apache/drill/example_drill_dag.py +++ b/tests/system/providers/apache/drill/example_drill_dag.py @@ -31,7 +31,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/druid/example_druid_dag.py b/tests/system/providers/apache/druid/example_druid_dag.py index 0552e10588950..c4a84ec6cd683 100644 --- a/tests/system/providers/apache/druid/example_druid_dag.py +++ b/tests/system/providers/apache/druid/example_druid_dag.py @@ -31,7 +31,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/hive/example_twitter_dag.py b/tests/system/providers/apache/hive/example_twitter_dag.py index 9e36d3a7c1402..4954303b19d82 100644 --- a/tests/system/providers/apache/hive/example_twitter_dag.py +++ b/tests/system/providers/apache/hive/example_twitter_dag.py @@ -78,7 +78,7 @@ def transfer_to_db(): 'owner': 'Ekhtiar', 'retries': 1, }, - schedule_interval="@daily", + schedule="@daily", start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/apache/kylin/example_kylin_dag.py b/tests/system/providers/apache/kylin/example_kylin_dag.py index 9b562b039b5ca..fd00db0601ea1 100644 --- a/tests/system/providers/apache/kylin/example_kylin_dag.py +++ b/tests/system/providers/apache/kylin/example_kylin_dag.py @@ -32,7 +32,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, default_args={'project': 'learn_kylin', 'cube': 'kylin_sales_cube'}, diff --git a/tests/system/providers/apache/livy/example_livy.py b/tests/system/providers/apache/livy/example_livy.py index 8420e756ec661..b57e9c2af0cd9 100644 --- a/tests/system/providers/apache/livy/example_livy.py +++ b/tests/system/providers/apache/livy/example_livy.py @@ -33,7 +33,7 @@ with DAG( dag_id=DAG_ID, default_args={'args': [10]}, - schedule_interval='@daily', + schedule='@daily', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/apache/pig/example_pig.py b/tests/system/providers/apache/pig/example_pig.py index 325f05b8eb7e1..b2ce2a5ccf3ff 100644 --- a/tests/system/providers/apache/pig/example_pig.py +++ b/tests/system/providers/apache/pig/example_pig.py @@ -29,7 +29,7 @@ with DAG( dag_id='example_pig_operator', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/apache/spark/example_spark_dag.py b/tests/system/providers/apache/spark/example_spark_dag.py index b747e9ae4daf0..934da06025ee5 100644 --- a/tests/system/providers/apache/spark/example_spark_dag.py +++ b/tests/system/providers/apache/spark/example_spark_dag.py @@ -34,7 +34,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/cncf/kubernetes/example_kubernetes.py b/tests/system/providers/cncf/kubernetes/example_kubernetes.py index d77d0889d6aac..111e6cc4af0e2 100644 --- a/tests/system/providers/cncf/kubernetes/example_kubernetes.py +++ b/tests/system/providers/cncf/kubernetes/example_kubernetes.py @@ -103,7 +103,7 @@ with DAG( dag_id='example_kubernetes_operator', - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], ) as dag: diff --git a/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py b/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py index 20cba7a9f1f7e..14b86ab39888c 100644 --- a/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py +++ b/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py @@ -49,7 +49,7 @@ DAG_ID, default_args={'max_active_runs': 1}, description='submit spark-pi as sparkApplication on kubernetes', - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/common/sql/example_sql_column_table_check.py b/tests/system/providers/common/sql/example_sql_column_table_check.py index 034060ae8b882..7d9dda2dce83f 100644 --- a/tests/system/providers/common/sql/example_sql_column_table_check.py +++ b/tests/system/providers/common/sql/example_sql_column_table_check.py @@ -35,7 +35,7 @@ description="Example DAG for SQLColumnCheckOperator and SQLTableCheckOperator.", default_args=connection_args, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, ) as dag: """ diff --git a/tests/system/providers/databricks/example_databricks.py b/tests/system/providers/databricks/example_databricks.py index 48a3ca922a0bb..1f6004624d723 100644 --- a/tests/system/providers/databricks/example_databricks.py +++ b/tests/system/providers/databricks/example_databricks.py @@ -42,7 +42,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='@daily', + schedule='@daily', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/databricks/example_databricks_repos.py b/tests/system/providers/databricks/example_databricks_repos.py index eb76abcf198ac..025e437eca2ec 100644 --- a/tests/system/providers/databricks/example_databricks_repos.py +++ b/tests/system/providers/databricks/example_databricks_repos.py @@ -36,7 +36,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='@daily', + schedule='@daily', start_date=datetime(2021, 1, 1), default_args=default_args, tags=['example'], diff --git a/tests/system/providers/databricks/example_databricks_sql.py b/tests/system/providers/databricks/example_databricks_sql.py index 33ee37c972856..c9d4efa27e723 100644 --- a/tests/system/providers/databricks/example_databricks_sql.py +++ b/tests/system/providers/databricks/example_databricks_sql.py @@ -45,7 +45,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='@daily', + schedule='@daily', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/dbt/cloud/example_dbt_cloud.py b/tests/system/providers/dbt/cloud/example_dbt_cloud.py index e03c856519374..c6445e0f78313 100644 --- a/tests/system/providers/dbt/cloud/example_dbt_cloud.py +++ b/tests/system/providers/dbt/cloud/example_dbt_cloud.py @@ -39,7 +39,7 @@ dag_id=DAG_ID, default_args={"dbt_cloud_conn_id": "dbt", "account_id": 39151}, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, catchup=False, ) as dag: begin = EmptyOperator(task_id="begin") diff --git a/tests/system/providers/dingding/example_dingding.py b/tests/system/providers/dingding/example_dingding.py index 905881358bfb3..77d42958e05ef 100644 --- a/tests/system/providers/dingding/example_dingding.py +++ b/tests/system/providers/dingding/example_dingding.py @@ -54,7 +54,7 @@ def failure_callback(context): with DAG( dag_id=DAG_ID, default_args={'retries': 3, 'on_failure_callback': failure_callback}, - schedule_interval='@once', + schedule='@once', dagrun_timeout=timedelta(minutes=60), start_date=datetime(2021, 1, 1), tags=['example'], diff --git a/tests/system/providers/docker/example_docker.py b/tests/system/providers/docker/example_docker.py index 8a00c408b7251..6fd82fb606614 100644 --- a/tests/system/providers/docker/example_docker.py +++ b/tests/system/providers/docker/example_docker.py @@ -27,7 +27,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "docker"], diff --git a/tests/system/providers/docker/example_docker_copy_data.py b/tests/system/providers/docker/example_docker_copy_data.py index 56a3e20d28611..1192cf2f7f310 100644 --- a/tests/system/providers/docker/example_docker_copy_data.py +++ b/tests/system/providers/docker/example_docker_copy_data.py @@ -39,7 +39,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "docker"], diff --git a/tests/system/providers/docker/example_docker_swarm.py b/tests/system/providers/docker/example_docker_swarm.py index d216f11b94733..569fafd9af36d 100644 --- a/tests/system/providers/docker/example_docker_swarm.py +++ b/tests/system/providers/docker/example_docker_swarm.py @@ -26,7 +26,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', "docker"], diff --git a/tests/system/providers/docker/example_taskflow_api_etl_docker_virtualenv.py b/tests/system/providers/docker/example_taskflow_api_etl_docker_virtualenv.py index dc0f56dbb66c9..0b4fc5c5ce774 100644 --- a/tests/system/providers/docker/example_taskflow_api_etl_docker_virtualenv.py +++ b/tests/system/providers/docker/example_taskflow_api_etl_docker_virtualenv.py @@ -101,7 +101,7 @@ def load(total_order_value: float): with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "docker"], diff --git a/tests/system/providers/elasticsearch/example_elasticsearch_query.py b/tests/system/providers/elasticsearch/example_elasticsearch_query.py index 6daa04eda08c0..057fe8623e1ef 100644 --- a/tests/system/providers/elasticsearch/example_elasticsearch_query.py +++ b/tests/system/providers/elasticsearch/example_elasticsearch_query.py @@ -66,7 +66,7 @@ def use_elasticsearch_hook(): with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "elasticsearch"], diff --git a/tests/system/providers/google/ads/example_ads.py b/tests/system/providers/google/ads/example_ads.py index 015bd00aee45b..5206f06d77b36 100644 --- a/tests/system/providers/google/ads/example_ads.py +++ b/tests/system/providers/google/ads/example_ads.py @@ -72,7 +72,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "ads"], diff --git a/tests/system/providers/google/cloud/automl/example_automl_nl_text_extraction.py b/tests/system/providers/google/cloud/automl/example_automl_nl_text_extraction.py index ed9f42564c560..6deb89be6dc4c 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_nl_text_extraction.py +++ b/tests/system/providers/google/cloud/automl/example_automl_nl_text_extraction.py @@ -62,7 +62,7 @@ # Example DAG for AutoML Natural Language Entities Extraction with models.DAG( DAG_ID, - schedule_interval="@once", # Override to match your needs + schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/tests/system/providers/google/cloud/automl/example_automl_vision_classification.py b/tests/system/providers/google/cloud/automl/example_automl_vision_classification.py index 8a6756223a14d..647e033216c94 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_vision_classification.py +++ b/tests/system/providers/google/cloud/automl/example_automl_vision_classification.py @@ -63,7 +63,7 @@ # Example DAG for AutoML Vision Classification with models.DAG( DAG_ID, - schedule_interval="@once", # Override to match your needs + schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/tests/system/providers/google/cloud/azure/example_azure_fileshare_to_gcs.py b/tests/system/providers/google/cloud/azure/example_azure_fileshare_to_gcs.py index 54671fb191752..21ec900f22649 100644 --- a/tests/system/providers/google/cloud/azure/example_azure_fileshare_to_gcs.py +++ b/tests/system/providers/google/cloud/azure/example_azure_fileshare_to_gcs.py @@ -41,7 +41,7 @@ 'retries': 1, 'retry_delay': timedelta(minutes=5), }, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'azure'], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py index f5e6d7f044020..97b8b024fcc40 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py @@ -40,7 +40,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py index c2320f96e6278..076ce6fd32332 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py @@ -44,7 +44,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_operations_location.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations_location.py index 391a336bfdfbd..a7bc372900c5e 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_operations_location.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations_location.py @@ -39,7 +39,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py index 4efe8f2191ea8..0133d792a9753 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py @@ -67,7 +67,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py index b201fdfd3d975..45e44343a9279 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py @@ -58,7 +58,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py index fd62dcd5ce652..8bef1d3c1befc 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py @@ -53,7 +53,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_bigquery.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_bigquery.py index b0358046c59df..0fd3b91aaf10d 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_bigquery.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_bigquery.py @@ -41,7 +41,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs.py index e5275d91ba281..7996697a1a98b 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs.py @@ -44,7 +44,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py index 18890344cfcf7..ceca1e23eb837 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py @@ -41,7 +41,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", # Override to match your needs + schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_transfer.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_transfer.py index 892955c535a62..ddbf234c9c3cd 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_transfer.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_transfer.py @@ -45,7 +45,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "bigquery"], diff --git a/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_memcached.py b/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_memcached.py index 95d844b7c29b9..5c6bdfd5cc092 100644 --- a/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_memcached.py +++ b/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_memcached.py @@ -50,7 +50,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_redis.py b/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_redis.py index c971b9a1f1766..41dd1f2a1300b 100644 --- a/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_redis.py +++ b/tests/system/providers/google/cloud/cloud_memorystore/example_cloud_memorystore_redis.py @@ -67,7 +67,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql.py b/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql.py index 3d03f8adba4ec..125be7ecb397b 100644 --- a/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql.py +++ b/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql.py @@ -141,7 +141,7 @@ with models.DAG( DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "cloud_sql"], diff --git a/tests/system/providers/google/cloud/compute_igm/example_compute_igm.py b/tests/system/providers/google/cloud/compute_igm/example_compute_igm.py index 33275f0cc9341..2816a72082f73 100644 --- a/tests/system/providers/google/cloud/compute_igm/example_compute_igm.py +++ b/tests/system/providers/google/cloud/compute_igm/example_compute_igm.py @@ -94,7 +94,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'igm'], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch.py index 2868f5ae9b849..9c02302d5253c 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch.py @@ -46,7 +46,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_persistent.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_persistent.py index 36b411ff09c39..279ba978b8216 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_persistent.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_persistent.py @@ -68,7 +68,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_generator.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_generator.py index 5210f877bbaf7..d86488f741ef1 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_generator.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_generator.py @@ -67,7 +67,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py index b3b7730a02a9e..9d2807b7a0aaf 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py @@ -71,7 +71,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_hadoop.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_hadoop.py index a14693d3c6cd3..4a63f76b96fef 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_hadoop.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_hadoop.py @@ -83,7 +83,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_hive.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_hive.py index bf8f3178a7be9..22c427e662011 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_hive.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_hive.py @@ -70,7 +70,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_pig.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_pig.py index 2e69edb4910bc..cc83ea7140496 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_pig.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_pig.py @@ -68,7 +68,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_pyspark.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_pyspark.py index e52373977cfc2..340e8d3c00987 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_pyspark.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_pyspark.py @@ -74,7 +74,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark.py index 3b3f7b2f8663b..b3e106a9786e1 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark.py @@ -70,7 +70,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_async.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_async.py index e620ae8b2621f..9890e553b179a 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_async.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_async.py @@ -68,7 +68,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_sql.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_sql.py index 7533cc98614bc..44629741fe369 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_sql.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_sql.py @@ -66,7 +66,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_sparkr.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_sparkr.py index f6efba7057cb9..1f6261d477ddf 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_sparkr.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_sparkr.py @@ -73,7 +73,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_update.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_update.py index 590cf15ee127e..ea243e49fa03f 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_update.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_update.py @@ -68,7 +68,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow.py index c9273ac15570f..2d6f3a5a9c596 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow.py @@ -63,7 +63,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "dataproc"], diff --git a/tests/system/providers/google/cloud/datastore/example_datastore_commit.py b/tests/system/providers/google/cloud/datastore/example_datastore_commit.py index c00847fa2c599..8d890b811e808 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_commit.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_commit.py @@ -52,7 +52,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["datastore", "example"], diff --git a/tests/system/providers/google/cloud/datastore/example_datastore_export_import.py b/tests/system/providers/google/cloud/datastore/example_datastore_export_import.py index ab791b8c3b23d..81fbca37642af 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_export_import.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_export_import.py @@ -43,7 +43,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["datastore", "example"], diff --git a/tests/system/providers/google/cloud/datastore/example_datastore_query.py b/tests/system/providers/google/cloud/datastore/example_datastore_query.py index d42255034f55a..4526040cfdce9 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_query.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_query.py @@ -48,7 +48,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["datastore", "example"], diff --git a/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py b/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py index 2fc102f6bb6d8..32a1a017fcbcf 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py @@ -40,7 +40,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["datastore", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_calendar_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_calendar_to_gcs.py index 7f6a058b2f8b7..5afc02e253ba2 100644 --- a/tests/system/providers/google/cloud/gcs/example_calendar_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_calendar_to_gcs.py @@ -34,7 +34,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "calendar"], diff --git a/tests/system/providers/google/cloud/gcs/example_firestore.py b/tests/system/providers/google/cloud/gcs/example_firestore.py index 088bf5fc8751d..48215eea25702 100644 --- a/tests/system/providers/google/cloud/gcs/example_firestore.py +++ b/tests/system/providers/google/cloud/gcs/example_firestore.py @@ -76,7 +76,7 @@ with models.DAG( DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval='@once', + schedule='@once', catchup=False, tags=["example", "firestore"], ) as dag: diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_acl.py b/tests/system/providers/google/cloud/gcs/example_gcs_acl.py index 83783abbfd497..c9f4f6966d2fb 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_acl.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_acl.py @@ -49,7 +49,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "acl", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_copy_delete.py b/tests/system/providers/google/cloud/gcs/example_gcs_copy_delete.py index b00c947def291..bc14fd5e7f7f7 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_copy_delete.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_copy_delete.py @@ -50,7 +50,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py b/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py index 90c6016db9b71..869cd59c1af7e 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py @@ -48,7 +48,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery.py index 52822e38ebdf5..5f8a1d7e057a2 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery.py @@ -40,7 +40,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', "gcs"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py index f067c29ef74d9..fd9748ee89944 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py @@ -47,7 +47,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_to_gdrive.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_gdrive.py index 06773adecc7b8..2fe0944ed0611 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_to_gdrive.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_to_gdrive.py @@ -31,7 +31,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'gcs'], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_to_sheets.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_sheets.py index 5c7f2109d2f12..7179d8531db57 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_to_sheets.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_to_sheets.py @@ -36,7 +36,7 @@ with models.DAG( DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs catchup=False, tags=["example", "gcs"], ) as dag: diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_transform.py b/tests/system/providers/google/cloud/gcs/example_gcs_transform.py index 75bef5113aaf3..09390ff718f99 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_transform.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_transform.py @@ -47,7 +47,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_transform_timespan.py b/tests/system/providers/google/cloud/gcs/example_gcs_transform_timespan.py index 40f2eeafa059c..2e1f45b626d0b 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_transform_timespan.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_transform_timespan.py @@ -53,7 +53,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_upload_download.py b/tests/system/providers/google/cloud/gcs/example_gcs_upload_download.py index 98934aaea43e4..79ded8734642a 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_upload_download.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_upload_download.py @@ -42,7 +42,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=["gcs", "example"], diff --git a/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py index d6819c8e08e7e..987d57f69a921 100644 --- a/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py @@ -34,7 +34,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'mssql'], diff --git a/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py index a891a72a3b1e9..42b7c157382db 100644 --- a/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py @@ -34,7 +34,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'mysql'], diff --git a/tests/system/providers/google/cloud/gcs/example_oracle_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_oracle_to_gcs.py index 5c32b1b400c30..9f640933fb730 100644 --- a/tests/system/providers/google/cloud/gcs/example_oracle_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_oracle_to_gcs.py @@ -33,7 +33,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'oracle'], diff --git a/tests/system/providers/google/cloud/gcs/example_s3_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_s3_to_gcs.py index 9242f8f0e6660..582e4797cdc7c 100644 --- a/tests/system/providers/google/cloud/gcs/example_s3_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_s3_to_gcs.py @@ -45,7 +45,7 @@ def upload_file(): with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 's3'], diff --git a/tests/system/providers/google/cloud/gcs/example_sheets.py b/tests/system/providers/google/cloud/gcs/example_sheets.py index c17543d4f8d6f..415c50725dd5a 100644 --- a/tests/system/providers/google/cloud/gcs/example_sheets.py +++ b/tests/system/providers/google/cloud/gcs/example_sheets.py @@ -42,7 +42,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "sheets"], diff --git a/tests/system/providers/google/cloud/gcs/example_sheets_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_sheets_to_gcs.py index 310131717df53..bcdcd3b5533d0 100644 --- a/tests/system/providers/google/cloud/gcs/example_sheets_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_sheets_to_gcs.py @@ -33,7 +33,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "sheets"], diff --git a/tests/system/providers/google/cloud/gcs/example_trino_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_trino_to_gcs.py index 4f1021a9dda02..54145ce0b1f32 100644 --- a/tests/system/providers/google/cloud/gcs/example_trino_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_trino_to_gcs.py @@ -52,7 +52,7 @@ def safe_name(s: str) -> str: with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "gcs"], diff --git a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py index 945c31d3cdcbf..9a7e43a0c7ba6 100644 --- a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py +++ b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py @@ -43,7 +43,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/google/cloud/life_sciences/example_life_sciences.py b/tests/system/providers/google/cloud/life_sciences/example_life_sciences.py index 6297def813818..c56ea87f9cbdf 100644 --- a/tests/system/providers/google/cloud/life_sciences/example_life_sciences.py +++ b/tests/system/providers/google/cloud/life_sciences/example_life_sciences.py @@ -78,7 +78,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/google/cloud/natural_language/example_natural_language.py b/tests/system/providers/google/cloud/natural_language/example_natural_language.py index fe79c25dd6941..ef8295146c2b0 100644 --- a/tests/system/providers/google/cloud/natural_language/example_natural_language.py +++ b/tests/system/providers/google/cloud/natural_language/example_natural_language.py @@ -55,7 +55,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/google/cloud/pubsub/example_pubsub.py b/tests/system/providers/google/cloud/pubsub/example_pubsub.py index 335509e691bd1..6e4c6d765857a 100644 --- a/tests/system/providers/google/cloud/pubsub/example_pubsub.py +++ b/tests/system/providers/google/cloud/pubsub/example_pubsub.py @@ -54,7 +54,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/google/cloud/spanner/example_spanner.py b/tests/system/providers/google/cloud/spanner/example_spanner.py index ccea398017c0f..7d01aa170025f 100644 --- a/tests/system/providers/google/cloud/spanner/example_spanner.py +++ b/tests/system/providers/google/cloud/spanner/example_spanner.py @@ -50,7 +50,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'spanner'], diff --git a/tests/system/providers/google/cloud/speech_to_text/example_speech_to_text.py b/tests/system/providers/google/cloud/speech_to_text/example_speech_to_text.py index c4e4679c4f764..65231a860dd1c 100644 --- a/tests/system/providers/google/cloud/speech_to_text/example_speech_to_text.py +++ b/tests/system/providers/google/cloud/speech_to_text/example_speech_to_text.py @@ -48,7 +48,7 @@ with models.DAG( DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "speech_to_text"], diff --git a/tests/system/providers/google/cloud/sql_to_sheets/example_sql_to_sheets.py b/tests/system/providers/google/cloud/sql_to_sheets/example_sql_to_sheets.py index d4c850353f98a..b99bf80bb44fd 100644 --- a/tests/system/providers/google/cloud/sql_to_sheets/example_sql_to_sheets.py +++ b/tests/system/providers/google/cloud/sql_to_sheets/example_sql_to_sheets.py @@ -30,7 +30,7 @@ with models.DAG( DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs catchup=False, tags=["example", "sql"], ) as dag: diff --git a/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py b/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py index b259b68523145..61eba33ebe4f2 100644 --- a/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py +++ b/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py @@ -125,7 +125,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', "stackdriver"], diff --git a/tests/system/providers/google/cloud/tasks/example_queue.py b/tests/system/providers/google/cloud/tasks/example_queue.py index 61a83df4c6b42..863c498bd91f0 100644 --- a/tests/system/providers/google/cloud/tasks/example_queue.py +++ b/tests/system/providers/google/cloud/tasks/example_queue.py @@ -52,7 +52,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', "tasks"], diff --git a/tests/system/providers/google/cloud/tasks/example_tasks.py b/tests/system/providers/google/cloud/tasks/example_tasks.py index 3b2cf46c1e188..a31a3a0ac25eb 100644 --- a/tests/system/providers/google/cloud/tasks/example_tasks.py +++ b/tests/system/providers/google/cloud/tasks/example_tasks.py @@ -64,7 +64,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', "tasks"], diff --git a/tests/system/providers/google/cloud/text_to_speech/example_text_to_speech.py b/tests/system/providers/google/cloud/text_to_speech/example_text_to_speech.py index 83bbdab7c0380..d609fde9e55b5 100644 --- a/tests/system/providers/google/cloud/text_to_speech/example_text_to_speech.py +++ b/tests/system/providers/google/cloud/text_to_speech/example_text_to_speech.py @@ -42,7 +42,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "text_to_speech"], diff --git a/tests/system/providers/google/cloud/transfers/example_gcs_to_sftp.py b/tests/system/providers/google/cloud/transfers/example_gcs_to_sftp.py index c8b2d2dfafe7b..ba91f1c5275d6 100644 --- a/tests/system/providers/google/cloud/transfers/example_gcs_to_sftp.py +++ b/tests/system/providers/google/cloud/transfers/example_gcs_to_sftp.py @@ -41,7 +41,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, tags=['example', 'gcs'], diff --git a/tests/system/providers/google/cloud/translate/example_translate.py b/tests/system/providers/google/cloud/translate/example_translate.py index e458372e97e4c..fe14e9d92819d 100644 --- a/tests/system/providers/google/cloud/translate/example_translate.py +++ b/tests/system/providers/google/cloud/translate/example_translate.py @@ -31,7 +31,7 @@ with models.DAG( DAG_ID, - schedule_interval="@once", # Override to match your needs + schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/google/cloud/translate_speech/example_translate_speech.py b/tests/system/providers/google/cloud/translate_speech/example_translate_speech.py index 7af801e9228bb..d8aac3794c622 100644 --- a/tests/system/providers/google/cloud/translate_speech/example_translate_speech.py +++ b/tests/system/providers/google/cloud/translate_speech/example_translate_speech.py @@ -54,7 +54,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/google/cloud/workflows/example_workflows.py b/tests/system/providers/google/cloud/workflows/example_workflows.py index 47ca818bf830e..567b04de8d250 100644 --- a/tests/system/providers/google/cloud/workflows/example_workflows.py +++ b/tests/system/providers/google/cloud/workflows/example_workflows.py @@ -88,7 +88,7 @@ with DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/google/datacatalog/example_datacatalog_entries.py b/tests/system/providers/google/datacatalog/example_datacatalog_entries.py index cc7430452b0bd..18ce63eb6e610 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_entries.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_entries.py @@ -50,7 +50,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/google/datacatalog/example_datacatalog_search_catalog.py b/tests/system/providers/google/datacatalog/example_datacatalog_search_catalog.py index e12ee63c9bdcc..7e93bb6bf4130 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_search_catalog.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_search_catalog.py @@ -54,7 +54,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/google/datacatalog/example_datacatalog_tag_templates.py b/tests/system/providers/google/datacatalog/example_datacatalog_tag_templates.py index 291af290da795..ab5357bb51003 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_tag_templates.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_tag_templates.py @@ -49,7 +49,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/google/datacatalog/example_datacatalog_tags.py b/tests/system/providers/google/datacatalog/example_datacatalog_tags.py index 20eeb3895a240..277087ae07916 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_tags.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_tags.py @@ -55,7 +55,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', + schedule='@once', start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/google/leveldb/example_leveldb.py b/tests/system/providers/google/leveldb/example_leveldb.py index cc9f6519b98c0..136eb45ad25b5 100644 --- a/tests/system/providers/google/leveldb/example_leveldb.py +++ b/tests/system/providers/google/leveldb/example_leveldb.py @@ -32,7 +32,7 @@ with models.DAG( DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval='@once', + schedule='@once', catchup=False, tags=['example'], ) as dag: diff --git a/tests/system/providers/google/marketing_platform/example_analytics.py b/tests/system/providers/google/marketing_platform/example_analytics.py index b5de17148d358..f5fbabab8044c 100644 --- a/tests/system/providers/google/marketing_platform/example_analytics.py +++ b/tests/system/providers/google/marketing_platform/example_analytics.py @@ -43,7 +43,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "analytics"], diff --git a/tests/system/providers/google/marketing_platform/example_campaign_manager.py b/tests/system/providers/google/marketing_platform/example_campaign_manager.py index bd02ba3bc5f84..23ac650908602 100644 --- a/tests/system/providers/google/marketing_platform/example_campaign_manager.py +++ b/tests/system/providers/google/marketing_platform/example_campaign_manager.py @@ -92,7 +92,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=datetime(2021, 1, 1), catchup=False, tags=["example", "campaign"], diff --git a/tests/system/providers/google/marketing_platform/example_search_ads.py b/tests/system/providers/google/marketing_platform/example_search_ads.py index 6693d5bc47849..ac5d98b98e4ad 100644 --- a/tests/system/providers/google/marketing_platform/example_search_ads.py +++ b/tests/system/providers/google/marketing_platform/example_search_ads.py @@ -49,7 +49,7 @@ with models.DAG( DAG_ID, - schedule_interval='@once', # Override to match your needs, + schedule='@once', # Override to match your needs, start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/influxdb/example_influxdb.py b/tests/system/providers/influxdb/example_influxdb.py index f6f89553df814..9bd97f5fda417 100644 --- a/tests/system/providers/influxdb/example_influxdb.py +++ b/tests/system/providers/influxdb/example_influxdb.py @@ -53,7 +53,7 @@ def test_influxdb_hook(): with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), max_active_runs=1, tags=['example'], diff --git a/tests/system/providers/jdbc/example_jdbc_queries.py b/tests/system/providers/jdbc/example_jdbc_queries.py index 2bb5e088dbbbb..7036ffec78071 100644 --- a/tests/system/providers/jdbc/example_jdbc_queries.py +++ b/tests/system/providers/jdbc/example_jdbc_queries.py @@ -34,7 +34,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', start_date=datetime(2021, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=['example'], diff --git a/tests/system/providers/jenkins/example_jenkins_job_trigger.py b/tests/system/providers/jenkins/example_jenkins_job_trigger.py index 248beafe55a14..321cc9f5f17c4 100644 --- a/tests/system/providers/jenkins/example_jenkins_job_trigger.py +++ b/tests/system/providers/jenkins/example_jenkins_job_trigger.py @@ -37,7 +37,7 @@ "max_active_runs": 8, }, start_date=datetime(2017, 6, 1), - schedule_interval=None, + schedule=None, ) as dag: job_trigger = JenkinsJobTriggerOperator( task_id="trigger_job", diff --git a/tests/system/providers/microsoft/azure/example_adf_run_pipeline.py b/tests/system/providers/microsoft/azure/example_adf_run_pipeline.py index 03a1a71d63313..6c8eb3757fed2 100644 --- a/tests/system/providers/microsoft/azure/example_adf_run_pipeline.py +++ b/tests/system/providers/microsoft/azure/example_adf_run_pipeline.py @@ -34,7 +34,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 8, 13), - schedule_interval="@daily", + schedule="@daily", catchup=False, default_args={ "retries": 1, diff --git a/tests/system/providers/microsoft/azure/example_adls_delete.py b/tests/system/providers/microsoft/azure/example_adls_delete.py index e592160c1d03d..dc3fe06e7e831 100644 --- a/tests/system/providers/microsoft/azure/example_adls_delete.py +++ b/tests/system/providers/microsoft/azure/example_adls_delete.py @@ -31,7 +31,7 @@ with models.DAG( DAG_ID, start_date=datetime(2021, 1, 1), - schedule_interval=None, + schedule=None, tags=['example'], ) as dag: diff --git a/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py b/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py index 00e021432c7f8..6c59ae8dbf14c 100644 --- a/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py +++ b/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py @@ -37,7 +37,7 @@ # [START how_to_azure_blob_to_gcs] with DAG( DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), # Override to match your needs default_args={"container_name": AZURE_CONTAINER_NAME, "blob_name": BLOB_NAME}, ) as dag: diff --git a/tests/system/providers/microsoft/azure/example_azure_container_instances.py b/tests/system/providers/microsoft/azure/example_azure_container_instances.py index 0f281844762e5..8afe6f5257bfd 100644 --- a/tests/system/providers/microsoft/azure/example_azure_container_instances.py +++ b/tests/system/providers/microsoft/azure/example_azure_container_instances.py @@ -30,7 +30,7 @@ with DAG( dag_id=DAG_ID, default_args={'retries': 1}, - schedule_interval=timedelta(days=1), + schedule=timedelta(days=1), start_date=datetime(2018, 11, 1), catchup=False, tags=['example'], diff --git a/tests/system/providers/microsoft/azure/example_azure_service_bus.py b/tests/system/providers/microsoft/azure/example_azure_service_bus.py index a17a3a0bee404..6295d28b723d8 100644 --- a/tests/system/providers/microsoft/azure/example_azure_service_bus.py +++ b/tests/system/providers/microsoft/azure/example_azure_service_bus.py @@ -43,7 +43,7 @@ with DAG( dag_id="example_azure_service_bus", start_date=datetime(2021, 8, 13), - schedule_interval=None, + schedule=None, catchup=False, default_args={ "execution_timeout": timedelta(hours=EXECUTION_TIMEOUT), diff --git a/tests/system/providers/microsoft/azure/example_fileshare.py b/tests/system/providers/microsoft/azure/example_fileshare.py index f336b699a5cb7..32826d18c66d3 100644 --- a/tests/system/providers/microsoft/azure/example_fileshare.py +++ b/tests/system/providers/microsoft/azure/example_fileshare.py @@ -47,7 +47,7 @@ def delete_fileshare(): with DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/microsoft/azure/example_local_to_adls.py b/tests/system/providers/microsoft/azure/example_local_to_adls.py index 9ff15a45850b5..091e3843deeae 100644 --- a/tests/system/providers/microsoft/azure/example_local_to_adls.py +++ b/tests/system/providers/microsoft/azure/example_local_to_adls.py @@ -31,7 +31,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval=None, + schedule=None, tags=['example'], ) as dag: # [START howto_operator_local_to_adls] diff --git a/tests/system/providers/microsoft/azure/example_local_to_wasb.py b/tests/system/providers/microsoft/azure/example_local_to_wasb.py index df7ca9b4dd302..7b532a91a915d 100644 --- a/tests/system/providers/microsoft/azure/example_local_to_wasb.py +++ b/tests/system/providers/microsoft/azure/example_local_to_wasb.py @@ -31,7 +31,7 @@ with DAG( DAG_ID, - schedule_interval="@once", + schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, default_args={"container_name": "mycontainer", "blob_name": "myblob"}, diff --git a/tests/system/providers/microsoft/azure/example_sftp_to_wasb.py b/tests/system/providers/microsoft/azure/example_sftp_to_wasb.py index a8073242aeb5c..0a032266cb726 100644 --- a/tests/system/providers/microsoft/azure/example_sftp_to_wasb.py +++ b/tests/system/providers/microsoft/azure/example_sftp_to_wasb.py @@ -44,7 +44,7 @@ def delete_sftp_file(): with DAG( DAG_ID, - schedule_interval=None, + schedule=None, catchup=False, start_date=datetime(2021, 1, 1), # Override to match your needs ) as dag: diff --git a/tests/system/providers/microsoft/mssql/example_mssql.py b/tests/system/providers/microsoft/mssql/example_mssql.py index 04b9ea0186337..464b3e1b3ada6 100644 --- a/tests/system/providers/microsoft/mssql/example_mssql.py +++ b/tests/system/providers/microsoft/mssql/example_mssql.py @@ -32,7 +32,7 @@ with DAG( DAG_ID, - schedule_interval='@daily', + schedule='@daily', start_date=datetime(2021, 10, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/microsoft/winrm/example_winrm.py b/tests/system/providers/microsoft/winrm/example_winrm.py index 15d8c6b031c8b..fc7e00dcbf1ef 100644 --- a/tests/system/providers/microsoft/winrm/example_winrm.py +++ b/tests/system/providers/microsoft/winrm/example_winrm.py @@ -44,7 +44,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='0 0 * * *', + schedule='0 0 * * *', start_date=datetime(2021, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=['example'], diff --git a/tests/system/providers/opsgenie/example_opsgenie_alert.py b/tests/system/providers/opsgenie/example_opsgenie_alert.py index abe645b417ae6..d40020f957699 100644 --- a/tests/system/providers/opsgenie/example_opsgenie_alert.py +++ b/tests/system/providers/opsgenie/example_opsgenie_alert.py @@ -29,7 +29,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/papermill/example_papermill.py b/tests/system/providers/papermill/example_papermill.py index 8f828e4ed11f8..d08906d9a94e4 100644 --- a/tests/system/providers/papermill/example_papermill.py +++ b/tests/system/providers/papermill/example_papermill.py @@ -34,7 +34,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=SCHEDULE_INTERVAL, + schedule=SCHEDULE_INTERVAL, start_date=START_DATE, dagrun_timeout=DAGRUN_TIMEOUT, tags=['example'], diff --git a/tests/system/providers/papermill/example_papermill_verify.py b/tests/system/providers/papermill/example_papermill_verify.py index dd24fc51f6363..c7dbda2206399 100644 --- a/tests/system/providers/papermill/example_papermill_verify.py +++ b/tests/system/providers/papermill/example_papermill_verify.py @@ -55,7 +55,7 @@ def check_notebook(inlets, execution_date): with DAG( dag_id='example_papermill_operator_verify', - schedule_interval=SCHEDULE_INTERVAL, + schedule=SCHEDULE_INTERVAL, start_date=START_DATE, dagrun_timeout=DAGRUN_TIMEOUT, catchup=False, diff --git a/tests/system/providers/plexus/example_plexus.py b/tests/system/providers/plexus/example_plexus.py index 17c7f499c0aa0..8032f70749fd6 100644 --- a/tests/system/providers/plexus/example_plexus.py +++ b/tests/system/providers/plexus/example_plexus.py @@ -31,7 +31,7 @@ default_args={'owner': 'core scientific', 'retries': 1}, description='testing plexus operator', start_date=datetime(2021, 1, 1), - schedule_interval='@once', + schedule='@once', catchup=False, ) as dag: # [START plexus_job_op] diff --git a/tests/system/providers/postgres/example_postgres.py b/tests/system/providers/postgres/example_postgres.py index b7d6c0d0a83f1..b45e8062c1853 100644 --- a/tests/system/providers/postgres/example_postgres.py +++ b/tests/system/providers/postgres/example_postgres.py @@ -32,7 +32,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime.datetime(2020, 2, 2), - schedule_interval="@once", + schedule="@once", catchup=False, ) as dag: # [START postgres_operator_howto_guide_create_pet_table] diff --git a/tests/system/providers/presto/example_gcs_to_presto.py b/tests/system/providers/presto/example_gcs_to_presto.py index 7c062e2d72536..90d081b82c19b 100644 --- a/tests/system/providers/presto/example_gcs_to_presto.py +++ b/tests/system/providers/presto/example_gcs_to_presto.py @@ -33,7 +33,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/qubole/example_qubole.py b/tests/system/providers/qubole/example_qubole.py index 89e6d9007affc..03e89a5359e84 100644 --- a/tests/system/providers/qubole/example_qubole.py +++ b/tests/system/providers/qubole/example_qubole.py @@ -41,7 +41,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=START_DATE, tags=['example'], ) as dag: diff --git a/tests/system/providers/qubole/example_qubole_sensors.py b/tests/system/providers/qubole/example_qubole_sensors.py index 2ee52276f56c1..d7dad84a753d3 100644 --- a/tests/system/providers/qubole/example_qubole_sensors.py +++ b/tests/system/providers/qubole/example_qubole_sensors.py @@ -31,7 +31,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=START_DATE, tags=['example'], ) as dag: diff --git a/tests/system/providers/salesforce/example_bulk.py b/tests/system/providers/salesforce/example_bulk.py index 8d3d3b66c923d..2f7cddd4fff77 100644 --- a/tests/system/providers/salesforce/example_bulk.py +++ b/tests/system/providers/salesforce/example_bulk.py @@ -25,7 +25,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/salesforce/example_salesforce_apex_rest.py b/tests/system/providers/salesforce/example_salesforce_apex_rest.py index 0053531e0753a..b2fd4eabb7f18 100644 --- a/tests/system/providers/salesforce/example_salesforce_apex_rest.py +++ b/tests/system/providers/salesforce/example_salesforce_apex_rest.py @@ -26,7 +26,7 @@ with DAG( dag_id="salesforce_apex_rest_operator_dag", - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/singularity/example_singularity.py b/tests/system/providers/singularity/example_singularity.py index eff80652bb5d9..7ac4fd3243859 100644 --- a/tests/system/providers/singularity/example_singularity.py +++ b/tests/system/providers/singularity/example_singularity.py @@ -29,7 +29,7 @@ with DAG( DAG_ID, default_args={'retries': 1}, - schedule_interval=timedelta(minutes=10), + schedule=timedelta(minutes=10), start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/system/providers/slack/example_slack.py b/tests/system/providers/slack/example_slack.py index 9cc847d18fca8..ddb74ac5fca34 100644 --- a/tests/system/providers/slack/example_slack.py +++ b/tests/system/providers/slack/example_slack.py @@ -27,7 +27,7 @@ # [START slack_operator_howto_guide] with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), default_args={'slack_conn_id': 'slack', 'channel': '#general', 'initial_comment': 'Hello World!'}, max_active_runs=1, diff --git a/tests/system/providers/slack/example_sql_to_slack.py b/tests/system/providers/slack/example_sql_to_slack.py index afdf3bcf62baa..5dd8fe6136d03 100644 --- a/tests/system/providers/slack/example_sql_to_slack.py +++ b/tests/system/providers/slack/example_sql_to_slack.py @@ -32,7 +32,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/sqlite/example_sqlite.py b/tests/system/providers/sqlite/example_sqlite.py index cf0c74c1936ab..c63f3e746ca68 100644 --- a/tests/system/providers/sqlite/example_sqlite.py +++ b/tests/system/providers/sqlite/example_sqlite.py @@ -35,7 +35,7 @@ with DAG( dag_id=DAG_ID, - schedule_interval='@daily', + schedule='@daily', start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/tests/system/providers/tableau/example_tableau.py b/tests/system/providers/tableau/example_tableau.py index 14f225164e915..30e6583c5199a 100644 --- a/tests/system/providers/tableau/example_tableau.py +++ b/tests/system/providers/tableau/example_tableau.py @@ -34,7 +34,7 @@ dag_id='example_tableau', default_args={'site_id': 'my_site'}, dagrun_timeout=timedelta(hours=2), - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], ) as dag: diff --git a/tests/system/providers/tableau/example_tableau_refresh_workbook.py b/tests/system/providers/tableau/example_tableau_refresh_workbook.py index cc973792c5155..999abcb73c369 100644 --- a/tests/system/providers/tableau/example_tableau_refresh_workbook.py +++ b/tests/system/providers/tableau/example_tableau_refresh_workbook.py @@ -33,7 +33,7 @@ with DAG( dag_id=DAG_ID, dagrun_timeout=timedelta(hours=2), - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), default_args={'site_id': 'my_site'}, tags=['example'], diff --git a/tests/system/providers/tabular/example_tabular.py b/tests/system/providers/tabular/example_tabular.py index 7021b0e8742ef..4cdb9a692aa7b 100644 --- a/tests/system/providers/tabular/example_tabular.py +++ b/tests/system/providers/tabular/example_tabular.py @@ -37,7 +37,7 @@ "email_on_retry": False, }, start_date=datetime(2021, 1, 1), - schedule_interval=timedelta(1), + schedule=timedelta(1), catchup=False, ) as dag: # This also works for the SparkSubmit operator diff --git a/tests/system/providers/trino/example_gcs_to_trino.py b/tests/system/providers/trino/example_gcs_to_trino.py index ae56f92691268..98ebf57812d66 100644 --- a/tests/system/providers/trino/example_gcs_to_trino.py +++ b/tests/system/providers/trino/example_gcs_to_trino.py @@ -33,7 +33,7 @@ with models.DAG( dag_id=DAG_ID, - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/trino/example_trino.py b/tests/system/providers/trino/example_trino.py index f7916d7c66b30..04374dfe094e1 100644 --- a/tests/system/providers/trino/example_trino.py +++ b/tests/system/providers/trino/example_trino.py @@ -33,7 +33,7 @@ with models.DAG( dag_id="example_trino", - schedule_interval='@once', # Override to match your needs + schedule='@once', # Override to match your needs start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/tests/system/providers/yandex/example_yandexcloud.py b/tests/system/providers/yandex/example_yandexcloud.py index 708a6049a75ad..89d28f8639d75 100644 --- a/tests/system/providers/yandex/example_yandexcloud.py +++ b/tests/system/providers/yandex/example_yandexcloud.py @@ -169,7 +169,7 @@ def delete_cluster( with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], ) as dag: diff --git a/tests/system/providers/yandex/example_yandexcloud_dataproc.py b/tests/system/providers/yandex/example_yandexcloud_dataproc.py index e8c2998adb137..8315cd20a3c11 100644 --- a/tests/system/providers/yandex/example_yandexcloud_dataproc.py +++ b/tests/system/providers/yandex/example_yandexcloud_dataproc.py @@ -44,7 +44,7 @@ with DAG( DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], ) as dag: diff --git a/tests/system/providers/yandex/example_yandexcloud_dataproc_lightweight.py b/tests/system/providers/yandex/example_yandexcloud_dataproc_lightweight.py index d5faa0865e947..24ba89d2b78ba 100644 --- a/tests/system/providers/yandex/example_yandexcloud_dataproc_lightweight.py +++ b/tests/system/providers/yandex/example_yandexcloud_dataproc_lightweight.py @@ -40,7 +40,7 @@ with DAG( DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), tags=['example'], ) as dag: diff --git a/tests/system/providers/zendesk/example_zendesk_custom_get.py b/tests/system/providers/zendesk/example_zendesk_custom_get.py index 94ea3eb1dde87..36a540a0add49 100644 --- a/tests/system/providers/zendesk/example_zendesk_custom_get.py +++ b/tests/system/providers/zendesk/example_zendesk_custom_get.py @@ -38,7 +38,7 @@ def fetch_organizations() -> List[Dict]: with DAG( dag_id=DAG_ID, - schedule_interval=None, + schedule=None, start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/tests/test_utils/perf/dags/elastic_dag.py b/tests/test_utils/perf/dags/elastic_dag.py index e958d68d521ec..fef48cfe8f5b7 100644 --- a/tests/test_utils/perf/dags/elastic_dag.py +++ b/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}", ] ) ), is_paused_upon_creation=False, default_args=args, - schedule_interval=SCHEDULE_INTERVAL, + schedule=SCHEDULE_INTERVAL, ) elastic_dag_tasks = [ diff --git a/tests/test_utils/perf/dags/perf_dag_1.py b/tests/test_utils/perf/dags/perf_dag_1.py index 4305ec4f2ec9b..5a9ecd8c202d1 100644 --- a/tests/test_utils/perf/dags/perf_dag_1.py +++ b/tests/test_utils/perf/dags/perf_dag_1.py @@ -31,7 +31,7 @@ dag = DAG( dag_id='perf_dag_1', default_args=args, - schedule_interval='@daily', + schedule='@daily', dagrun_timeout=datetime.timedelta(minutes=60), ) diff --git a/tests/test_utils/perf/dags/perf_dag_2.py b/tests/test_utils/perf/dags/perf_dag_2.py index 5b2c02486491c..0b23675b7e7a1 100644 --- a/tests/test_utils/perf/dags/perf_dag_2.py +++ b/tests/test_utils/perf/dags/perf_dag_2.py @@ -31,7 +31,7 @@ dag = DAG( dag_id='perf_dag_2', default_args=args, - schedule_interval='@daily', + schedule='@daily', dagrun_timeout=datetime.timedelta(minutes=60), ) diff --git a/tests/test_utils/perf/dags/sql_perf_dag.py b/tests/test_utils/perf/dags/sql_perf_dag.py index 46f040df52e83..a4842ac15e04a 100644 --- a/tests/test_utils/perf/dags/sql_perf_dag.py +++ b/tests/test_utils/perf/dags/sql_perf_dag.py @@ -61,7 +61,7 @@ def generate_parallel_tasks(name_prefix, num_of_tasks, deps): DAG_ID, default_args=default_args, catchup=True, - schedule_interval=timedelta(minutes=1), + schedule=timedelta(minutes=1), is_paused_upon_creation=False, ): zero_level_tasks = generate_parallel_tasks("l0", 1, []) diff --git a/tests/ti_deps/deps/test_not_previously_skipped_dep.py b/tests/ti_deps/deps/test_not_previously_skipped_dep.py index 9fd4bd546525a..9a0b086d29609 100644 --- a/tests/ti_deps/deps/test_not_previously_skipped_dep.py +++ b/tests/ti_deps/deps/test_not_previously_skipped_dep.py @@ -42,7 +42,7 @@ def test_no_parent(session, dag_maker): start_date = pendulum.datetime(2020, 1, 1) with dag_maker( "test_test_no_parent_dag", - schedule_interval=None, + schedule=None, start_date=start_date, session=session, ): @@ -64,7 +64,7 @@ def test_no_skipmixin_parent(session, dag_maker): start_date = pendulum.datetime(2020, 1, 1) with dag_maker( "test_no_skipmixin_parent_dag", - schedule_interval=None, + schedule=None, start_date=start_date, session=session, ): @@ -88,7 +88,7 @@ def test_parent_follow_branch(session, dag_maker): start_date = pendulum.datetime(2020, 1, 1) with dag_maker( "test_parent_follow_branch_dag", - schedule_interval=None, + schedule=None, start_date=start_date, session=session, ): @@ -113,7 +113,7 @@ def test_parent_skip_branch(session, dag_maker): start_date = pendulum.datetime(2020, 1, 1) with dag_maker( "test_parent_skip_branch_dag", - schedule_interval=None, + schedule=None, start_date=start_date, session=session, ): @@ -142,7 +142,7 @@ def test_parent_not_executed(session, dag_maker): start_date = pendulum.datetime(2020, 1, 1) with dag_maker( "test_parent_not_executed_dag", - schedule_interval=None, + schedule=None, start_date=start_date, session=session, ): diff --git a/tests/ti_deps/deps/test_runnable_exec_date_dep.py b/tests/ti_deps/deps/test_runnable_exec_date_dep.py index 0577e67e500e8..d9241b8812e26 100644 --- a/tests/ti_deps/deps/test_runnable_exec_date_dep.py +++ b/tests/ti_deps/deps/test_runnable_exec_date_dep.py @@ -66,7 +66,7 @@ def test_exec_date_dep( 'test_localtaskjob_heartbeat', start_date=datetime(2015, 1, 1), end_date=datetime(2016, 11, 5), - schedule_interval=schedule_interval, + schedule=schedule_interval, with_dagrun_type=DagRunType.MANUAL, session=session, ) @@ -83,7 +83,7 @@ def test_exec_date_after_end_date(session, dag_maker, create_dummy_dag): 'test_localtaskjob_heartbeat', start_date=datetime(2015, 1, 1), end_date=datetime(2016, 11, 5), - schedule_interval=None, + schedule=None, with_dagrun_type=DagRunType.MANUAL, session=session, ) diff --git a/tests/utils/log/test_log_reader.py b/tests/utils/log/test_log_reader.py index 9a76ada725d33..c2b1c6ddfd974 100644 --- a/tests/utils/log/test_log_reader.py +++ b/tests/utils/log/test_log_reader.py @@ -267,7 +267,7 @@ def test_task_log_filename_unique(self, dag_maker): def echo_run_type(dag_run: DagRun, **kwargs): print(dag_run.run_type) - with dag_maker(dag_id, start_date=self.DEFAULT_DATE, schedule_interval="@daily") as dag: + with dag_maker(dag_id, start_date=self.DEFAULT_DATE, schedule="@daily") as dag: PythonOperator(task_id=task_id, python_callable=echo_run_type) start = pendulum.datetime(2021, 1, 1) diff --git a/tests/utils/test_task_group.py b/tests/utils/test_task_group.py index 864b2fb68a45b..b8f6fa2ff2957 100644 --- a/tests/utils/test_task_group.py +++ b/tests/utils/test_task_group.py @@ -1025,7 +1025,7 @@ def add_one(i): def increment(num): return num + 1 - @dag(schedule_interval=None, start_date=pendulum.DateTime(2022, 1, 1), default_args={"owner": "airflow"}) + @dag(schedule=None, start_date=pendulum.DateTime(2022, 1, 1), default_args={"owner": "airflow"}) def wrap(): total_1 = one() assert isinstance(total_1, XComArg) diff --git a/tests/www/test_security.py b/tests/www/test_security.py index 483b7b8c477bd..40501e26344c2 100644 --- a/tests/www/test_security.py +++ b/tests/www/test_security.py @@ -452,7 +452,7 @@ def test_get_accessible_dag_ids(app, security_manager, session): ], ) as user: - dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule_interval="2 2 * * *") + dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule="2 2 * * *") session.add(dag_model) session.commit() @@ -480,7 +480,7 @@ def test_dont_get_inaccessible_dag_ids_for_dag_resource_permission(app, security ], ) as user: - dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule_interval="2 2 * * *") + dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule="2 2 * * *") session.add(dag_model) session.commit() From 59fd25c425dd7719c30921796d1c601fdee9b267 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 13:24:12 -0700 Subject: [PATCH 07/38] stacklevel --- airflow/models/dag.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 46cd12df8cddd..7f8b9ec1df4a8 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -475,12 +475,14 @@ def __init__( "Param `schedule_interval` is deprecated and will be removed in a future release. " "Please use `schedule` instead. ", DeprecationWarning, + stacklevel=2, ) if timetable is not NOTSET: 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 From 0c96ac0acfc9e73141f306a967c5d51175932d92 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 29 Jul 2022 13:31:11 -0700 Subject: [PATCH 08/38] fix param handling --- airflow/models/dag.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 7f8b9ec1df4a8..01045d69eec3b 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -115,7 +115,7 @@ # but Mypy cannot handle that right now. Track progress of PEP 661 for progress. # See also: https://discuss.python.org/t/9126/7 ScheduleIntervalArg = Union[ArgNotSet, ScheduleInterval] -ScheduleArg = Union[ArgNotSet, ScheduleInterval, Timetable, Sequence["Dataset"]] +ScheduleArg = Union[ArgNotSet, ScheduleInterval, Timetable, List["Dataset"]] SLAMissCallback = Callable[["DAG", str, str, List["SlaMiss"], List[TaskInstance]], None] @@ -477,7 +477,7 @@ def __init__( DeprecationWarning, stacklevel=2, ) - if timetable is not NOTSET: + if timetable is not None: warnings.warn( "Param `timetable` is deprecated and will be removed in a future release. " "Please use `schedule` instead. ", @@ -489,8 +489,8 @@ def __init__( self.dataset_triggers: Optional[List[Dataset]] = None if schedule: - if isinstance(schedule, Sequence): - # if Sequence, only support Sequence[Dataset] + 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( From 8d90bc4aaff4f7d71e4c59f2cc19166d72560ed9 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sat, 6 Aug 2022 18:54:28 -0700 Subject: [PATCH 09/38] update some docs --- docs/apache-airflow/dag-run.rst | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/dag-run.rst index 2e8d70b1ec3dd..974c5bd5fa622 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/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 `_ 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 `_, or a ``datetime.timedelta`` object, +or one of the following cron "presets". For more elaborate scheduling requirements, you can implement a :doc:`custom timetable ` .. tip:: You can use an online editor for CRON expressions such as `Crontab guru `_ @@ -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 (24:00). A DAG run is usually scheduled *after* its associated data interval has ended, @@ -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, logical date, or data interval, see :doc:`/concepts/timetable`. For more information on ``logical date``, see :ref:`concepts:dag-run` and :ref:`faq:what-does-execution-date-mean` @@ -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. +The scheduler, by default, will kick off a DAG Run for any data interval that has not been run since the last data interval (or has been cleared). This concept is called Catchup. If your DAG is not written to handle its catchup (i.e., not limited to the interval, but instead to ``Now`` for instance.), From f80e3a63897ef7c93a3f5fdf9465b67efedd3dbe Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 11:18:50 -0700 Subject: [PATCH 10/38] update docs --- docs/apache-airflow/concepts/dags.rst | 8 ++--- docs/apache-airflow/concepts/scheduler.rst | 15 +++++----- docs/apache-airflow/concepts/tasks.rst | 2 +- docs/apache-airflow/concepts/timetable.rst | 30 ++++++++++--------- docs/apache-airflow/faq.rst | 20 ++++++------- docs/apache-airflow/howto/add-owner-links.rst | 2 +- docs/apache-airflow/templates-ref.rst | 11 ++----- docs/apache-airflow/tutorial.rst | 2 +- newsfragments/25410.significant.rst | 3 ++ 9 files changed, 46 insertions(+), 47 deletions(-) create mode 100644 newsfragments/25410.significant.rst diff --git a/docs/apache-airflow/concepts/dags.rst b/docs/apache-airflow/concepts/dags.rst index 881082965ab38..c3dfacfc858a6 100644 --- a/docs/apache-airflow/concepts/dags.rst +++ b/docs/apache-airflow/concepts/dags.rst @@ -145,21 +145,21 @@ DAGs will run in one of two ways: - When they are *triggered* either manually or via the API - On a defined *schedule*, which is defined as part of the DAG -DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule_interval`` argument, like this:: +DAGs do not *require* a schedule, but it's very common to define one. You define it via the ``schedule`` argument, like this:: with DAG("my_daily_dag", schedule="@daily"): ... -The ``schedule_interval`` argument takes any value that is a valid `Crontab `_ schedule value, so you could also do:: +The ``schedule`` argument takes any value that is a valid `Crontab `_ schedule value, so you could also do:: with DAG("my_daily_dag", schedule="0 * * * *"): ... .. tip:: - For more information on ``schedule_interval`` values, see :doc:`DAG Run `. + For more information on ``schedule`` values, see :doc:`DAG Run `. - If ``schedule_interval`` is not enough to express the DAG's schedule, see :doc:`Timetables `. + If ``schedule`` is not enough to express the DAG's schedule, see :doc:`Timetables `. For more information on ``logical date``, see :ref:`data-interval` and :ref:`faq:what-does-execution-date-mean`. diff --git a/docs/apache-airflow/concepts/scheduler.rst b/docs/apache-airflow/concepts/scheduler.rst index 655633e83d771..e6b4f14575027 100644 --- a/docs/apache-airflow/concepts/scheduler.rst +++ b/docs/apache-airflow/concepts/scheduler.rst @@ -44,19 +44,18 @@ Your DAGs will start executing once the scheduler is running successfully. .. note:: The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG. - Subsequent DAG Runs are created by the scheduler process, based on your DAG's ``schedule_interval``, - sequentially. + Subsequent DAG Runs are created according to your DAG's :doc:`timetable `. -The scheduler won't trigger your tasks until the period it covers has ended e.g., A job with ``schedule_interval`` set as ``@daily`` runs after the day +For dags with a cron or timedelta schedule, scheduler won't trigger your tasks until the period it covers has ended e.g., A job with ``schedule`` set as ``@daily`` runs after the day has ended. This technique makes sure that whatever data is required for that period is fully available before the DAG is executed. In the UI, it appears as if Airflow is running your tasks a day **late** .. note:: - If you run a DAG on a ``schedule_interval`` of one day, the run with data interval starting on ``2019-11-21`` triggers after ``2019-11-21T23:59``. + If you run a DAG on a ``schedule`` of one day, the run with data interval starting on ``2019-11-21`` triggers after ``2019-11-21T23:59``. - **Let's Repeat That**, the scheduler runs your job one ``schedule_interval`` AFTER the start date, at the END of the interval. + **Let's Repeat That**, the scheduler runs your job one ``schedule`` AFTER the start date, at the END of the interval. You should refer to :doc:`/dag-run` for details on scheduling a DAG. @@ -73,9 +72,9 @@ Triggering DAG with Future Date ------------------------------- If you want to use 'external trigger' to run future-dated data intervals, set ``allow_trigger_in_future = True`` in ``scheduler`` section in ``airflow.cfg``. -This only has effect if your DAG has no ``schedule_interval``. -If you keep default ``allow_trigger_in_future = False`` and try 'external trigger' to run future-dated data intervals, -the scheduler won't execute it now but the scheduler will execute it in the future once the current date rolls over to the start of the data interval. +This only has effect if your DAG is defined with ``schedule=None``. +When set to ``False`` (the default value), if you manually trigger a run with future-dated data intervals, +the scheduler will not execute it until its ``data_interval_start`` is in the past. .. _scheduler:ha: diff --git a/docs/apache-airflow/concepts/tasks.rst b/docs/apache-airflow/concepts/tasks.rst index a0e7bf14e0d2a..c48cad4d80f0c 100644 --- a/docs/apache-airflow/concepts/tasks.rst +++ b/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 `. + 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 `. .. _concepts:sla_miss_callback: diff --git a/docs/apache-airflow/concepts/timetable.rst b/docs/apache-airflow/concepts/timetable.rst index 28dc949ee8338..b9f4f3a12ebac 100644 --- a/docs/apache-airflow/concepts/timetable.rst +++ b/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 `, 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. + +Some examples: * Data intervals with "holes" between. (Instead of continuous, as both the cron expression and ``timedelta`` schedules represent.) @@ -49,7 +51,7 @@ As such, Airflow allows for custom timetables to be written in plugins and used DAGs. An example demonstrating a custom timetable can be found in the :doc:`/howto/timetable` how-to guide. -Built In Timetables +Built-in Timetables ------------------- Airflow comes with several common timetables built in to cover the most common use cases. Additional timetables @@ -104,11 +106,11 @@ DeltaDataIntervalTimetable ^^^^^^^^^^^^^^^^^^^^^^^^^^ Schedules data intervals with a time delta. Can be selected by providing a -:class:`datetime.timedelta` or ``dateutil.relativedelta.relativedelta`` to the ``schedule_interval`` parameter of a DAG. +:class:`datetime.timedelta` or ``dateutil.relativedelta.relativedelta`` to the ``schedule`` parameter of a DAG. .. code-block:: python - @dag(schedule_interval=datetime.timedelta(minutes=30)) + @dag(schedule=datetime.timedelta(minutes=30)) def example_dag(): pass @@ -122,12 +124,12 @@ trigger points, and triggers a DAG run at the end of each data interval. .. seealso:: `Differences between the two cron timetables`_ -This can be selected by providing a string that is a valid cron expression to the ``schedule_interval`` +This can be selected by providing a string that is a valid cron expression to the ``schedule`` parameter of a DAG as described in the :doc:`/concepts/dags` documentation. .. code-block:: python - @dag(schedule_interval="0 1 * * 3") # At 01:00 on Wednesday. + @dag(schedule="0 1 * * 3") # At 01:00 on Wednesday. def example_dag(): pass diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index 4958851972d91..e34aa592c215d 100644 --- a/docs/apache-airflow/faq.rst +++ b/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. -- 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 + is one day (``datetime.timedelta(1)``). You must specify a different ``schedule`` directly to the DAG object you instantiate, not as a ``default_param``, as task instances - do not override their parent DAG's ``schedule_interval``. + do not override their parent DAG's ``schedule``. - Is your ``start_date`` beyond where you can see it in the UI? If you set your ``start_date`` to some time say 3 months ago, you won't be able to see @@ -212,7 +212,7 @@ a global ``start_date`` for your tasks. This can be done by declaring your ``start_date`` directly in the ``DAG()`` object. The first DagRun to be created will be based on the ``min(start_date)`` for all your tasks. From that point on, the scheduler creates new DagRuns based on -your ``schedule_interval`` and the corresponding task instances run as your +your ``schedule`` and the corresponding task instances run as your dependencies are met. When introducing new tasks to your DAG, you need to pay special attention to ``start_date``, and may want to reactivate inactive DagRuns to get the new task onboarded properly. @@ -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`` minutes:seconds, a ``@daily`` job at midnight, a ``@monthly`` job on the first of the month. This is no longer required. Airflow will now auto align -the ``start_date`` and the ``schedule_interval``, by using the ``start_date`` +the ``start_date`` and the ``schedule``, by using the ``start_date`` as the moment to start looking. You can use any sensor or a ``TimeDeltaSensor`` to delay the execution of tasks within the schedule interval. -While ``schedule_interval`` does allow specifying a ``datetime.timedelta`` +While ``schedule`` does allow specifying a ``datetime.timedelta`` object, we recommend using the macros or cron expressions instead, as it enforces this idea of rounded schedules. @@ -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``, the ``next_ds``, ``next_ds_nodash``, ``prev_ds``, ``prev_ds_nodash`` values will be set to ``None``. - When manually triggering DAG, the schedule will be ignored, and ``prev_ds == next_ds == ds``. diff --git a/docs/apache-airflow/howto/add-owner-links.rst b/docs/apache-airflow/howto/add-owner-links.rst index 0dd67169db7cd..4f6525ff668d0 100644 --- a/docs/apache-airflow/howto/add-owner-links.rst +++ b/docs/apache-airflow/howto/add-owner-links.rst @@ -36,7 +36,7 @@ After that, define a task that will use this owner, and the link in the DAGs vie dag = DAG( dag_id="example_dag_owners", - schedule_interval="0 0 * * *", + schedule="0 0 * * *", start_date=datetime(2022, 8, 5), owner_links={"airflow": "https://airflow.apache.org/"}, ) diff --git a/docs/apache-airflow/templates-ref.rst b/docs/apache-airflow/templates-ref.rst index 2f78ebcce3fc0..7b9cab1c6c026 100644 --- a/docs/apache-airflow/templates-ref.rst +++ b/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. ``{{ next_ds }}`` the next execution date as ``YYYY-MM-DD`` if exists, else ``None`` ``{{ next_ds_nodash }}`` the next execution date as ``YYYYMMDD`` if exists, else ``None`` -``{{ prev_execution_date }}`` the previous execution date (if available) (`pendulum.DateTime`_) - if ``{{ execution_date }}`` is ``2018-01-08 00:00:00`` and - ``schedule_interval`` is ``@weekly``, ``{{ prev_execution_date }}`` - will be ``2018-01-01 00:00:00`` +``{{ prev_execution_date }}`` the logical date of the previous scheduled run (if applicable) ``{{ prev_ds }}`` the previous execution date as ``YYYY-MM-DD`` if exists, else ``None`` ``{{ prev_ds_nodash }}`` the previous execution date as ``YYYYMMDD`` if exists, else ``None`` ``{{ yesterday_ds }}`` the day before the execution date as ``YYYY-MM-DD`` diff --git a/docs/apache-airflow/tutorial.rst b/docs/apache-airflow/tutorial.rst index 1409adcbc1957..7e641ba229066 100644 --- a/docs/apache-airflow/tutorial.rst +++ b/docs/apache-airflow/tutorial.rst @@ -95,7 +95,7 @@ Instantiate a DAG We'll need a DAG object to nest our tasks into. Here we pass a string that defines the ``dag_id``, which serves as a unique identifier for your DAG. We also pass the default argument dictionary that we just defined and -define a ``schedule_interval`` of 1 day for the DAG. +define a ``schedule`` of 1 day for the DAG. .. exampleinclude:: /../../airflow/example_dags/tutorial.py :language: python diff --git a/newsfragments/25410.significant.rst b/newsfragments/25410.significant.rst new file mode 100644 index 0000000000000..6a3f82f2de546 --- /dev/null +++ b/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. From 68cd2ee42f361443b156ff9612693bf7aa62bffe Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 11:22:28 -0700 Subject: [PATCH 11/38] fix tests --- tests/api_connexion/schemas/test_dataset_schema.py | 2 +- tests/serialization/test_dag_serialization.py | 2 +- tests/www/views/test_views_grid.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/api_connexion/schemas/test_dataset_schema.py b/tests/api_connexion/schemas/test_dataset_schema.py index 4e6ad64c26be7..9703e2f0c355d 100644 --- a/tests/api_connexion/schemas/test_dataset_schema.py +++ b/tests/api_connexion/schemas/test_dataset_schema.py @@ -55,7 +55,7 @@ def test_serialize(self, dag_maker, session): with dag_maker(dag_id="test_dataset_upstream_schema", serialized=True, session=session): EmptyOperator(task_id="task1", outlets=[dataset]) with dag_maker( - dag_id="test_dataset_downstream_schema", schedule_on=[dataset], serialized=True, session=session + dag_id="test_dataset_downstream_schema", schedule=[dataset], serialized=True, session=session ): EmptyOperator(task_id="task2") session.flush() diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py index 42b9ca6d4cc00..a4c8293b3ebcf 100644 --- a/tests/serialization/test_dag_serialization.py +++ b/tests/serialization/test_dag_serialization.py @@ -1423,7 +1423,7 @@ def test_dag_deps_datasets(self): d3 = Dataset('d3') d4 = Dataset('d4') execution_date = datetime(2020, 1, 1) - with DAG(dag_id="test", start_date=execution_date, schedule_on=[d1]) as dag: + with DAG(dag_id="test", start_date=execution_date, schedule=[d1]) as dag: ExternalTaskSensor( task_id="task1", external_dag_id="external_dag_id", diff --git a/tests/www/views/test_views_grid.py b/tests/www/views/test_views_grid.py index e6f5a698a8298..c7eb6e15bbba8 100644 --- a/tests/www/views/test_views_grid.py +++ b/tests/www/views/test_views_grid.py @@ -331,7 +331,7 @@ def test_next_run_datasets(admin_client, dag_maker, session, app, monkeypatch): session.add_all(datasets) session.commit() - with dag_maker(dag_id=DAG_ID, schedule_on=datasets, serialized=True, session=session): + with dag_maker(dag_id=DAG_ID, schedule=datasets, serialized=True, session=session): EmptyOperator(task_id='task1') m.setattr(app, 'dag_bag', dag_maker.dagbag) From d08c5d5b202fa4372d3bf3147b8e260aa6d3d61a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 11:33:30 -0700 Subject: [PATCH 12/38] fix test dag endpoint --- tests/api_connexion/endpoints/test_dag_endpoint.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/api_connexion/endpoints/test_dag_endpoint.py b/tests/api_connexion/endpoints/test_dag_endpoint.py index cd14a7f7f8ac1..09ef3c28ae050 100644 --- a/tests/api_connexion/endpoints/test_dag_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_endpoint.py @@ -122,7 +122,7 @@ def _create_dag_models(self, count, session=None): dag_model = DagModel( dag_id=f"TEST_DAG_{num}", fileloc=f"/tmp/dag_{num}.py", - schedule="2 2 * * *", + schedule_interval="2 2 * * *", is_active=True, is_paused=False, ) @@ -133,7 +133,7 @@ def _create_deactivated_dag(self, session=None): dag_model = DagModel( dag_id="TEST_DAG_DELETED_1", fileloc="/tmp/dag_del_1.py", - schedule="2 2 * * *", + schedule_interval="2 2 * * *", is_active=False, ) session.add(dag_model) @@ -179,7 +179,7 @@ def test_should_respond_200_with_schedule_interval_none(self, session): dag_model = DagModel( dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", - schedule=None, + schedule_interval=None, is_paused=False, ) session.add(dag_model) @@ -901,7 +901,7 @@ def test_should_respond_404(self): @provide_session def _create_dag_model(self, session=None): dag_model = DagModel( - dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", schedule="2 2 * * *", is_paused=True + dag_id="TEST_DAG_1", fileloc="/tmp/dag_1.py", schedule_interval="2 2 * * *", is_paused=True ) session.add(dag_model) return dag_model From ae3b1d0a3190cffc99dd9e9a720885ac37954ea6 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 12:26:08 -0700 Subject: [PATCH 13/38] fix dag run endpoint test --- airflow/models/dag.py | 2 +- tests/api_connexion/endpoints/test_dag_run_endpoint.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 01045d69eec3b..d8d1f5fdd8e7c 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -488,7 +488,7 @@ def __init__( self.schedule_interval: ScheduleInterval self.dataset_triggers: Optional[List[Dataset]] = None - if schedule: + if schedule is not NOTSET: if isinstance(schedule, List): # if List, only support List[Dataset] if any(isinstance(x, Dataset) for x in schedule): diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index e9d5cea0ef2b6..1214fff42e294 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -1044,7 +1044,7 @@ def test_should_respond_200(self, logical_date_field_name, dag_run_id, logical_d expected_dag_run_id = f"manual__{expected_logical_date}" else: expected_dag_run_id = dag_run_id - assert { + assert response.json == { "conf": {}, "dag_id": "TEST_DAG_ID", "dag_run_id": expected_dag_run_id, @@ -1058,7 +1058,7 @@ def test_should_respond_200(self, logical_date_field_name, dag_run_id, logical_d "data_interval_start": expected_logical_date, "last_scheduling_decision": None, "run_type": "manual", - } == response.json + } def test_should_respond_400_if_a_dag_has_import_errors(self, session): """Test that if a dagmodel has import errors, dags won't be triggered""" From cf11b5323790a8b01ce378e019904d1ee25f0efa Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 12:31:32 -0700 Subject: [PATCH 14/38] fix param namee --- tests/models/test_dag.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 915a0e3fcc757..f83f1becf707d 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -1719,14 +1719,14 @@ def test_next_dagrun_info_catchup(self): Test to check that a DAG with catchup = False only schedules beginning now, not back to the start date """ - def make_dag(dag_id, schedule_interval, start_date, catchup): + def make_dag(dag_id, schedule, start_date, catchup): default_args = { 'owner': 'airflow', 'depends_on_past': False, } dag = DAG( dag_id, - schedule=schedule_interval, + schedule=schedule, start_date=start_date, catchup=catchup, default_args=default_args, From d8e06fae5b600649678b76d5d26d40058220719b Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 12:57:36 -0700 Subject: [PATCH 15/38] fix type --- airflow/models/dag.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index d8d1f5fdd8e7c..e0a1ee9ac38b0 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -3256,7 +3256,7 @@ def dag( jinja_environment_kwargs: Optional[Dict] = None, render_template_as_native_obj: bool = False, tags: Optional[List[str]] = None, - schedule: Optional[Sequence["Dataset"]] = None, + schedule: Optional[ScheduleArg] = None, owner_links: Optional[Dict[str, str]] = None, ) -> Callable[[Callable], Callable[..., DAG]]: """ From a4b99cc23a45b083800d7e02d5ee61a86d5cbe6a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 13:37:10 -0700 Subject: [PATCH 16/38] fix tests --- airflow/models/dag.py | 3 ++- airflow/serialization/serialized_objects.py | 2 +- tests/api_connexion/schemas/test_dag_schema.py | 2 +- tests/serialization/test_dag_serialization.py | 3 ++- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index e0a1ee9ac38b0..64e80d051a7d4 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -2808,7 +2808,7 @@ def get_num_task_instances(dag_id, task_ids=None, states=None, session=NEW_SESSI def get_serialized_fields(cls): """Stringified DAGs and operators contain exactly these fields.""" if not cls.__serialized_fields: - cls.__serialized_fields = frozenset(vars(DAG(dag_id='test')).keys()) - { + exclusion_list = { 'parent_dag', '_old_context_manager_dags', 'safe_dag_id', @@ -2830,6 +2830,7 @@ def get_serialized_fields(cls): 'has_on_success_callback', 'has_on_failure_callback', } + cls.__serialized_fields = frozenset(vars(DAG(dag_id='test')).keys()) - exclusion_list return cls.__serialized_fields def get_edge_info(self, upstream_task_id: str, downstream_task_id: str) -> EdgeInfoType: diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index 2ec8db7daf697..9ddec956c5425 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -1111,7 +1111,7 @@ def deserialize_dag(cls, encoded_dag: Dict[str, Any]) -> 'SerializedDAG': v = cls._deserialize(v) elif k == "params": v = cls._deserialize_params_dict(v) - elif k == "upstream_datasets": + elif k == "dataset_triggers": v = cls._deserialize(v) # else use v as it is diff --git a/tests/api_connexion/schemas/test_dag_schema.py b/tests/api_connexion/schemas/test_dag_schema.py index 4f59b2108d2e9..040ce41f1d066 100644 --- a/tests/api_connexion/schemas/test_dag_schema.py +++ b/tests/api_connexion/schemas/test_dag_schema.py @@ -37,7 +37,7 @@ def test_serialize_test_dag_schema(url_safe_serializer): fileloc="/root/airflow/dags/my_dag.py", owners="airflow1,airflow2", description="The description", - schedule="5 4 * * *", + schedule_interval="5 4 * * *", tags=[DagTag(name="tag-1"), DagTag(name="tag-2")], ) serialized_dag = DAGSchema().dump(dag_model) diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py index a4c8293b3ebcf..37da287cf1de1 100644 --- a/tests/serialization/test_dag_serialization.py +++ b/tests/serialization/test_dag_serialization.py @@ -487,7 +487,7 @@ def validate_deserialized_dag(self, serialized_dag, dag): Verify that all example DAGs work with DAG Serialization by checking fields between Serialized Dags & non-Serialized Dags """ - fields_to_check = dag.get_serialized_fields() - { + exclusion_list = { # Doesn't implement __eq__ properly. Check manually. 'timetable', 'timezone', @@ -496,6 +496,7 @@ def validate_deserialized_dag(self, serialized_dag, dag): "_task_group", 'params', } + fields_to_check = dag.get_serialized_fields() - exclusion_list for field in fields_to_check: assert getattr(serialized_dag, field) == getattr( dag, field From 244e49607556966825fcf74c393a4387bd7187f8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 7 Aug 2022 19:23:28 -0700 Subject: [PATCH 17/38] fix tests --- tests/www/test_security.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/www/test_security.py b/tests/www/test_security.py index 40501e26344c2..483b7b8c477bd 100644 --- a/tests/www/test_security.py +++ b/tests/www/test_security.py @@ -452,7 +452,7 @@ def test_get_accessible_dag_ids(app, security_manager, session): ], ) as user: - dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule="2 2 * * *") + dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule_interval="2 2 * * *") session.add(dag_model) session.commit() @@ -480,7 +480,7 @@ def test_dont_get_inaccessible_dag_ids_for_dag_resource_permission(app, security ], ) as user: - dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule="2 2 * * *") + dag_model = DagModel(dag_id=dag_id, fileloc="/tmp/dag_.py", schedule_interval="2 2 * * *") session.add(dag_model) session.commit() From 759c10cbaff83c4ead8346d8005dbbfc541d6e6f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 09:58:31 -0700 Subject: [PATCH 18/38] provider examples should stick with schedule_interval for now --- .../amazon/aws/example_dags/example_appflow.py | 2 +- .../amazon/aws/example_dags/example_dms.py | 2 +- .../aws/example_dags/example_dynamodb_to_s3.py | 2 +- .../amazon/aws/example_dags/example_ec2.py | 2 +- .../amazon/aws/example_dags/example_ecs.py | 2 +- .../aws/example_dags/example_eks_templated.py | 2 +- .../example_eks_with_fargate_in_one_step.py | 2 +- .../example_eks_with_fargate_profile.py | 2 +- .../example_eks_with_nodegroup_in_one_step.py | 2 +- .../example_dags/example_eks_with_nodegroups.py | 2 +- .../amazon/aws/example_dags/example_emr.py | 2 +- .../amazon/aws/example_dags/example_emr_eks.py | 2 +- .../amazon/aws/example_dags/example_ftp_to_s3.py | 2 +- .../amazon/aws/example_dags/example_gcs_to_s3.py | 2 +- .../aws/example_dags/example_glacier_to_gcs.py | 2 +- .../example_google_api_sheets_to_s3.py | 2 +- .../example_google_api_youtube_to_s3.py | 2 +- .../aws/example_dags/example_hive_to_dynamodb.py | 2 +- .../example_dags/example_imap_attachment_to_s3.py | 2 +- .../amazon/aws/example_dags/example_mongo_to_s3.py | 2 +- .../amazon/aws/example_dags/example_quicksight.py | 2 +- .../amazon/aws/example_dags/example_s3.py | 2 +- .../amazon/aws/example_dags/example_s3_to_ftp.py | 2 +- .../aws/example_dags/example_s3_to_redshift.py | 2 +- .../amazon/aws/example_dags/example_s3_to_sftp.py | 2 +- .../aws/example_dags/example_salesforce_to_s3.py | 2 +- .../amazon/aws/example_dags/example_sftp_to_s3.py | 2 +- .../amazon/aws/example_dags/example_sql_to_s3.py | 2 +- .../example_automl_nl_text_classification.py | 2 +- .../example_automl_nl_text_sentiment.py | 2 +- .../cloud/example_dags/example_automl_tables.py | 8 ++++---- .../example_dags/example_automl_translation.py | 2 +- ...ple_automl_video_intelligence_classification.py | 2 +- .../example_automl_video_intelligence_tracking.py | 2 +- .../example_automl_vision_object_detection.py | 2 +- .../cloud/example_dags/example_bigquery_dts.py | 2 +- .../google/cloud/example_dags/example_bigtable.py | 2 +- .../cloud/example_dags/example_cloud_build.py | 4 ++-- .../cloud/example_dags/example_cloud_composer.py | 4 ++-- .../cloud/example_dags/example_cloud_sql_query.py | 2 +- .../example_cloud_storage_transfer_service_aws.py | 2 +- .../example_cloud_storage_transfer_service_gcp.py | 2 +- .../google/cloud/example_dags/example_compute.py | 2 +- .../cloud/example_dags/example_compute_ssh.py | 2 +- .../google/cloud/example_dags/example_dataflow.py | 8 ++++---- .../example_dags/example_dataflow_flex_template.py | 2 +- .../cloud/example_dags/example_dataflow_sql.py | 2 +- .../cloud/example_dags/example_datafusion.py | 2 +- .../google/cloud/example_dags/example_dataplex.py | 4 +++- .../google/cloud/example_dags/example_dataprep.py | 2 +- .../google/cloud/example_dags/example_dlp.py | 8 ++++---- .../example_dags/example_facebook_ads_to_gcs.py | 2 +- .../google/cloud/example_dags/example_functions.py | 2 +- .../cloud/example_dags/example_gdrive_to_gcs.py | 2 +- .../cloud/example_dags/example_gdrive_to_local.py | 2 +- .../google/cloud/example_dags/example_looker.py | 2 +- .../google/cloud/example_dags/example_mlengine.py | 2 +- .../cloud/example_dags/example_postgres_to_gcs.py | 2 +- .../cloud/example_dags/example_presto_to_gcs.py | 2 +- .../example_dags/example_salesforce_to_gcs.py | 2 +- .../cloud/example_dags/example_sftp_to_gcs.py | 2 +- .../google/cloud/example_dags/example_vertex_ai.py | 14 +++++++------- .../example_dags/example_video_intelligence.py | 2 +- .../google/cloud/example_dags/example_vision.py | 6 +++--- .../example_dags/example_display_video.py | 6 +++--- .../suite/example_dags/example_local_to_drive.py | 2 +- 66 files changed, 89 insertions(+), 87 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index ff4b7924e9712..5ce6a7b6a1eeb 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -34,7 +34,7 @@ with DAG( "example_appflow", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/amazon/aws/example_dags/example_dms.py b/airflow/providers/amazon/aws/example_dags/example_dms.py index 27444a5917157..e3d0b247bca17 100644 --- a/airflow/providers/amazon/aws/example_dags/example_dms.py +++ b/airflow/providers/amazon/aws/example_dags/example_dms.py @@ -227,7 +227,7 @@ def delete_dms_assets(): with DAG( dag_id='example_dms', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py index 0630193495bc0..a3d05055b0a13 100644 --- a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py @@ -27,7 +27,7 @@ with DAG( dag_id='example_dynamodb_to_s3', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ec2.py b/airflow/providers/amazon/aws/example_dags/example_ec2.py index 50d8a97daeab8..fc364a5223cbd 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ec2.py +++ b/airflow/providers/amazon/aws/example_dags/example_ec2.py @@ -26,7 +26,7 @@ with DAG( dag_id='example_ec2', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ecs.py b/airflow/providers/amazon/aws/example_dags/example_ecs.py index 5cd9fb541aee7..16f569a6be925 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ecs.py +++ b/airflow/providers/amazon/aws/example_dags/example_ecs.py @@ -53,7 +53,7 @@ with DAG( dag_id=DAG_ID, - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_templated.py b/airflow/providers/amazon/aws/example_dags/example_eks_templated.py index bef8b220f043f..f1b6ff3bca2d9 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_templated.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_templated.py @@ -49,7 +49,7 @@ with DAG( dag_id='example_eks_templated', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example', 'templated'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py index 35c5eecf35b0d..8c7854be6f82f 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py @@ -44,7 +44,7 @@ with DAG( dag_id='example_eks_with_fargate_in_one_step', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py index 6715698ac6a05..fdf599008903a 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py @@ -47,7 +47,7 @@ with DAG( dag_id='example_eks_with_fargate_profile', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py index f6c4cbaaf9cb8..bb39825d0566b 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py @@ -43,7 +43,7 @@ with DAG( dag_id='example_eks_with_nodegroup_in_one_step', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py index 670d136ddfbd4..ba333ab3d0340 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py @@ -46,7 +46,7 @@ with DAG( dag_id='example_eks_with_nodegroups', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_emr.py b/airflow/providers/amazon/aws/example_dags/example_emr.py index 51fa6e0d0ef23..0d9bf0cc640c1 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr.py @@ -68,7 +68,7 @@ with DAG( dag_id='example_emr', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_eks.py b/airflow/providers/amazon/aws/example_dags/example_emr_eks.py index 02bdd4dcce48e..66222611a2ad7 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr_eks.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr_eks.py @@ -54,7 +54,7 @@ with DAG( dag_id='example_emr_eks', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py index 97372de06b527..f5d4318c919e3 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py @@ -27,7 +27,7 @@ with models.DAG( "example_ftp_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py index 37869b75ca9bf..ea52b400ff46d 100644 --- a/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py @@ -26,7 +26,7 @@ with DAG( dag_id="example_gcs_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=["example"], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py index 4a8a5ab288ff0..05591b9e3e7e3 100644 --- a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py +++ b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py @@ -28,7 +28,7 @@ with DAG( "example_glacier_to_gcs", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), # Override to match your needs catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py index 32a85b4649114..c5705a1dd8a60 100644 --- a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py @@ -31,7 +31,7 @@ with DAG( dag_id="example_google_api_sheets_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py index 5865857a8919f..8ff9874cf1247 100644 --- a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py @@ -66,7 +66,7 @@ def transform_video_ids(**kwargs): with DAG( dag_id="example_google_api_youtube_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py b/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py index e21482d3e6935..ac20a9cec7f21 100644 --- a/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py +++ b/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py @@ -105,7 +105,7 @@ def configure_hive_connection(): with DAG( dag_id='example_hive_to_dynamodb', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py index 917e4d0960ccb..bb355468a3bf5 100644 --- a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py @@ -35,7 +35,7 @@ with DAG( dag_id="example_imap_attachment_to_s3", start_date=datetime(2021, 1, 1), - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 catchup=False, tags=['example'], ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py index 788ad68cf3c0f..fc4218410da86 100644 --- a/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py @@ -29,7 +29,7 @@ with models.DAG( "example_mongo_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_quicksight.py b/airflow/providers/amazon/aws/example_dags/example_quicksight.py index ee8d2589d9e79..27b93971b65fc 100644 --- a/airflow/providers/amazon/aws/example_dags/example_quicksight.py +++ b/airflow/providers/amazon/aws/example_dags/example_quicksight.py @@ -28,7 +28,7 @@ with DAG( dag_id="example_quicksight", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=["example"], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_s3.py b/airflow/providers/amazon/aws/example_dags/example_s3.py index 6144af57a7a25..be0cc3b1e965d 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3.py @@ -57,7 +57,7 @@ with DAG( dag_id='example_s3', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py index e2e79782991de..b4298671b22a2 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py @@ -27,7 +27,7 @@ with models.DAG( "example_s3_to_ftp", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py index 483b75cb1e987..aa1856547b0e7 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py @@ -46,7 +46,7 @@ def task_remove_sample_data_from_s3(): with DAG( dag_id="example_s3_to_redshift", start_date=datetime(2021, 1, 1), - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 catchup=False, tags=['example'], ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py index 812815ed52f33..7cc1cc6d545e7 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py @@ -26,7 +26,7 @@ with models.DAG( "example_s3_to_sftp", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py index c5677ea34ccae..5438393e83e14 100644 --- a/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py @@ -32,7 +32,7 @@ with DAG( dag_id="example_salesforce_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 7, 8), catchup=False, tags=["example"], diff --git a/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py index 6b72886dce037..8d34671a52002 100644 --- a/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py @@ -27,7 +27,7 @@ with models.DAG( "example_sftp_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py index 2f7aa7b057a5d..84d4ddfee1893 100644 --- a/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py @@ -28,7 +28,7 @@ with models.DAG( "example_sql_to_s3", - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py index e3350d069253f..4b93bb02fbb54 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py @@ -59,7 +59,7 @@ # Example DAG for AutoML Natural Language Text Classification with models.DAG( "example_automl_text_cls", - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py index 14b8c9f8fd8d2..dabaf7f16486a 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py @@ -59,7 +59,7 @@ # Example DAG for AutoML Natural Language Text Sentiment with models.DAG( "example_automl_text_sentiment", - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_tables.py b/airflow/providers/google/cloud/example_dags/example_automl_tables.py index cb06224d53b54..6f5af0e3d0445 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_tables.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_tables.py @@ -86,7 +86,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: # Example DAG to create dataset, train model_id and deploy it. with models.DAG( "example_create_and_deploy", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, user_defined_macros={ @@ -199,7 +199,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: # Example DAG for AutoML datasets operations with models.DAG( "example_automl_dataset", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, @@ -268,7 +268,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: with models.DAG( "example_gcp_get_deploy", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], @@ -294,7 +294,7 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: with models.DAG( "example_gcp_predict", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_automl_translation.py b/airflow/providers/google/cloud/example_dags/example_automl_translation.py index 6ed845928134a..e033165fbb8a7 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_translation.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_translation.py @@ -65,7 +65,7 @@ # Example DAG for AutoML Translation with models.DAG( "example_automl_translation", - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py index c9bda04122d5e..e1f4fbf4972e0 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py @@ -62,7 +62,7 @@ # Example DAG for AutoML Video Intelligence Classification with models.DAG( "example_automl_video", - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py index ccd4e8df1b906..1fa804b963e39 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py @@ -63,7 +63,7 @@ # Example DAG for AutoML Video Intelligence Object Tracking with models.DAG( "example_automl_video_tracking", - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py index bc19d91cf7aaa..442d01875fe4b 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py @@ -62,7 +62,7 @@ # Example DAG for AutoML Vision Object Detection with models.DAG( "example_automl_vision_detection", - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py index 2e5d0be0170d1..f7dba38a156bf 100644 --- a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py +++ b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py @@ -64,7 +64,7 @@ with models.DAG( "example_gcp_bigquery_dts", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_bigtable.py b/airflow/providers/google/cloud/example_dags/example_bigtable.py index 70044541e2663..f2e6f7afa5053 100644 --- a/airflow/providers/google/cloud/example_dags/example_bigtable.py +++ b/airflow/providers/google/cloud/example_dags/example_bigtable.py @@ -80,7 +80,7 @@ with models.DAG( 'example_gcp_bigtable_operators', - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_build.py b/airflow/providers/google/cloud/example_dags/example_cloud_build.py index 06777ff619a3d..98e383b67c899 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_build.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_build.py @@ -116,7 +116,7 @@ with models.DAG( "example_gcp_cloud_build", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], @@ -205,7 +205,7 @@ with models.DAG( "example_gcp_cloud_build_trigger", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_composer.py b/airflow/providers/google/cloud/example_dags/example_cloud_composer.py index 6b8bd6b08e2bb..2e22b0f068fe2 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_composer.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_composer.py @@ -55,7 +55,7 @@ with models.DAG( "composer_dag1", - schedule="@once", # Override to match your needs + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], @@ -118,7 +118,7 @@ with models.DAG( "composer_dag_deferrable1", - schedule="@once", # Override to match your needs + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py index a956a9c4a52cc..409ce5f3bc497 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py @@ -270,7 +270,7 @@ def get_absolute_path(path): with models.DAG( dag_id='example_gcp_sql_query', - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py index d6a88e4e63308..e6ef42f72c579 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py @@ -106,7 +106,7 @@ with models.DAG( 'example_gcp_transfer_aws', - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py index 499c5098bd6d7..263277c3e014a 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py @@ -101,7 +101,7 @@ with models.DAG( "example_gcp_transfer", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_compute.py b/airflow/providers/google/cloud/example_dags/example_compute.py index 181062a81ce21..05b3081ddd5a5 100644 --- a/airflow/providers/google/cloud/example_dags/example_compute.py +++ b/airflow/providers/google/cloud/example_dags/example_compute.py @@ -52,7 +52,7 @@ with models.DAG( 'example_gcp_compute', - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_compute_ssh.py b/airflow/providers/google/cloud/example_dags/example_compute_ssh.py index d1353b05feab5..73a225bb49f63 100644 --- a/airflow/providers/google/cloud/example_dags/example_compute_ssh.py +++ b/airflow/providers/google/cloud/example_dags/example_compute_ssh.py @@ -30,7 +30,7 @@ with models.DAG( 'example_compute_ssh', - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow.py b/airflow/providers/google/cloud/example_dags/example_dataflow.py index e131ec66e5a07..06c75464d6907 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow.py @@ -64,7 +64,7 @@ with models.DAG( "example_gcp_dataflow_native_java", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=['example'], @@ -115,7 +115,7 @@ default_args=default_args, start_date=START_DATE, catchup=False, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_native_python: @@ -151,7 +151,7 @@ default_args=default_args, start_date=START_DATE, catchup=False, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_native_python_async: # [START howto_operator_start_python_job_async] @@ -253,7 +253,7 @@ def check_autoscaling_event(autoscaling_events: List[dict]) -> bool: default_args=default_args, start_date=START_DATE, catchup=False, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_template: # [START howto_operator_start_template_job] diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py b/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py index a5a6bd84c1c12..69713304a3e5e 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py @@ -47,7 +47,7 @@ dag_id="example_gcp_dataflow_flex_template_java", start_date=datetime(2021, 1, 1), catchup=False, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 ) as dag_flex_template: # [START howto_operator_start_template_job] start_flex_template = DataflowStartFlexTemplateOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py b/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py index 538ad559a9c6f..c04941be43221 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py @@ -38,7 +38,7 @@ dag_id="example_gcp_dataflow_sql", start_date=datetime(2021, 1, 1), catchup=False, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_sql: # [START howto_operator_start_sql_job] diff --git a/airflow/providers/google/cloud/example_dags/example_datafusion.py b/airflow/providers/google/cloud/example_dags/example_datafusion.py index 5a43d2b4b571b..d4f6861c84ebe 100644 --- a/airflow/providers/google/cloud/example_dags/example_datafusion.py +++ b/airflow/providers/google/cloud/example_dags/example_datafusion.py @@ -147,7 +147,7 @@ with models.DAG( "example_data_fusion", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_dataplex.py b/airflow/providers/google/cloud/example_dags/example_dataplex.py index dc47dbe3c59f0..3fcc135200605 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataplex.py +++ b/airflow/providers/google/cloud/example_dags/example_dataplex.py @@ -51,7 +51,9 @@ # [END howto_dataplex_configuration] with models.DAG( - "example_dataplex", start_date=datetime.datetime(2021, 1, 1), schedule="@once", catchup=False + "example_dataplex", + start_date=datetime.datetime(2021, 1, 1), + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 ) as dag: # [START howto_dataplex_create_task_operator] create_dataplex_task = DataplexCreateTaskOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dataprep.py b/airflow/providers/google/cloud/example_dags/example_dataprep.py index 5f663824a668b..19b5f5da2384f 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataprep.py +++ b/airflow/providers/google/cloud/example_dags/example_dataprep.py @@ -52,7 +52,7 @@ with models.DAG( "example_dataprep", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), # Override to match your needs catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_dlp.py b/airflow/providers/google/cloud/example_dags/example_dlp.py index cbdbb89aa0a50..883903e0513ba 100644 --- a/airflow/providers/google/cloud/example_dags/example_dlp.py +++ b/airflow/providers/google/cloud/example_dags/example_dlp.py @@ -63,7 +63,7 @@ with models.DAG( "example_gcp_dlp", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=['example'], @@ -113,7 +113,7 @@ with models.DAG( "example_gcp_dlp_info_types", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example", "dlp", "info-types"], @@ -157,7 +157,7 @@ with models.DAG( "example_gcp_dlp_job", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example", "dlp_job"], @@ -203,7 +203,7 @@ with models.DAG( "example_gcp_dlp_deidentify_content", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example", "dlp", "deidentify"], diff --git a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py index 4028bdd5838f6..28d3bcfe0de35 100644 --- a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py @@ -57,7 +57,7 @@ with models.DAG( "example_facebook_ads_to_gcs", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_functions.py b/airflow/providers/google/cloud/example_dags/example_functions.py index d2999e78a0275..1d10c38a5a2cd 100644 --- a/airflow/providers/google/cloud/example_dags/example_functions.py +++ b/airflow/providers/google/cloud/example_dags/example_functions.py @@ -94,7 +94,7 @@ with models.DAG( 'example_gcp_function', default_args=default_args, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py index 1597796a40646..dcbc5c1a8b223 100644 --- a/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py @@ -32,7 +32,7 @@ "example_gdrive_to_gcs_with_gdrive_sensor", start_date=datetime(2021, 1, 1), catchup=False, - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=["example"], ) as dag: # [START detect_file] diff --git a/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py b/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py index 92deb49139ebd..1ec7d19e021cc 100644 --- a/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py +++ b/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py @@ -31,7 +31,7 @@ "example_gdrive_to_local_with_gdrive_sensor", start_date=datetime(2021, 1, 1), catchup=False, - schedule=None, # Override to match your needs + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 tags=["example"], ) as dag: # [START detect_file] diff --git a/airflow/providers/google/cloud/example_dags/example_looker.py b/airflow/providers/google/cloud/example_dags/example_looker.py index 28e350206ab1b..c3f0f19cc5866 100644 --- a/airflow/providers/google/cloud/example_dags/example_looker.py +++ b/airflow/providers/google/cloud/example_dags/example_looker.py @@ -28,7 +28,7 @@ with models.DAG( dag_id='example_gcp_looker', - schedule=None, + schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_mlengine.py b/airflow/providers/google/cloud/example_dags/example_mlengine.py index 7fc42b44fb816..3c574e4767cdf 100644 --- a/airflow/providers/google/cloud/example_dags/example_mlengine.py +++ b/airflow/providers/google/cloud/example_dags/example_mlengine.py @@ -59,7 +59,7 @@ with models.DAG( "example_gcp_mlengine", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py index cc727b0b06794..b2eb28944d8ba 100644 --- a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py @@ -31,7 +31,7 @@ with models.DAG( dag_id='example_postgres_to_gcs', - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py index 9a61fb988f98c..bf6dec9d9f7fe 100644 --- a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py @@ -48,7 +48,7 @@ def safe_name(s: str) -> str: with models.DAG( dag_id="example_presto_to_gcs", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py index eb16de6fbff14..dafeb795e11d0 100644 --- a/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py @@ -44,7 +44,7 @@ with models.DAG( "example_salesforce_to_gcs", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py index afb908bf39f72..df956b46e8dd6 100644 --- a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py @@ -38,7 +38,7 @@ with models.DAG( "example_sftp_to_gcs", - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py index 84482061d41ba..cd0b14f66e8a3 100644 --- a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py +++ b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py @@ -238,7 +238,7 @@ with models.DAG( "example_gcp_vertex_ai_custom_jobs", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as custom_jobs_dag: @@ -328,7 +328,7 @@ with models.DAG( "example_gcp_vertex_ai_dataset", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dataset_dag: @@ -431,7 +431,7 @@ with models.DAG( "example_gcp_vertex_ai_auto_ml", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as auto_ml_dag: @@ -548,7 +548,7 @@ with models.DAG( "example_gcp_vertex_ai_batch_prediction_job", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as batch_prediction_job_dag: @@ -588,7 +588,7 @@ with models.DAG( "example_gcp_vertex_ai_endpoint", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as endpoint_dag: @@ -644,7 +644,7 @@ with models.DAG( "example_gcp_vertex_ai_hyperparameter_tuning_job", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as hyperparameter_tuning_job_dag: @@ -717,7 +717,7 @@ with models.DAG( "example_gcp_vertex_ai_model_service", - schedule="@once", + schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as model_service_dag: diff --git a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py index 4692060a2a4b6..59fb8ba110c43 100644 --- a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py +++ b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py @@ -49,7 +49,7 @@ with models.DAG( "example_gcp_video_intelligence", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_vision.py b/airflow/providers/google/cloud/example_dags/example_vision.py index 4da627a06b03d..552a7ed5efe87 100644 --- a/airflow/providers/google/cloud/example_dags/example_vision.py +++ b/airflow/providers/google/cloud/example_dags/example_vision.py @@ -118,7 +118,7 @@ with models.DAG( 'example_gcp_vision_autogenerated_id', - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag_autogenerated_id: @@ -276,7 +276,7 @@ with models.DAG( 'example_gcp_vision_explicit_id', - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag_explicit_id: @@ -444,7 +444,7 @@ with models.DAG( 'example_gcp_vision_annotate_image', - schedule='@once', + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag_annotate_image: diff --git a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py index 915c79ff5aa89..47f07c0297b73 100644 --- a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py +++ b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py @@ -86,7 +86,7 @@ with models.DAG( "example_display_video", - schedule='@once', # Override to match your needs, + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag1: @@ -129,7 +129,7 @@ with models.DAG( "example_display_video_misc", - schedule='@once', # Override to match your needs, + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag2: @@ -163,7 +163,7 @@ with models.DAG( "example_display_video_sdf", - schedule='@once', # Override to match your needs, + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag3: diff --git a/airflow/providers/google/suite/example_dags/example_local_to_drive.py b/airflow/providers/google/suite/example_dags/example_local_to_drive.py index 7a34e45a6cba4..e5171be54bc95 100644 --- a/airflow/providers/google/suite/example_dags/example_local_to_drive.py +++ b/airflow/providers/google/suite/example_dags/example_local_to_drive.py @@ -31,7 +31,7 @@ with models.DAG( "example_local_to_drive", - schedule='@once', # Override to match your needs + schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], From b32a0aacbc503f830f75b2ebb6eb3db65e524bda Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 12:56:57 -0700 Subject: [PATCH 19/38] remove schedule_interval param from example dags --- .../amazon/aws/example_dags/example_appflow.py | 1 - .../providers/amazon/aws/example_dags/example_dms.py | 1 - .../aws/example_dags/example_dynamodb_to_s3.py | 1 - .../providers/amazon/aws/example_dags/example_ec2.py | 1 - .../providers/amazon/aws/example_dags/example_ecs.py | 1 - .../amazon/aws/example_dags/example_eks_templated.py | 1 - .../example_eks_with_fargate_in_one_step.py | 1 - .../example_dags/example_eks_with_fargate_profile.py | 1 - .../example_eks_with_nodegroup_in_one_step.py | 1 - .../aws/example_dags/example_eks_with_nodegroups.py | 1 - .../providers/amazon/aws/example_dags/example_emr.py | 1 - .../amazon/aws/example_dags/example_emr_eks.py | 1 - .../amazon/aws/example_dags/example_ftp_to_s3.py | 1 - .../amazon/aws/example_dags/example_gcs_to_s3.py | 1 - .../aws/example_dags/example_glacier_to_gcs.py | 1 - .../example_dags/example_google_api_sheets_to_s3.py | 1 - .../example_dags/example_google_api_youtube_to_s3.py | 1 - .../aws/example_dags/example_hive_to_dynamodb.py | 1 - .../example_dags/example_imap_attachment_to_s3.py | 1 - .../amazon/aws/example_dags/example_mongo_to_s3.py | 1 - .../amazon/aws/example_dags/example_quicksight.py | 1 - .../providers/amazon/aws/example_dags/example_s3.py | 1 - .../amazon/aws/example_dags/example_s3_to_ftp.py | 1 - .../aws/example_dags/example_s3_to_redshift.py | 1 - .../amazon/aws/example_dags/example_s3_to_sftp.py | 1 - .../aws/example_dags/example_salesforce_to_s3.py | 1 - .../amazon/aws/example_dags/example_sftp_to_s3.py | 1 - .../amazon/aws/example_dags/example_sql_to_s3.py | 1 - .../example_automl_nl_text_classification.py | 1 - .../example_dags/example_automl_nl_text_sentiment.py | 1 - .../cloud/example_dags/example_automl_tables.py | 4 ---- .../cloud/example_dags/example_automl_translation.py | 1 - ...ample_automl_video_intelligence_classification.py | 1 - .../example_automl_video_intelligence_tracking.py | 1 - .../example_automl_vision_object_detection.py | 1 - .../cloud/example_dags/example_bigquery_dts.py | 1 - .../google/cloud/example_dags/example_bigtable.py | 1 - .../google/cloud/example_dags/example_cloud_build.py | 2 -- .../cloud/example_dags/example_cloud_composer.py | 2 -- .../cloud/example_dags/example_cloud_sql_query.py | 1 - .../example_cloud_storage_transfer_service_aws.py | 1 - .../example_cloud_storage_transfer_service_gcp.py | 1 - .../google/cloud/example_dags/example_compute.py | 1 - .../google/cloud/example_dags/example_compute_ssh.py | 1 - .../google/cloud/example_dags/example_dataflow.py | 4 ---- .../example_dags/example_dataflow_flex_template.py | 1 - .../cloud/example_dags/example_dataflow_sql.py | 1 - .../google/cloud/example_dags/example_datafusion.py | 1 - .../google/cloud/example_dags/example_dataplex.py | 1 - .../google/cloud/example_dags/example_dataprep.py | 1 - .../google/cloud/example_dags/example_dlp.py | 4 ---- .../example_dags/example_facebook_ads_to_gcs.py | 1 - .../google/cloud/example_dags/example_functions.py | 1 - .../cloud/example_dags/example_gdrive_to_gcs.py | 1 - .../cloud/example_dags/example_gdrive_to_local.py | 1 - .../google/cloud/example_dags/example_looker.py | 1 - .../google/cloud/example_dags/example_mlengine.py | 1 - .../cloud/example_dags/example_postgres_to_gcs.py | 1 - .../cloud/example_dags/example_presto_to_gcs.py | 1 - .../cloud/example_dags/example_salesforce_to_gcs.py | 1 - .../google/cloud/example_dags/example_sftp_to_gcs.py | 1 - .../google/cloud/example_dags/example_vertex_ai.py | 7 ------- .../cloud/example_dags/example_video_intelligence.py | 1 - .../google/cloud/example_dags/example_vision.py | 3 --- .../example_dags/example_display_video.py | 3 --- .../suite/example_dags/example_local_to_drive.py | 1 - scripts/in_container/verify_providers.py | 12 ++++++++++++ 67 files changed, 12 insertions(+), 87 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index 5ce6a7b6a1eeb..155521912db5e 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -34,7 +34,6 @@ with DAG( "example_appflow", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2022, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/amazon/aws/example_dags/example_dms.py b/airflow/providers/amazon/aws/example_dags/example_dms.py index e3d0b247bca17..77bcbe3ef22aa 100644 --- a/airflow/providers/amazon/aws/example_dags/example_dms.py +++ b/airflow/providers/amazon/aws/example_dags/example_dms.py @@ -227,7 +227,6 @@ def delete_dms_assets(): with DAG( dag_id='example_dms', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py index a3d05055b0a13..5abf23d29cf62 100644 --- a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py @@ -27,7 +27,6 @@ with DAG( dag_id='example_dynamodb_to_s3', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ec2.py b/airflow/providers/amazon/aws/example_dags/example_ec2.py index fc364a5223cbd..5c6b04301d25b 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ec2.py +++ b/airflow/providers/amazon/aws/example_dags/example_ec2.py @@ -26,7 +26,6 @@ with DAG( dag_id='example_ec2', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ecs.py b/airflow/providers/amazon/aws/example_dags/example_ecs.py index 16f569a6be925..3f14c11c76970 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ecs.py +++ b/airflow/providers/amazon/aws/example_dags/example_ecs.py @@ -53,7 +53,6 @@ with DAG( dag_id=DAG_ID, - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_templated.py b/airflow/providers/amazon/aws/example_dags/example_eks_templated.py index f1b6ff3bca2d9..0d9276e499b21 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_templated.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_templated.py @@ -49,7 +49,6 @@ with DAG( dag_id='example_eks_templated', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example', 'templated'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py index 8c7854be6f82f..b86c3f73431af 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py @@ -44,7 +44,6 @@ with DAG( dag_id='example_eks_with_fargate_in_one_step', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py index fdf599008903a..8cb247035a907 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py @@ -47,7 +47,6 @@ with DAG( dag_id='example_eks_with_fargate_profile', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py index bb39825d0566b..aaf84cf3ef71c 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py @@ -43,7 +43,6 @@ with DAG( dag_id='example_eks_with_nodegroup_in_one_step', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py index ba333ab3d0340..eb408af043894 100644 --- a/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py +++ b/airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py @@ -46,7 +46,6 @@ with DAG( dag_id='example_eks_with_nodegroups', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_emr.py b/airflow/providers/amazon/aws/example_dags/example_emr.py index 0d9bf0cc640c1..9ddeb5498d2fe 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr.py @@ -68,7 +68,6 @@ with DAG( dag_id='example_emr', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_eks.py b/airflow/providers/amazon/aws/example_dags/example_emr_eks.py index 66222611a2ad7..0e7774f81dddf 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr_eks.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr_eks.py @@ -54,7 +54,6 @@ with DAG( dag_id='example_emr_eks', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py index f5d4318c919e3..9e95400391853 100644 --- a/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_ftp_to_s3.py @@ -27,7 +27,6 @@ with models.DAG( "example_ftp_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py index ea52b400ff46d..e45089cf57928 100644 --- a/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_gcs_to_s3.py @@ -26,7 +26,6 @@ with DAG( dag_id="example_gcs_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=["example"], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py index 05591b9e3e7e3..0f3aaf9e8e798 100644 --- a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py +++ b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py @@ -28,7 +28,6 @@ with DAG( "example_glacier_to_gcs", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), # Override to match your needs catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py index c5705a1dd8a60..cc83a6d1e701f 100644 --- a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py @@ -31,7 +31,6 @@ with DAG( dag_id="example_google_api_sheets_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py index 8ff9874cf1247..869276c67d043 100644 --- a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py @@ -66,7 +66,6 @@ def transform_video_ids(**kwargs): with DAG( dag_id="example_google_api_youtube_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py b/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py index ac20a9cec7f21..b033dbb80c615 100644 --- a/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py +++ b/airflow/providers/amazon/aws/example_dags/example_hive_to_dynamodb.py @@ -105,7 +105,6 @@ def configure_hive_connection(): with DAG( dag_id='example_hive_to_dynamodb', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=['example'], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py index bb355468a3bf5..a84f7ac1dd410 100644 --- a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py @@ -35,7 +35,6 @@ with DAG( dag_id="example_imap_attachment_to_s3", start_date=datetime(2021, 1, 1), - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 catchup=False, tags=['example'], ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py index fc4218410da86..3b23c4651c2b2 100644 --- a/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_mongo_to_s3.py @@ -29,7 +29,6 @@ with models.DAG( "example_mongo_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_quicksight.py b/airflow/providers/amazon/aws/example_dags/example_quicksight.py index 27b93971b65fc..9efba12d9f42e 100644 --- a/airflow/providers/amazon/aws/example_dags/example_quicksight.py +++ b/airflow/providers/amazon/aws/example_dags/example_quicksight.py @@ -28,7 +28,6 @@ with DAG( dag_id="example_quicksight", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), tags=["example"], catchup=False, diff --git a/airflow/providers/amazon/aws/example_dags/example_s3.py b/airflow/providers/amazon/aws/example_dags/example_s3.py index be0cc3b1e965d..33b90877e4fbb 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3.py @@ -57,7 +57,6 @@ with DAG( dag_id='example_s3', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py index b4298671b22a2..3f8c1b0ec33bd 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_ftp.py @@ -27,7 +27,6 @@ with models.DAG( "example_s3_to_ftp", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py index aa1856547b0e7..5cd224573d640 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py @@ -46,7 +46,6 @@ def task_remove_sample_data_from_s3(): with DAG( dag_id="example_s3_to_redshift", start_date=datetime(2021, 1, 1), - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 catchup=False, tags=['example'], ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py index 7cc1cc6d545e7..a544866e020b1 100644 --- a/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py +++ b/airflow/providers/amazon/aws/example_dags/example_s3_to_sftp.py @@ -26,7 +26,6 @@ with models.DAG( "example_s3_to_sftp", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py index 5438393e83e14..78864578cea9c 100644 --- a/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_salesforce_to_s3.py @@ -32,7 +32,6 @@ with DAG( dag_id="example_salesforce_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 7, 8), catchup=False, tags=["example"], diff --git a/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py index 8d34671a52002..d67ef54b68eeb 100644 --- a/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_sftp_to_s3.py @@ -27,7 +27,6 @@ with models.DAG( "example_sftp_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py index 84d4ddfee1893..47abd74ca3cb1 100644 --- a/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py +++ b/airflow/providers/amazon/aws/example_dags/example_sql_to_s3.py @@ -28,7 +28,6 @@ with models.DAG( "example_sql_to_s3", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py index 4b93bb02fbb54..cfa1eb17c88b8 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py @@ -59,7 +59,6 @@ # Example DAG for AutoML Natural Language Text Classification with models.DAG( "example_automl_text_cls", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py index dabaf7f16486a..a85ce801417e4 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py @@ -59,7 +59,6 @@ # Example DAG for AutoML Natural Language Text Sentiment with models.DAG( "example_automl_text_sentiment", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_tables.py b/airflow/providers/google/cloud/example_dags/example_automl_tables.py index 6f5af0e3d0445..53de60ca36a3b 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_tables.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_tables.py @@ -86,7 +86,6 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: # Example DAG to create dataset, train model_id and deploy it. with models.DAG( "example_create_and_deploy", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, user_defined_macros={ @@ -199,7 +198,6 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: # Example DAG for AutoML datasets operations with models.DAG( "example_automl_dataset", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, @@ -268,7 +266,6 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: with models.DAG( "example_gcp_get_deploy", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], @@ -294,7 +291,6 @@ def get_target_column_spec(columns_specs: List[Dict], column_name: str) -> str: with models.DAG( "example_gcp_predict", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_automl_translation.py b/airflow/providers/google/cloud/example_dags/example_automl_translation.py index e033165fbb8a7..96370c7b9564d 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_translation.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_translation.py @@ -65,7 +65,6 @@ # Example DAG for AutoML Translation with models.DAG( "example_automl_translation", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py index e1f4fbf4972e0..f98be55bb0e39 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py @@ -62,7 +62,6 @@ # Example DAG for AutoML Video Intelligence Classification with models.DAG( "example_automl_video", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py index 1fa804b963e39..6c3aacdf3b62a 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py @@ -63,7 +63,6 @@ # Example DAG for AutoML Video Intelligence Object Tracking with models.DAG( "example_automl_video_tracking", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py index 442d01875fe4b..5cf0299e823c6 100644 --- a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py +++ b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py @@ -62,7 +62,6 @@ # Example DAG for AutoML Vision Object Detection with models.DAG( "example_automl_vision_detection", - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, user_defined_macros={"extract_object_id": extract_object_id}, diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py index f7dba38a156bf..06380827f5450 100644 --- a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py +++ b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py @@ -64,7 +64,6 @@ with models.DAG( "example_gcp_bigquery_dts", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_bigtable.py b/airflow/providers/google/cloud/example_dags/example_bigtable.py index f2e6f7afa5053..c2f41ae8cff70 100644 --- a/airflow/providers/google/cloud/example_dags/example_bigtable.py +++ b/airflow/providers/google/cloud/example_dags/example_bigtable.py @@ -80,7 +80,6 @@ with models.DAG( 'example_gcp_bigtable_operators', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_build.py b/airflow/providers/google/cloud/example_dags/example_cloud_build.py index 98e383b67c899..33f7e74d40abe 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_build.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_build.py @@ -116,7 +116,6 @@ with models.DAG( "example_gcp_cloud_build", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], @@ -205,7 +204,6 @@ with models.DAG( "example_gcp_cloud_build_trigger", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_composer.py b/airflow/providers/google/cloud/example_dags/example_cloud_composer.py index 2e22b0f068fe2..32a06633223a7 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_composer.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_composer.py @@ -55,7 +55,6 @@ with models.DAG( "composer_dag1", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], @@ -118,7 +117,6 @@ with models.DAG( "composer_dag_deferrable1", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py index 409ce5f3bc497..edc6d6778710b 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py @@ -270,7 +270,6 @@ def get_absolute_path(path): with models.DAG( dag_id='example_gcp_sql_query', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py index e6ef42f72c579..9e4c7e334b428 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py @@ -106,7 +106,6 @@ with models.DAG( 'example_gcp_transfer_aws', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py index 263277c3e014a..116a89a2c41d4 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py @@ -101,7 +101,6 @@ with models.DAG( "example_gcp_transfer", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_compute.py b/airflow/providers/google/cloud/example_dags/example_compute.py index 05b3081ddd5a5..afdf897102564 100644 --- a/airflow/providers/google/cloud/example_dags/example_compute.py +++ b/airflow/providers/google/cloud/example_dags/example_compute.py @@ -52,7 +52,6 @@ with models.DAG( 'example_gcp_compute', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_compute_ssh.py b/airflow/providers/google/cloud/example_dags/example_compute_ssh.py index 73a225bb49f63..576dc1d90afbc 100644 --- a/airflow/providers/google/cloud/example_dags/example_compute_ssh.py +++ b/airflow/providers/google/cloud/example_dags/example_compute_ssh.py @@ -30,7 +30,6 @@ with models.DAG( 'example_compute_ssh', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow.py b/airflow/providers/google/cloud/example_dags/example_dataflow.py index 06c75464d6907..bba66949fb4f8 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow.py @@ -64,7 +64,6 @@ with models.DAG( "example_gcp_dataflow_native_java", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=['example'], @@ -115,7 +114,6 @@ default_args=default_args, start_date=START_DATE, catchup=False, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_native_python: @@ -151,7 +149,6 @@ default_args=default_args, start_date=START_DATE, catchup=False, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_native_python_async: # [START howto_operator_start_python_job_async] @@ -253,7 +250,6 @@ def check_autoscaling_event(autoscaling_events: List[dict]) -> bool: default_args=default_args, start_date=START_DATE, catchup=False, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_template: # [START howto_operator_start_template_job] diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py b/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py index 69713304a3e5e..650fcc5162757 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow_flex_template.py @@ -47,7 +47,6 @@ dag_id="example_gcp_dataflow_flex_template_java", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 ) as dag_flex_template: # [START howto_operator_start_template_job] start_flex_template = DataflowStartFlexTemplateOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py b/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py index c04941be43221..26e494494b0aa 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py +++ b/airflow/providers/google/cloud/example_dags/example_dataflow_sql.py @@ -38,7 +38,6 @@ dag_id="example_gcp_dataflow_sql", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=['example'], ) as dag_sql: # [START howto_operator_start_sql_job] diff --git a/airflow/providers/google/cloud/example_dags/example_datafusion.py b/airflow/providers/google/cloud/example_dags/example_datafusion.py index d4f6861c84ebe..afadb1926279e 100644 --- a/airflow/providers/google/cloud/example_dags/example_datafusion.py +++ b/airflow/providers/google/cloud/example_dags/example_datafusion.py @@ -147,7 +147,6 @@ with models.DAG( "example_data_fusion", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_dataplex.py b/airflow/providers/google/cloud/example_dags/example_dataplex.py index 3fcc135200605..996228316cfe3 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataplex.py +++ b/airflow/providers/google/cloud/example_dags/example_dataplex.py @@ -53,7 +53,6 @@ with models.DAG( "example_dataplex", start_date=datetime.datetime(2021, 1, 1), - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 ) as dag: # [START howto_dataplex_create_task_operator] create_dataplex_task = DataplexCreateTaskOperator( diff --git a/airflow/providers/google/cloud/example_dags/example_dataprep.py b/airflow/providers/google/cloud/example_dags/example_dataprep.py index 19b5f5da2384f..57e5122de9e76 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataprep.py +++ b/airflow/providers/google/cloud/example_dags/example_dataprep.py @@ -52,7 +52,6 @@ with models.DAG( "example_dataprep", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), # Override to match your needs catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_dlp.py b/airflow/providers/google/cloud/example_dags/example_dlp.py index 883903e0513ba..c843695d643b1 100644 --- a/airflow/providers/google/cloud/example_dags/example_dlp.py +++ b/airflow/providers/google/cloud/example_dags/example_dlp.py @@ -63,7 +63,6 @@ with models.DAG( "example_gcp_dlp", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=['example'], @@ -113,7 +112,6 @@ with models.DAG( "example_gcp_dlp_info_types", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example", "dlp", "info-types"], @@ -157,7 +155,6 @@ with models.DAG( "example_gcp_dlp_job", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example", "dlp_job"], @@ -203,7 +200,6 @@ with models.DAG( "example_gcp_dlp_deidentify_content", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, tags=["example", "dlp", "deidentify"], diff --git a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py index 28d3bcfe0de35..5cf520557e8b2 100644 --- a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py @@ -57,7 +57,6 @@ with models.DAG( "example_facebook_ads_to_gcs", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_functions.py b/airflow/providers/google/cloud/example_dags/example_functions.py index 1d10c38a5a2cd..12b255627cfa1 100644 --- a/airflow/providers/google/cloud/example_dags/example_functions.py +++ b/airflow/providers/google/cloud/example_dags/example_functions.py @@ -94,7 +94,6 @@ with models.DAG( 'example_gcp_function', default_args=default_args, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py index dcbc5c1a8b223..f2ccd9365886e 100644 --- a/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_gdrive_to_gcs.py @@ -32,7 +32,6 @@ "example_gdrive_to_gcs_with_gdrive_sensor", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 tags=["example"], ) as dag: # [START detect_file] diff --git a/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py b/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py index 1ec7d19e021cc..7ac2e14f91287 100644 --- a/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py +++ b/airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py @@ -31,7 +31,6 @@ "example_gdrive_to_local_with_gdrive_sensor", start_date=datetime(2021, 1, 1), catchup=False, - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 tags=["example"], ) as dag: # [START detect_file] diff --git a/airflow/providers/google/cloud/example_dags/example_looker.py b/airflow/providers/google/cloud/example_dags/example_looker.py index c3f0f19cc5866..ac583c6a1cd3f 100644 --- a/airflow/providers/google/cloud/example_dags/example_looker.py +++ b/airflow/providers/google/cloud/example_dags/example_looker.py @@ -28,7 +28,6 @@ with models.DAG( dag_id='example_gcp_looker', - schedule_interval=None, # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_mlengine.py b/airflow/providers/google/cloud/example_dags/example_mlengine.py index 3c574e4767cdf..35862aedb1254 100644 --- a/airflow/providers/google/cloud/example_dags/example_mlengine.py +++ b/airflow/providers/google/cloud/example_dags/example_mlengine.py @@ -59,7 +59,6 @@ with models.DAG( "example_gcp_mlengine", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py index b2eb28944d8ba..c967c745769e6 100644 --- a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py @@ -31,7 +31,6 @@ with models.DAG( dag_id='example_postgres_to_gcs', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py index bf6dec9d9f7fe..2d939f02843aa 100644 --- a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py @@ -48,7 +48,6 @@ def safe_name(s: str) -> str: with models.DAG( dag_id="example_presto_to_gcs", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py index dafeb795e11d0..d3ef59f18cc4b 100644 --- a/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_salesforce_to_gcs.py @@ -44,7 +44,6 @@ with models.DAG( "example_salesforce_to_gcs", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py index df956b46e8dd6..527c81837d5a0 100644 --- a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py +++ b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py @@ -38,7 +38,6 @@ with models.DAG( "example_sftp_to_gcs", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dag: diff --git a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py index cd0b14f66e8a3..c2a5cdbf7d52e 100644 --- a/airflow/providers/google/cloud/example_dags/example_vertex_ai.py +++ b/airflow/providers/google/cloud/example_dags/example_vertex_ai.py @@ -238,7 +238,6 @@ with models.DAG( "example_gcp_vertex_ai_custom_jobs", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as custom_jobs_dag: @@ -328,7 +327,6 @@ with models.DAG( "example_gcp_vertex_ai_dataset", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as dataset_dag: @@ -431,7 +429,6 @@ with models.DAG( "example_gcp_vertex_ai_auto_ml", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as auto_ml_dag: @@ -548,7 +545,6 @@ with models.DAG( "example_gcp_vertex_ai_batch_prediction_job", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as batch_prediction_job_dag: @@ -588,7 +584,6 @@ with models.DAG( "example_gcp_vertex_ai_endpoint", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as endpoint_dag: @@ -644,7 +639,6 @@ with models.DAG( "example_gcp_vertex_ai_hyperparameter_tuning_job", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as hyperparameter_tuning_job_dag: @@ -717,7 +711,6 @@ with models.DAG( "example_gcp_vertex_ai_model_service", - schedule_interval="@once", # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, ) as model_service_dag: diff --git a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py index 59fb8ba110c43..9e8183e78545c 100644 --- a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py +++ b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py @@ -49,7 +49,6 @@ with models.DAG( "example_gcp_video_intelligence", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=['example'], diff --git a/airflow/providers/google/cloud/example_dags/example_vision.py b/airflow/providers/google/cloud/example_dags/example_vision.py index 552a7ed5efe87..24bc094e02845 100644 --- a/airflow/providers/google/cloud/example_dags/example_vision.py +++ b/airflow/providers/google/cloud/example_dags/example_vision.py @@ -118,7 +118,6 @@ with models.DAG( 'example_gcp_vision_autogenerated_id', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag_autogenerated_id: @@ -276,7 +275,6 @@ with models.DAG( 'example_gcp_vision_explicit_id', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag_explicit_id: @@ -444,7 +442,6 @@ with models.DAG( 'example_gcp_vision_annotate_image', - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag_annotate_image: diff --git a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py index 47f07c0297b73..4ddedf8f35826 100644 --- a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py +++ b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py @@ -86,7 +86,6 @@ with models.DAG( "example_display_video", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag1: @@ -129,7 +128,6 @@ with models.DAG( "example_display_video_misc", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag2: @@ -163,7 +161,6 @@ with models.DAG( "example_display_video_sdf", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=START_DATE, catchup=False, ) as dag3: diff --git a/airflow/providers/google/suite/example_dags/example_local_to_drive.py b/airflow/providers/google/suite/example_dags/example_local_to_drive.py index e5171be54bc95..27a30b24475e2 100644 --- a/airflow/providers/google/suite/example_dags/example_local_to_drive.py +++ b/airflow/providers/google/suite/example_dags/example_local_to_drive.py @@ -31,7 +31,6 @@ with models.DAG( "example_local_to_drive", - schedule_interval='@once', # TODO: replace with `schedule=` once min ver >= 2.4 start_date=datetime(2021, 1, 1), catchup=False, tags=["example"], diff --git a/scripts/in_container/verify_providers.py b/scripts/in_container/verify_providers.py index 366f90fa49104..bf9ddb327819c 100755 --- a/scripts/in_container/verify_providers.py +++ b/scripts/in_container/verify_providers.py @@ -178,6 +178,17 @@ class ProviderPackageDetails(NamedTuple): ("This module is deprecated. Please use `airflow.operators.empty`.", "winrm"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "dbt"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "jdbc"), + *[ + ( + "Param `schedule_interval` is deprecated and will be removed in a future release. " + "Please use `schedule` instead.", + x, + ) + for x in ( + 'amazon', + 'google', + ) + ], ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "azure"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "qubole"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "winrm"), @@ -200,6 +211,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", } # The set of warning messages generated by direct importing of some deprecated modules. We should only From c1d0c34c04b1320ea1c82883c6c1122235f0514f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 13:01:55 -0700 Subject: [PATCH 20/38] fixup! remove schedule_interval param from example dags --- scripts/in_container/verify_providers.py | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/scripts/in_container/verify_providers.py b/scripts/in_container/verify_providers.py index bf9ddb327819c..49b17dbeb0db6 100755 --- a/scripts/in_container/verify_providers.py +++ b/scripts/in_container/verify_providers.py @@ -178,17 +178,6 @@ class ProviderPackageDetails(NamedTuple): ("This module is deprecated. Please use `airflow.operators.empty`.", "winrm"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "dbt"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "jdbc"), - *[ - ( - "Param `schedule_interval` is deprecated and will be removed in a future release. " - "Please use `schedule` instead.", - x, - ) - for x in ( - 'amazon', - 'google', - ) - ], ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "azure"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "qubole"), ("This class is deprecated. Please use `airflow.operators.empty.EmptyOperator`.", "winrm"), From 7bc7f762bffbddfab94c24413a9304ba2f890599 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 13:53:15 -0700 Subject: [PATCH 21/38] fix test --- .../google/cloud/example_dags/example_dataproc_metastore.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py b/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py index 0db1d33825f73..8f56e02f3479e 100644 --- a/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py +++ b/airflow/providers/google/cloud/example_dags/example_dataproc_metastore.py @@ -94,7 +94,8 @@ with models.DAG( - "example_gcp_dataproc_metastore", start_date=datetime.datetime(2021, 1, 1), schedule="@once" + dag_id="example_gcp_dataproc_metastore", + start_date=datetime.datetime(2021, 1, 1), ) as dag: # [START how_to_cloud_dataproc_metastore_create_service_operator] create_service = DataprocMetastoreCreateServiceOperator( From 721881e59fe4674b9fd84ebea8e4d9b480b0d4ce Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:00:59 -0700 Subject: [PATCH 22/38] Update scripts/in_container/verify_providers.py Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- scripts/in_container/verify_providers.py | 1 - 1 file changed, 1 deletion(-) diff --git a/scripts/in_container/verify_providers.py b/scripts/in_container/verify_providers.py index 49b17dbeb0db6..366f90fa49104 100755 --- a/scripts/in_container/verify_providers.py +++ b/scripts/in_container/verify_providers.py @@ -200,7 +200,6 @@ 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", } # The set of warning messages generated by direct importing of some deprecated modules. We should only From 6d00cefe895dcd1d3615ce2b3e7d20945a90b2e1 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:02:31 -0700 Subject: [PATCH 23/38] Update docs/apache-airflow/faq.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- docs/apache-airflow/faq.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index e34aa592c215d..0d4f7e270b48e 100644 --- a/docs/apache-airflow/faq.rst +++ b/docs/apache-airflow/faq.rst @@ -41,7 +41,7 @@ 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 won't trigger the task until the +- Is your ``start_date`` set properly? The Airflow scheduler won't trigger the task until after the first schedule interval following the start date has passed. - Is your ``schedule`` param set properly? The default From fd808bd935c734d7e6ad98c7e002ef0379efd4c5 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:04:29 -0700 Subject: [PATCH 24/38] Update docs/apache-airflow/dag-run.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- docs/apache-airflow/dag-run.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/dag-run.rst index 974c5bd5fa622..1ec655b8c71b5 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/docs/apache-airflow/dag-run.rst @@ -116,7 +116,7 @@ DAG run fails. Catchup ------- -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. +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. The scheduler, by default, will kick off a DAG Run for any data interval that has not been run since the last data interval (or has been cleared). This concept is called Catchup. From fd3d9e97d444eccd0c38633414cca300147ad9e4 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:04:49 -0700 Subject: [PATCH 25/38] Update docs/apache-airflow/dag-run.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- docs/apache-airflow/dag-run.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/dag-run.rst index 1ec655b8c71b5..721fbb4131f42 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/docs/apache-airflow/dag-run.rst @@ -81,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 -ts data interval would start each day at midnight (00:00) and end at midnight +its data interval would start each day at midnight (00:00) and end at midnight (24:00). A DAG run is usually scheduled *after* its associated data interval has ended, From b49c39ba92d9b3a6729525c4f58770fa30a9a290 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:18:27 -0700 Subject: [PATCH 26/38] add example --- docs/apache-airflow/faq.rst | 4 +-- newsfragments/25410.significant.rst | 40 +++++++++++++++++++++++++++++ 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index 0d4f7e270b48e..7356d67ff7ffe 100644 --- a/docs/apache-airflow/faq.rst +++ b/docs/apache-airflow/faq.rst @@ -41,8 +41,8 @@ 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 won't trigger the task until - after the first schedule interval following the start date has 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`` param set properly? The default is one day (``datetime.timedelta(1)``). You must specify a different ``schedule`` diff --git a/newsfragments/25410.significant.rst b/newsfragments/25410.significant.rst index 6a3f82f2de546..0a0c12887b23f 100644 --- a/newsfragments/25410.significant.rst +++ b/newsfragments/25410.significant.rst @@ -1,3 +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, 8, 27, tz="America/Chicago")]), + ) 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, 8, 27, tz="America/Chicago")]), + ) as dag: + ... From 2509e8ee8f9edcb5776297bc8086feb31a96ece2 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:21:47 -0700 Subject: [PATCH 27/38] simplify example --- newsfragments/25410.significant.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/newsfragments/25410.significant.rst b/newsfragments/25410.significant.rst index 0a0c12887b23f..cfdb7f4eca2e0 100644 --- a/newsfragments/25410.significant.rst +++ b/newsfragments/25410.significant.rst @@ -28,7 +28,7 @@ And the same is true if you used a custom timetable. Previously you would have with DAG( dag_id='my_example', start_date=datetime(2021, 1, 1), - timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5, 8, 27, tz="America/Chicago")]), + timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]), ) as dag: ... @@ -38,6 +38,6 @@ Now you should use the ``schedule`` param: with DAG( dag_id='my_example', start_date=datetime(2021, 1, 1), - timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5, 8, 27, tz="America/Chicago")]), + timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]), ) as dag: ... From 17531f8e4925eb1cb55812fe1bd8369fc2212d90 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:34:57 -0700 Subject: [PATCH 28/38] Update newsfragments/25410.significant.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- newsfragments/25410.significant.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/newsfragments/25410.significant.rst b/newsfragments/25410.significant.rst index cfdb7f4eca2e0..6162f3a2aba0c 100644 --- a/newsfragments/25410.significant.rst +++ b/newsfragments/25410.significant.rst @@ -38,6 +38,6 @@ Now you should use the ``schedule`` param: with DAG( dag_id='my_example', start_date=datetime(2021, 1, 1), - timetable=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]), + schedule=EventsTimetable(event_dates=[pendulum.datetime(2022, 4, 5)]), ) as dag: ... From 8e77857f0afce86720e0a9c8c739f0555156eb83 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 17:34:26 -0700 Subject: [PATCH 29/38] add test --- tests/models/test_dag.py | 29 ++++++++++++++++++++++++----- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index f83f1becf707d..c273848bfd5da 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -56,7 +56,7 @@ from airflow.security import permissions from airflow.templates import NativeEnvironment, SandboxedEnvironment from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable -from airflow.timetables.simple import NullTimetable, OnceTimetable +from airflow.timetables.simple import DatasetTriggeredTimetable, NullTimetable, OnceTimetable from airflow.utils import timezone from airflow.utils.file import list_py_file_paths from airflow.utils.session import create_session, provide_session @@ -1351,14 +1351,20 @@ def test_get_paused_dag_ids(self): ("30 21 * * 5 1", cron_timetable("30 21 * * 5 1"), ""), ] ) - def test_timetable_and_description_from_schedule_interval( - self, schedule_interval, expected_timetable, interval_description + def test_timetable_and_description_from_schedule_interval_arg( + self, schedule_interval_arg, expected_timetable, interval_description ): - dag = DAG("test_schedule_interval", schedule=schedule_interval) + dag = DAG("test_schedule_interval_arg", schedule=schedule_interval_arg) assert dag.timetable == expected_timetable - assert dag.schedule_interval == schedule_interval + assert dag.schedule_interval == schedule_interval_arg assert dag.timetable.description == interval_description + def test_timetable_and_description_from_dataset(self): + dag = DAG("test_schedule_interval_arg", schedule=[Dataset(uri='hello')]) + assert dag.timetable == DatasetTriggeredTimetable() + assert dag.schedule_interval == 'Dataset' + assert dag.timetable.description == 'Triggered by datasets' + @parameterized.expand( [ (NullTimetable(), "Never, external triggers only"), @@ -2018,6 +2024,19 @@ def test_dag_owner_links(self): with pytest.raises(AirflowException): DAG('dag', start_date=DEFAULT_DATE, owner_links={"owner1": "my-bad-link"}) + def test_schedule_dag_param(self): + with pytest.raises(ValueError, match='At most one'): + with DAG(dag_id='hello', schedule_interval='@daily', schedule='@daily'): + pass + with pytest.raises(ValueError, match='At most one'): + with DAG(dag_id='hello', timetable=NullTimetable(), schedule=NullTimetable()): + pass + with pytest.raises(ValueError, match='At most one'): + with DAG(dag_id='hello', timetable=NullTimetable(), schedule_interval='@daily'): + pass + with DAG(dag_id='hello', timetable=NullTimetable(), schedule_interval='@daily'): + pass + class TestDagModel: def test_dags_needing_dagruns_not_too_early(self): From 46ee72c29ee7f2ece2814584bb981e63e8ff0be3 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 9 Aug 2022 12:20:15 +0800 Subject: [PATCH 30/38] Simplify and fix test --- tests/models/test_dag.py | 31 ++++++++++++++++++------------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index c273848bfd5da..12ec4b34788ba 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -81,14 +81,14 @@ def session(): session.rollback() -class TestDag(unittest.TestCase): - def setUp(self) -> None: +class TestDag: + def setup_method(self) -> None: clear_db_runs() clear_db_dags() self.patcher_dag_code = mock.patch('airflow.models.dag.DagCode.bulk_sync_to_db') self.patcher_dag_code.start() - def tearDown(self) -> None: + def teardown_method(self) -> None: clear_db_runs() clear_db_dags() self.patcher_dag_code.stop() @@ -2024,18 +2024,23 @@ def test_dag_owner_links(self): with pytest.raises(AirflowException): DAG('dag', start_date=DEFAULT_DATE, owner_links={"owner1": "my-bad-link"}) - def test_schedule_dag_param(self): - with pytest.raises(ValueError, match='At most one'): - with DAG(dag_id='hello', schedule_interval='@daily', schedule='@daily'): - pass - with pytest.raises(ValueError, match='At most one'): - with DAG(dag_id='hello', timetable=NullTimetable(), schedule=NullTimetable()): - pass + @pytest.mark.parametrize( + "kwargs", + [ + {"schedule_interval": "@daily", "schedule": "@weekly"}, + {"timetable": NullTimetable(), "schedule": "@weekly"}, + {"timetable": NullTimetable(), "schedule_interval": "@daily"}, + ], + ids=[ + "schedule_interval+schedule", + "timetable+schedule", + "timetable+schedule_interval", + ], + ) + def test_schedule_dag_param(self, kwargs): with pytest.raises(ValueError, match='At most one'): - with DAG(dag_id='hello', timetable=NullTimetable(), schedule_interval='@daily'): + with DAG(dag_id='hello', **kwargs): pass - with DAG(dag_id='hello', timetable=NullTimetable(), schedule_interval='@daily'): - pass class TestDagModel: From babf5c564b8ad00ef8c2e1cec06737663c603797 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 22:02:43 -0700 Subject: [PATCH 31/38] uppercase --- tests/test_utils/perf/dags/elastic_dag.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/test_utils/perf/dags/elastic_dag.py b/tests/test_utils/perf/dags/elastic_dag.py index fef48cfe8f5b7..45ac1cb05ecfb 100644 --- a/tests/test_utils/perf/dags/elastic_dag.py +++ b/tests/test_utils/perf/dags/elastic_dag.py @@ -175,7 +175,7 @@ class DagShape(Enum): f"DAGS_COUNT={dag_no}_of_{DAG_COUNT}", f"TASKS_COUNT=${TASKS_COUNT}", f"START_DATE=${START_DATE_ENV}", - f"schedule=${SCHEDULE_INTERVAL_ENV}", + f"SCHEDULE=${SCHEDULE_INTERVAL_ENV}", ] ) ), From 7858b38a20e2173a859a81ab92023c4429294ed9 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 8 Aug 2022 22:22:00 -0700 Subject: [PATCH 32/38] fix log testing --- tests/models/test_dag.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 12ec4b34788ba..57fb307398a18 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -1833,7 +1833,7 @@ def test_next_dagrun_info_timedelta_schedule_and_catchup_true(self): next_info = dag.next_dagrun_info(next_info.data_interval) assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 4) - def test_next_dagrun_info_timetable_exception(self): + def test_next_dagrun_info_timetable_exception(self, caplog): """Test the DAG does not crash the scheduler if the timetable raises an exception.""" class FailingTimetable(Timetable): @@ -1856,16 +1856,16 @@ def _check_logs(records: List[logging.LogRecord], data_interval: DataInterval) - f"for DAG 'test_next_dagrun_info_timetable_exception'" ) - with self.assertLogs(dag.log, level=logging.ERROR) as ctx: + with caplog.at_level(level=logging.ERROR): next_info = dag.next_dagrun_info(None) assert next_info is None, "failed next_dagrun_info should return None" - _check_logs(ctx.records, data_interval=None) - + _check_logs(caplog.records, data_interval=None) + caplog.clear() data_interval = DataInterval(timezone.datetime(2020, 5, 1), timezone.datetime(2020, 5, 2)) - with self.assertLogs(dag.log, level=logging.ERROR) as ctx: + with caplog.at_level(level=logging.ERROR): next_info = dag.next_dagrun_info(data_interval) assert next_info is None, "failed next_dagrun_info should return None" - _check_logs(ctx.records, data_interval) + _check_logs(caplog.records, data_interval) def test_next_dagrun_after_auto_align(self): """ From e5e50a32266b43f240045c2df5702f3d3382634e Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 9 Aug 2022 14:09:09 +0800 Subject: [PATCH 33/38] Unify comment format --- tests/cli/commands/test_dag_command.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/cli/commands/test_dag_command.py b/tests/cli/commands/test_dag_command.py index d175df2ed6e64..3544769ff0b2e 100644 --- a/tests/cli/commands/test_dag_command.py +++ b/tests/cli/commands/test_dag_command.py @@ -352,11 +352,11 @@ def test_cli_backfill_depends_on_past_backwards(self, mock_run): def test_next_execution(self): dag_ids = [ - 'example_bash_operator', # schedule_interval is '0 0 * * *' - 'latest_only', # schedule_interval is timedelta(hours=4) + 'example_bash_operator', # schedule='0 0 * * *' + 'latest_only', # schedule=timedelta(hours=4) 'example_python_operator', # schedule=None - 'example_xcom', - ] # schedule="@once" + 'example_xcom', # schedule="@once" + ] # Delete DagRuns with create_session() as session: From eaca727a14dac507d105ab5c1b37184aaae17341 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 9 Aug 2022 14:35:02 +0800 Subject: [PATCH 34/38] Fix casing an wording Most significantly, use "argument" instead of "param" or "parameter" where applicable, to avoid possible confusion, since we already have a DAG argument called "params". --- docs/apache-airflow/concepts/datasets.rst | 4 ++-- docs/apache-airflow/concepts/tasks.rst | 2 +- docs/apache-airflow/concepts/timetable.rst | 10 +++++----- newsfragments/25410.significant.rst | 12 ++++++------ 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/apache-airflow/concepts/datasets.rst b/docs/apache-airflow/concepts/datasets.rst index de24594991acc..4b0bd595d7e0d 100644 --- a/docs/apache-airflow/concepts/datasets.rst +++ b/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`` parameter: +Finally, define a DAG and reference this dataset in the DAG's ``schedule`` argument: .. 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`` 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`` 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. diff --git a/docs/apache-airflow/concepts/tasks.rst b/docs/apache-airflow/concepts/tasks.rst index c48cad4d80f0c..63fbe818e0167 100644 --- a/docs/apache-airflow/concepts/tasks.rst +++ b/docs/apache-airflow/concepts/tasks.rst @@ -170,7 +170,7 @@ To read more about configuring the emails, see :doc:`/howto/email-config`. .. note:: - 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 `. + 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 `. .. _concepts:sla_miss_callback: diff --git a/docs/apache-airflow/concepts/timetable.rst b/docs/apache-airflow/concepts/timetable.rst index b9f4f3a12ebac..9e1a9e0bf3355 100644 --- a/docs/apache-airflow/concepts/timetable.rst +++ b/docs/apache-airflow/concepts/timetable.rst @@ -19,16 +19,16 @@ Timetables ========== -For dags with time-based schedules (as opposed to event-driven), the scheduling -decisions are driven by its internal "timetable". The timetable class also +For DAGs with time-based schedules (as opposed to event-driven), the scheduling +decisions are driven by its internal "timetable". The timetable also determines the data interval and the logical date of each run created for the DAG. -Even for dags whose scheduling is defined by a cron expression or ``timedelta`` object, -internally these expressions converted to a timetable. +DAGs scheduled with a cron expression or ``timedelta`` object are +internally converted to always use 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. +you may create your own timetable class and pass that to the DAG's ``schedule`` argument instead. Some examples: diff --git a/newsfragments/25410.significant.rst b/newsfragments/25410.significant.rst index 6162f3a2aba0c..b95c102d99d26 100644 --- a/newsfragments/25410.significant.rst +++ b/newsfragments/25410.significant.rst @@ -1,8 +1,8 @@ -Deprecation of ``schedule_interval`` and ``timetable`` params +Deprecation of ``schedule_interval`` and ``timetable`` arguments -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. +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. -So if you previously used the ``@daily`` cron preset, your dag may have looked like this: +If you previously used the ``@daily`` cron preset, your DAG may have looked like this: .. code-block:: python with DAG( @@ -12,7 +12,7 @@ So if you previously used the ``@daily`` cron preset, your dag may have looked l ) as dag: ... -Going forward, instead you should use the ``schedule`` param instead: +Going forward, you should use the ``schedule`` argument instead: .. code-block:: python with DAG( @@ -22,7 +22,7 @@ Going forward, instead you should use the ``schedule`` param instead: ) as dag: ... -And the same is true if you used a custom timetable. Previously you would have used the ``timetable`` param: +The same is true if you used a custom timetable. Previously you would have used the ``timetable`` argument: .. code-block:: python with DAG( @@ -32,7 +32,7 @@ And the same is true if you used a custom timetable. Previously you would have ) as dag: ... -Now you should use the ``schedule`` param: +Now you should use the ``schedule`` argument: .. code-block:: python with DAG( From b168c2f8bd762f35153b845506636dfd7eb05ea9 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 9 Aug 2022 14:36:28 +0800 Subject: [PATCH 35/38] More casing and wording fixes --- docs/apache-airflow/dag-run.rst | 6 +++--- docs/apache-airflow/faq.rst | 6 +++--- docs/apache-airflow/templates-ref.rst | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/dag-run.rst index 721fbb4131f42..546fa562e1c25 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/docs/apache-airflow/dag-run.rst @@ -43,8 +43,8 @@ There are two possible terminal states for the DAG Run: Cron Presets '''''''''''' -You may set your DAG to run on a simple schedule by setting its ``schedule`` param with either a -`cron expression `_, or a ``datetime.timedelta`` object, +You may set your DAG to run on a simple schedule by setting its ``schedule`` argument to either a +`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 ` .. tip:: @@ -116,7 +116,7 @@ DAG run fails. Catchup ------- -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. +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. The scheduler, by default, will kick off a DAG Run for any data interval that has not been run since the last data interval (or has been cleared). This concept is called Catchup. diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index 7356d67ff7ffe..1621aa3cc6bfd 100644 --- a/docs/apache-airflow/faq.rst +++ b/docs/apache-airflow/faq.rst @@ -41,10 +41,10 @@ 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? For time-based dags, the task won't be triggered until the +- 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`` param set properly? The default +- Is your ``schedule`` argument set properly? The default is one day (``datetime.timedelta(1)``). You must specify a different ``schedule`` directly to the DAG object you instantiate, not as a ``default_param``, as task instances do not override their parent DAG's ``schedule``. @@ -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 - ``logical_date`` and the dag's schedule interval (if applicable). If you set ``schedule`` as ``None`` or ``@once``, + ``logical_date`` and the DAG's schedule interval (if applicable). If you set ``schedule`` as ``None`` or ``@once``, the ``next_ds``, ``next_ds_nodash``, ``prev_ds``, ``prev_ds_nodash`` values will be set to ``None``. - When manually triggering DAG, the schedule will be ignored, and ``prev_ds == next_ds == ds``. diff --git a/docs/apache-airflow/templates-ref.rst b/docs/apache-airflow/templates-ref.rst index 7b9cab1c6c026..da9a526152a46 100644 --- a/docs/apache-airflow/templates-ref.rst +++ b/docs/apache-airflow/templates-ref.rst @@ -89,8 +89,8 @@ 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 logical date of the next scheduled run (if applicable) - You may be able to use ``data_interval_end`` instead. +``{{ next_execution_date }}`` the logical date of the next scheduled run (if applicable); + you may be able to use ``data_interval_end`` instead ``{{ next_ds }}`` the next execution date as ``YYYY-MM-DD`` if exists, else ``None`` ``{{ next_ds_nodash }}`` the next execution date as ``YYYYMMDD`` if exists, else ``None`` ``{{ prev_execution_date }}`` the logical date of the previous scheduled run (if applicable) From d0559ac1625a34ecab36772e62efb4058886b707 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 9 Aug 2022 14:41:20 +0800 Subject: [PATCH 36/38] A few more formatting fixes --- docs/apache-airflow/dag-run.rst | 2 +- docs/apache-airflow/faq.rst | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/dag-run.rst index 546fa562e1c25..00c75cbeab29a 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/docs/apache-airflow/dag-run.rst @@ -101,7 +101,7 @@ scheduled one interval after ``start_date``. .. tip:: - If a cron expression or `timedelta` object 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, logical date, or data interval, see :doc:`/concepts/timetable`. For more information on ``logical date``, see :ref:`concepts:dag-run` and :ref:`faq:what-does-execution-date-mean` diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index 1621aa3cc6bfd..829451d53a722 100644 --- a/docs/apache-airflow/faq.rst +++ b/docs/apache-airflow/faq.rst @@ -224,7 +224,7 @@ an hour after now as ``now()`` moves along. Previously, we also recommended using rounded ``start_date`` in relation to your -DAG ``schedule``. This meant an ``@hourly`` would be at ``00:00`` +DAG's ``schedule``. This meant an ``@hourly`` would be at ``00:00`` minutes:seconds, a ``@daily`` job at midnight, a ``@monthly`` job on the first of the month. This is no longer required. Airflow will now auto align the ``start_date`` and the ``schedule``, by using the ``start_date`` @@ -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 - ``logical_date`` and the DAG's schedule interval (if applicable). If you set ``schedule`` as ``None`` or ``@once``, + ``logical_date`` and the DAG's schedule (if applicable). If you set ``schedule`` as ``None`` or ``@once``, the ``next_ds``, ``next_ds_nodash``, ``prev_ds``, ``prev_ds_nodash`` values will be set to ``None``. - When manually triggering DAG, the schedule will be ignored, and ``prev_ds == next_ds == ds``. From 86238c6bab5a6c72e6730a3862cab79d38664653 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 9 Aug 2022 14:50:54 +0800 Subject: [PATCH 37/38] Some more TaskDag conversions --- tests/models/test_dag.py | 80 ++++++++++++++-------------------------- 1 file changed, 28 insertions(+), 52 deletions(-) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 57fb307398a18..083b680f17731 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -22,7 +22,6 @@ import os import pickle import re -import unittest from contextlib import redirect_stdout from datetime import timedelta from pathlib import Path @@ -37,7 +36,6 @@ import pytest from dateutil.relativedelta import relativedelta from freezegun import freeze_time -from parameterized import parameterized from sqlalchemy import inspect from airflow import models, settings @@ -74,13 +72,6 @@ TEST_DATE = datetime_tz(2015, 1, 2, 0, 0) -@pytest.fixture -def session(): - with create_session() as session: - yield session - session.rollback() - - class TestDag: def setup_method(self) -> None: clear_db_runs() @@ -479,7 +470,8 @@ def test_template_undefined(self): jinja_env = dag.get_template_env() assert jinja_env.undefined is jinja2.Undefined - @parameterized.expand( + @pytest.mark.parametrize( + "use_native_obj, force_sandboxed, expected_env", [ (False, True, SandboxedEnvironment), (False, False, SandboxedEnvironment), @@ -756,7 +748,7 @@ def test_bulk_write_to_db(self): for row in session.query(DagModel.last_parsed_time).all(): assert row[0] is not None - @parameterized.expand([State.RUNNING, State.QUEUED]) + @pytest.mark.parametrize("state", [DagRunState.RUNNING, DagRunState.QUEUED]) def test_bulk_write_to_db_max_active_runs(self, state): """ Test that DagModel.next_dagrun_create_after is set to NULL when the dag cannot be created due to max @@ -1337,7 +1329,8 @@ def test_get_paused_dag_ids(self): with create_session() as session: session.query(DagModel).filter(DagModel.dag_id == dag_id).delete(synchronize_session=False) - @parameterized.expand( + @pytest.mark.parametrize( + "schedule_interval_arg, expected_timetable, interval_description", [ (None, NullTimetable(), "Never, external triggers only"), ("@daily", cron_timetable("0 0 * * *"), "At 00:00"), @@ -1349,7 +1342,7 @@ def test_get_paused_dag_ids(self): ("@once", OnceTimetable(), "Once, as soon as possible"), (datetime.timedelta(days=1), delta_timetable(datetime.timedelta(days=1)), ""), ("30 21 * * 5 1", cron_timetable("30 21 * * 5 1"), ""), - ] + ], ) def test_timetable_and_description_from_schedule_interval_arg( self, schedule_interval_arg, expected_timetable, interval_description @@ -1365,7 +1358,8 @@ def test_timetable_and_description_from_dataset(self): assert dag.schedule_interval == 'Dataset' assert dag.timetable.description == 'Triggered by datasets' - @parameterized.expand( + @pytest.mark.parametrize( + "timetable, expected_description", [ (NullTimetable(), "Never, external triggers only"), (cron_timetable("0 0 * * *"), "At 00:00"), @@ -1382,7 +1376,7 @@ def test_timetable_and_description_from_dataset(self): (OnceTimetable(), "Once, as soon as possible"), (delta_timetable(datetime.timedelta(days=1)), ""), (cron_timetable("30 21 * * 5 1"), ""), - ] + ], ) def test_description_from_timetable(self, timetable, expected_description): dag = DAG("test_schedule_interval_description", timetable=timetable) @@ -1423,12 +1417,7 @@ def test_dag_add_task_sets_default_task_group(self): assert task_group.get_child_by_label("task_with_task_group") == task_with_task_group assert dag.get_task("task_group.task_with_task_group") == task_with_task_group - @parameterized.expand( - [ - (State.QUEUED,), - (State.RUNNING,), - ] - ) + @pytest.mark.parametrize("dag_run_state", [DagRunState.QUEUED, DagRunState.RUNNING]) def test_clear_set_dagrun_state(self, dag_run_state): dag_id = 'test_clear_set_dagrun_state' self._clean_up(dag_id) @@ -1472,12 +1461,7 @@ def test_clear_set_dagrun_state(self, dag_run_state): dagrun = dagruns[0] # type: DagRun assert dagrun.state == dag_run_state - @parameterized.expand( - [ - (State.QUEUED,), - (State.RUNNING,), - ] - ) + @pytest.mark.parametrize("dag_run_state", [DagRunState.QUEUED, DagRunState.RUNNING]) def test_clear_set_dagrun_state_for_mapped_task(self, dag_run_state): dag_id = 'test_clear_set_dagrun_state' self._clean_up(dag_id) @@ -1578,12 +1562,7 @@ def _make_test_subdag(self, session): return dag, subdag - @parameterized.expand( - [ - (State.QUEUED,), - (State.RUNNING,), - ] - ) + @pytest.mark.parametrize("dag_run_state", [DagRunState.QUEUED, DagRunState.RUNNING]) def test_clear_set_dagrun_state_for_subdag(self, dag_run_state): session = settings.Session() dag, subdag = self._make_test_subdag(session) @@ -1608,12 +1587,7 @@ def test_clear_set_dagrun_state_for_subdag(self, dag_run_state): assert dagrun.state == dag_run_state session.rollback() - @parameterized.expand( - [ - (State.QUEUED,), - (State.RUNNING,), - ] - ) + @pytest.mark.parametrize("dag_run_state", [DagRunState.QUEUED, DagRunState.RUNNING]) def test_clear_set_dagrun_state_for_parent_dag(self, dag_run_state): session = settings.Session() dag, subdag = self._make_test_subdag(session) @@ -1637,11 +1611,18 @@ def test_clear_set_dagrun_state_for_parent_dag(self, dag_run_state): ) assert dagrun.state == dag_run_state - @parameterized.expand( - [(state, State.NONE) for state in State.task_states if state != State.RUNNING] - + [(State.RUNNING, State.RESTARTING)] # type: ignore + @pytest.mark.parametrize( + "ti_state_begin, ti_state_end", + [ + *((state, None) for state in State.task_states if state != TaskInstanceState.RUNNING), + (TaskInstanceState.RUNNING, TaskInstanceState.RESTARTING), + ], ) - def test_clear_dag(self, ti_state_begin, ti_state_end: Optional[str]): + def test_clear_dag( + self, + ti_state_begin: Optional[TaskInstanceState], + ti_state_end: Optional[TaskInstanceState], + ): dag_id = 'test_clear_dag' self._clean_up(dag_id) task_id = 't1' @@ -2158,19 +2139,14 @@ def test_relative_fileloc(self, fileloc, expected_relative): assert dag.relative_fileloc == expected_relative -class TestQueries(unittest.TestCase): - def setUp(self) -> None: +class TestQueries: + def setup_method(self) -> None: clear_db_runs() - def tearDown(self) -> None: + def teardown_method(self) -> None: clear_db_runs() - @parameterized.expand( - [ - (3,), - (12,), - ] - ) + @pytest.mark.parametrize("tasks_count", [3, 12]) def test_count_number_queries(self, tasks_count): dag = DAG('test_dagrun_query_count', start_date=DEFAULT_DATE) for i in range(tasks_count): From 6f0003b07edb21de97103a75bc491223966d088f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 9 Aug 2022 14:14:38 -0700 Subject: [PATCH 38/38] remove redundant param --- airflow/models/dag.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 64e80d051a7d4..79d5e0e780fd5 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -319,7 +319,6 @@ 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: List of upstream datasets if for use in triggering DAG runs. :param owner_links: Dict of owners and their links, that will be clickable on the DAGs view UI. Can be used as an HTTP link (for example the link to your Slack channel), or a mailto link. e.g: {"dag_owner": "https://airflow.apache.org/"}