From 514d30f6c5b2a26b9a8e26388d57a7f886774052 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Fri, 3 Nov 2023 00:49:59 +0100 Subject: [PATCH 1/6] Add a new dag param ignore_first_catchup to disable catchup for the first dag run --- airflow/api_connexion/openapi/v1.yaml | 3 ++ airflow/config_templates/config.yml | 8 ++++++ airflow/example_dags/plugins/workday.py | 8 ++++-- airflow/models/dag.py | 14 +++++++-- airflow/serialization/schema.json | 1 + airflow/timetables/base.py | 8 ++++-- airflow/timetables/interval.py | 5 +++- airflow/timetables/trigger.py | 5 +++- airflow/www/static/js/types/api-generated.ts | 1 + airflow/www/views.py | 2 +- tests/models/test_dag.py | 30 ++++++++++++++++++++ 11 files changed, 75 insertions(+), 10 deletions(-) diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index e4ae9c776f1c3..b9b5cf3f70f06 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -3581,6 +3581,9 @@ components: catchup: type: boolean readOnly: true + ignore_first_catchup: + type: boolean + readOnly: true orientation: type: string readOnly: true diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index ffb9dce073ad5..c3916ac47754c 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -2169,6 +2169,14 @@ scheduler: type: boolean example: ~ default: "True" + ignore_first_catchup: + description: | + Setting this to True will make the scheduler ignore the catchup + configuration on the first DAG run. + version_added: 2.8.0 + type: boolean + example: ~ + default: "False" max_tis_per_query: description: | This changes the batch size of queries in the scheduling main loop. diff --git a/airflow/example_dags/plugins/workday.py b/airflow/example_dags/plugins/workday.py index 2c3299c960bbb..ed5d51eaca3be 100644 --- a/airflow/example_dags/plugins/workday.py +++ b/airflow/example_dags/plugins/workday.py @@ -71,6 +71,9 @@ def next_dagrun_info( last_automated_data_interval: DataInterval | None, restriction: TimeRestriction, ) -> DagRunInfo | None: + should_catchup = restriction.catchup and ( + not restriction.ignore_first_catchup or last_automated_data_interval is not None + ) if last_automated_data_interval is not None: # There was a previous run on the regular schedule. last_start = last_automated_data_interval.start next_start = DateTime.combine((last_start + timedelta(days=1)).date(), Time.min).replace( @@ -80,8 +83,9 @@ def next_dagrun_info( next_start = restriction.earliest if next_start is None: # No start_date. Don't schedule. return None - if not restriction.catchup: - # If the DAG has catchup=False, today is the earliest to consider. + if not should_catchup: + # If the DAG has catchup=False, or it doesn't have any run and has ignore_first_dag=True, + # today is the earliest to consider. next_start = max(next_start, DateTime.combine(Date.today(), Time.min).replace(tzinfo=UTC)) elif next_start.time() != Time.min: # If earliest does not fall on midnight, skip to the next day. diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 897c0aa44ce44..a1c5319e203c6 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -366,6 +366,8 @@ class DAG(LoggingMixin): gantt, landing_times), default grid :param orientation: Specify DAG orientation in graph view (LR, TB, RL, BT), default LR :param catchup: Perform scheduler catchup (or only run latest)? Defaults to True + :param ignore_first_catchup: Whether to ignore the catchup for the first DAG run + if catchup is set to True. :param on_failure_callback: A function or list of functions to be called when a DagRun of this dag fails. A context dictionary is passed as a single parameter to this function. :param on_success_callback: Much like the ``on_failure_callback`` except @@ -452,6 +454,7 @@ def __init__( default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), catchup: bool = airflow_conf.getboolean("scheduler", "catchup_by_default"), + ignore_first_catchup: bool = airflow_conf.getboolean("scheduler", "ignore_first_catchup"), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, @@ -628,6 +631,7 @@ def __init__( f"{ORIENTATION_PRESETS}, but get {orientation}" ) self.catchup: bool = catchup + self.ignore_first_catchup: bool = ignore_first_catchup self.partial: bool = False self.on_success_callback = on_success_callback @@ -999,7 +1003,9 @@ def next_dagrun_info( if restricted: restriction = self._time_restriction else: - restriction = TimeRestriction(earliest=None, latest=None, catchup=True) + restriction = TimeRestriction( + earliest=None, latest=None, catchup=True, ignore_first_catchup=False + ) try: info = self.timetable.next_dagrun_info( last_automated_data_interval=data_interval, @@ -1044,7 +1050,7 @@ def _time_restriction(self) -> TimeRestriction: end_dates.append(self.end_date) if end_dates: latest = timezone.coerce_datetime(max(end_dates)) - return TimeRestriction(earliest, latest, self.catchup) + return TimeRestriction(earliest, latest, self.catchup, self.ignore_first_catchup) def iter_dagrun_infos_between( self, @@ -1076,7 +1082,7 @@ def iter_dagrun_infos_between( earliest = timezone.coerce_datetime(earliest) latest = timezone.coerce_datetime(latest) - restriction = TimeRestriction(earliest, latest, catchup=True) + restriction = TimeRestriction(earliest, latest, catchup=True, ignore_first_catchup=False) # HACK: Sub-DAGs are currently scheduled differently. For example, say # the schedule is @daily and start is 2021-06-03 22:16:00, a top-level @@ -3828,6 +3834,7 @@ def dag( default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), catchup: bool = airflow_conf.getboolean("scheduler", "catchup_by_default"), + ignore_first_catchup: bool = airflow_conf.getboolean("scheduler", "ignore_first_catchup"), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, @@ -3882,6 +3889,7 @@ def factory(*args, **kwargs): default_view=default_view, orientation=orientation, catchup=catchup, + ignore_first_catchup=ignore_first_catchup, on_success_callback=on_success_callback, on_failure_callback=on_failure_callback, doc_md=doc_md, diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 13489fedc611f..55b2f1fdadceb 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -131,6 +131,7 @@ } }, "catchup": { "type": "boolean" }, + "ignore_first_catchup": { "type": "boolean" }, "is_subdag": { "type": "boolean" }, "fileloc": { "type" : "string"}, "_processor_dags_folder": { diff --git a/airflow/timetables/base.py b/airflow/timetables/base.py index b5e95ef5f4662..28623069bc891 100644 --- a/airflow/timetables/base.py +++ b/airflow/timetables/base.py @@ -48,10 +48,13 @@ class TimeRestriction(NamedTuple): Specifically, the run must not be earlier than ``earliest``, nor later than ``latest``. If ``catchup`` is *False*, the run must also not be earlier than - the current time, i.e. "missed" schedules are not backfilled. + the current time, i.e. "missed" schedules are not backfilled. If + ``ignore_first_catchup`` is *True*, the run must also not be earlier than the + first DAG run's ``execution_date`` if there is any, and not earlier than the + current time otherwise. These values are generally set on the DAG or task's ``start_date``, - ``end_date``, and ``catchup`` arguments. + ``end_date``, ``catchup`` and ``ignore_first_catchup`` arguments. Both ``earliest`` and ``latest``, if not *None*, are inclusive; a DAG run can happen exactly at either point of time. They are guaranteed to be aware @@ -62,6 +65,7 @@ class TimeRestriction(NamedTuple): earliest: DateTime | None latest: DateTime | None catchup: bool + ignore_first_catchup: bool = False class DagRunInfo(NamedTuple): diff --git a/airflow/timetables/interval.py b/airflow/timetables/interval.py index 077c4195a7ae0..0221c31be0aca 100644 --- a/airflow/timetables/interval.py +++ b/airflow/timetables/interval.py @@ -85,8 +85,11 @@ def next_dagrun_info( last_automated_data_interval: DataInterval | None, restriction: TimeRestriction, ) -> DagRunInfo | None: + should_catchup = restriction.catchup and ( + not restriction.ignore_first_catchup or last_automated_data_interval is not None + ) earliest = restriction.earliest - if not restriction.catchup: + if not should_catchup: earliest = self._skip_to_latest(earliest) elif earliest is not None: earliest = self._align_to_next(earliest) diff --git a/airflow/timetables/trigger.py b/airflow/timetables/trigger.py index 95d29238037c4..34ae80ef0496a 100644 --- a/airflow/timetables/trigger.py +++ b/airflow/timetables/trigger.py @@ -85,7 +85,10 @@ def next_dagrun_info( last_automated_data_interval: DataInterval | None, restriction: TimeRestriction, ) -> DagRunInfo | None: - if restriction.catchup: + should_catchup = restriction.catchup and ( + not restriction.ignore_first_catchup or last_automated_data_interval is not None + ) + if should_catchup: if last_automated_data_interval is not None: next_start_time = self._get_next(last_automated_data_interval.end) elif restriction.earliest is None: diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index 607a173a6ba5b..06e8d55ce162f 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -1472,6 +1472,7 @@ export interface components { DAGDetail: components["schemas"]["DAG"] & { timezone?: components["schemas"]["Timezone"]; catchup?: boolean; + ignore_first_catchup?: boolean; orientation?: string; concurrency?: number; /** diff --git a/airflow/www/views.py b/airflow/www/views.py index 1c7a795a5aed7..05172c1f4e4cb 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -2897,7 +2897,7 @@ def _convert_to_date(session, column): ) year = last_automated_data_interval.end.year - restriction = TimeRestriction(dag.start_date, dag.end_date, False) + restriction = TimeRestriction(dag.start_date, dag.end_date, False, False) dates: dict[datetime.date, int] = collections.Counter() if isinstance(dag.timetable, CronMixin): diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 9fa0eb669d11b..6fc3c9655ca5d 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -2295,6 +2295,36 @@ 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) + @time_machine.travel(timezone.datetime(2020, 5, 4)) + def test_next_dagrun_info_timedelta_schedule_and_catchup_true_and_ignore_first_catchup_true(self): + """ + Test that the dag file processor creates multiple dagruns + if a dag is scheduled with 'timedelta' and catchup=True and ignore_first_depends_on_past=True + """ + dag = DAG( + "test_scheduler_dagrun_once_with_timedelta_and_catchup_true", + start_date=timezone.datetime(2020, 5, 1), + schedule=timedelta(days=1), + catchup=True, + ignore_first_catchup=True, + ) + + next_info = dag.next_dagrun_info(None) + # # if it is the first dagrun, the scheduling should be the same as catchup=False + # for the first dagrun, the dagrun date is the nearest data interval to current time (<= current time) + assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 3) + + next_info = dag.next_dagrun_info(timezone.datetime(2020, 5, 1)) + # if it is not the first dagrun, the scheduling should be the same as catchup=True + assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 2) + + next_info = dag.next_dagrun_info(next_info.data_interval) + assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 3) + + # The date to create is in the future, this is handled by "DagModel.dags_needing_dagruns" + 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, caplog): """Test the DAG does not crash the scheduler if the timetable raises an exception.""" From 7e1d6ddb3e9b7dce20981c00f574450c509c0f3b Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Tue, 7 Nov 2023 00:28:08 +0100 Subject: [PATCH 2/6] Merge ignore_first_catchup in catchup param --- airflow/api_connexion/openapi/v1.yaml | 5 +- airflow/config_templates/config.yml | 19 +++---- airflow/example_dags/example_bash_operator.py | 2 +- .../example_branch_datetime_operator.py | 6 +-- .../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 | 12 ++--- ...task_mapping_with_no_taskflow_operators.py | 2 +- .../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_dags/example_params_trigger_ui.py | 2 +- .../example_params_ui_tutorial.py | 2 +- ...example_passing_params_via_test_command.py | 2 +- .../example_dags/example_python_operator.py | 2 +- .../example_dags/example_sensor_decorator.py | 2 +- airflow/example_dags/example_sensors.py | 2 +- .../example_dags/example_setup_teardown.py | 2 +- .../example_setup_teardown_taskflow.py | 2 +- .../example_short_circuit_decorator.py | 2 +- .../example_short_circuit_operator.py | 2 +- airflow/example_dags/example_skip_dag.py | 2 +- airflow/example_dags/example_sla_dag.py | 2 +- airflow/example_dags/example_task_group.py | 2 +- .../example_task_group_decorator.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/plugins/workday.py | 7 +-- airflow/example_dags/subdags/subdag.py | 2 +- airflow/example_dags/tutorial.py | 2 +- airflow/example_dags/tutorial_dag.py | 2 +- .../example_dags/tutorial_objectstorage.py | 2 +- airflow/example_dags/tutorial_taskflow_api.py | 2 +- .../tutorial_taskflow_api_virtualenv.py | 2 +- airflow/models/dag.py | 23 ++++---- airflow/models/taskinstance.py | 2 +- .../arangodb/example_dags/example_arangodb.py | 2 +- .../example_dags/example_cloud_sql_query.py | 2 +- .../cloud/example_dags/example_cloud_task.py | 2 +- .../example_facebook_ads_to_gcs.py | 2 +- .../cloud/example_dags/example_looker.py | 2 +- .../example_dags/example_presto_to_gcs.py | 2 +- .../example_dags/example_salesforce_to_gcs.py | 2 +- .../example_dags/example_display_video.py | 6 +-- .../providers/microsoft/azure/CHANGELOG.rst | 2 +- .../oracle/example_dags/example_oracle.py | 2 +- airflow/serialization/schema.json | 3 +- airflow/timetables/base.py | 10 ++-- airflow/timetables/interval.py | 9 ++-- airflow/timetables/trigger.py | 5 +- airflow/utils/catchup.py | 52 +++++++++++++++++++ airflow/www/static/js/types/api-generated.ts | 3 +- dev/perf/dags/sql_perf_dag.py | 2 +- .../chime_notifier_howto_guide.rst | 2 +- .../notifications/sns.rst | 2 +- .../notifications/sqs.rst | 2 +- .../apprise_notifier_howto_guide.rst | 2 +- .../slackwebhook_notifier_howto_guide.rst | 2 +- .../smtp_notifier_howto_guide.rst | 2 +- .../administration-and-deployment/lineage.rst | 2 +- .../logging-monitoring/callbacks.rst | 2 +- docs/apache-airflow/best-practices.rst | 10 ++-- docs/apache-airflow/core-concepts/dag-run.rst | 6 +-- docs/apache-airflow/core-concepts/dags.rst | 2 +- .../core-concepts/executor/kubernetes.rst | 2 +- .../core-concepts/operators.rst | 2 +- .../apache-airflow/core-concepts/taskflow.rst | 2 +- .../howto/dynamic-dag-generation.rst | 2 +- docs/apache-airflow/howto/usage-cli.rst | 2 +- docs/apache-airflow/tutorial/pipeline.rst | 2 +- .../extending/embedding-dags/test_dag.py | 2 +- tests/dags/test_dags_folder.py | 2 +- tests/dags/test_datasets.py | 8 +-- tests/dags/test_sensor.py | 2 +- tests/jobs/test_scheduler_job.py | 6 +-- tests/models/test_dag.py | 34 ++++++------ tests/models/test_taskinstance.py | 14 ++--- .../providers/amazon/aws/example_appflow.py | 2 +- .../amazon/aws/example_appflow_run.py | 2 +- .../providers/amazon/aws/example_athena.py | 2 +- .../amazon/aws/example_azure_blob_to_s3.py | 2 +- .../providers/amazon/aws/example_batch.py | 2 +- .../amazon/aws/example_cloudformation.py | 2 +- .../providers/amazon/aws/example_datasync.py | 2 +- .../providers/amazon/aws/example_dms.py | 2 +- .../providers/amazon/aws/example_dynamodb.py | 2 +- .../amazon/aws/example_dynamodb_to_s3.py | 2 +- .../providers/amazon/aws/example_ec2.py | 2 +- .../providers/amazon/aws/example_ecs.py | 2 +- .../amazon/aws/example_ecs_fargate.py | 2 +- .../amazon/aws/example_eks_templated.py | 2 +- .../example_eks_with_fargate_in_one_step.py | 2 +- .../aws/example_eks_with_fargate_profile.py | 2 +- .../example_eks_with_nodegroup_in_one_step.py | 2 +- .../amazon/aws/example_eks_with_nodegroups.py | 2 +- .../providers/amazon/aws/example_emr.py | 2 +- .../providers/amazon/aws/example_emr_eks.py | 2 +- .../aws/example_emr_notebook_execution.py | 2 +- .../amazon/aws/example_emr_serverless.py | 2 +- .../amazon/aws/example_eventbridge.py | 2 +- .../providers/amazon/aws/example_ftp_to_s3.py | 2 +- .../providers/amazon/aws/example_gcs_to_s3.py | 2 +- .../amazon/aws/example_glacier_to_gcs.py | 2 +- .../providers/amazon/aws/example_glue.py | 2 +- .../aws/example_google_api_sheets_to_s3.py | 2 +- .../aws/example_google_api_youtube_to_s3.py | 2 +- .../amazon/aws/example_hive_to_dynamodb.py | 2 +- .../aws/example_imap_attachment_to_s3.py | 2 +- .../providers/amazon/aws/example_lambda.py | 2 +- .../amazon/aws/example_local_to_s3.py | 2 +- .../amazon/aws/example_mongo_to_s3.py | 2 +- .../amazon/aws/example_quicksight.py | 2 +- .../providers/amazon/aws/example_rds_event.py | 2 +- .../amazon/aws/example_rds_export.py | 2 +- .../amazon/aws/example_rds_instance.py | 2 +- .../amazon/aws/example_rds_snapshot.py | 2 +- .../providers/amazon/aws/example_redshift.py | 2 +- .../aws/example_redshift_s3_transfers.py | 2 +- .../system/providers/amazon/aws/example_s3.py | 2 +- .../providers/amazon/aws/example_s3_to_ftp.py | 2 +- .../amazon/aws/example_s3_to_sftp.py | 2 +- .../providers/amazon/aws/example_s3_to_sql.py | 2 +- .../providers/amazon/aws/example_sagemaker.py | 2 +- .../amazon/aws/example_sagemaker_endpoint.py | 2 +- .../amazon/aws/example_sagemaker_notebook.py | 2 +- .../amazon/aws/example_sagemaker_pipeline.py | 2 +- .../amazon/aws/example_salesforce_to_s3.py | 2 +- .../amazon/aws/example_sftp_to_s3.py | 2 +- .../providers/amazon/aws/example_sns.py | 2 +- .../providers/amazon/aws/example_sql_to_s3.py | 2 +- .../providers/amazon/aws/example_sqs.py | 2 +- .../amazon/aws/example_step_functions.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_async.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 +- .../kafka/example_dag_event_listener.py | 4 +- .../apache/kafka/example_dag_hello_kafka.py | 2 +- .../apache/kylin/example_kylin_dag.py | 2 +- .../providers/apache/livy/example_livy.py | 2 +- .../providers/apache/pig/example_pig.py | 2 +- .../apache/pinot/example_pinot_dag.py | 2 +- .../apache/spark/example_spark_dag.py | 2 +- tests/system/providers/asana/example_asana.py | 2 +- .../example_kubernetes_decorator.py | 2 +- .../kubernetes/example_kubernetes_resource.py | 2 +- .../kubernetes/example_spark_kubernetes.py | 2 +- .../example_cohere_embedding_operator.py | 4 +- .../sql/example_sql_column_table_check.py | 2 +- .../common/sql/example_sql_execute_query.py | 2 +- .../example_external_task_child_deferrable.py | 2 +- ...example_external_task_parent_deferrable.py | 2 +- .../providers/dbt/cloud/example_dbt_cloud.py | 2 +- .../system/providers/docker/example_docker.py | 2 +- .../docker/example_docker_copy_data.py | 2 +- .../providers/docker/example_docker_swarm.py | 2 +- .../example_taskflow_api_docker_virtualenv.py | 2 +- tests/system/providers/ftp/example_ftp.py | 2 +- .../system/providers/github/example_github.py | 2 +- .../providers/google/ads/example_ads.py | 2 +- .../cloud/automl/example_automl_dataset.py | 2 +- .../cloud/automl/example_automl_model.py | 2 +- .../example_automl_nl_text_classification.py | 2 +- .../example_automl_nl_text_extraction.py | 2 +- .../example_automl_nl_text_sentiment.py | 2 +- .../automl/example_automl_translation.py | 2 +- ...utoml_video_intelligence_classification.py | 2 +- ...mple_automl_video_intelligence_tracking.py | 2 +- .../example_automl_vision_classification.py | 2 +- .../example_automl_vision_object_detection.py | 2 +- .../azure/example_azure_fileshare_to_gcs.py | 2 +- .../bigquery/example_bigquery_dataset.py | 2 +- .../cloud/bigquery/example_bigquery_dts.py | 2 +- .../bigquery/example_bigquery_operations.py | 2 +- .../example_bigquery_operations_location.py | 2 +- .../bigquery/example_bigquery_queries.py | 2 +- .../example_bigquery_queries_async.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_gcs_async.py | 2 +- .../bigquery/example_bigquery_to_mssql.py | 2 +- .../bigquery/example_bigquery_to_mysql.py | 2 +- .../bigquery/example_bigquery_to_postgres.py | 2 +- .../bigquery/example_bigquery_transfer.py | 2 +- .../google/cloud/bigtable/example_bigtable.py | 2 +- .../cloud/cloud_run/example_cloud_run.py | 2 +- .../example_cloud_sql_query_mysql.py | 2 +- .../example_cloud_sql_query_postgres.py | 2 +- .../cloud/composer/example_cloud_composer.py | 2 +- .../example_cloud_composer_deferrable.py | 2 +- .../google/cloud/compute/example_compute.py | 2 +- .../cloud/compute/example_compute_igm.py | 2 +- .../cloud/compute/example_compute_ssh.py | 2 +- .../compute/example_compute_ssh_os_login.py | 2 +- .../compute/example_compute_ssh_parallel.py | 2 +- .../cloud/dataflow/example_dataflow_go.py | 2 +- .../dataflow/example_dataflow_native_java.py | 2 +- .../example_dataflow_native_python.py | 2 +- .../example_dataflow_native_python_async.py | 2 +- .../cloud/dataflow/example_dataflow_sql.py | 2 +- .../dataflow/example_dataflow_template.py | 2 +- .../google/cloud/dataform/example_dataform.py | 2 +- .../google/cloud/dataprep/example_dataprep.py | 2 +- .../cloud/dataproc/example_dataproc_batch.py | 2 +- .../example_dataproc_batch_deferrable.py | 2 +- .../example_dataproc_batch_persistent.py | 2 +- .../example_dataproc_cluster_deferrable.py | 2 +- .../example_dataproc_cluster_generator.py | 2 +- .../example_dataproc_cluster_update.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 +- .../cloud/dataproc/example_dataproc_presto.py | 2 +- .../dataproc/example_dataproc_pyspark.py | 2 +- .../cloud/dataproc/example_dataproc_spark.py | 2 +- .../dataproc/example_dataproc_spark_async.py | 2 +- .../example_dataproc_spark_deferrable.py | 2 +- .../dataproc/example_dataproc_spark_sql.py | 2 +- .../cloud/dataproc/example_dataproc_sparkr.py | 2 +- .../cloud/dataproc/example_dataproc_trino.py | 2 +- .../dataproc/example_dataproc_workflow.py | 2 +- .../example_dataproc_workflow_deferrable.py | 2 +- .../datastore/example_datastore_commit.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 +- .../gcs/example_gcs_to_bigquery_async.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_gdrive_to_gcs.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 +- .../cloud/gcs/example_s3_to_gcs_async.py | 2 +- .../google/cloud/gcs/example_sftp_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 +- .../cloud/ml_engine/example_mlengine.py | 2 +- .../google/cloud/pubsub/example_pubsub.py | 2 +- .../cloud/pubsub/example_pubsub_deferrable.py | 2 +- .../google/cloud/spanner/example_spanner.py | 2 +- .../google/cloud/tasks/example_queue.py | 2 +- .../google/cloud/tasks/example_tasks.py | 2 +- .../cloud/transfers/example_gcs_to_sftp.py | 2 +- .../transfers/example_gdrive_to_local.py | 2 +- .../cloud/translate/example_translate.py | 2 +- ...mple_vertex_ai_auto_ml_tabular_training.py | 2 +- ...ple_vertex_ai_custom_job_python_package.py | 2 +- ...ple_vertex_ai_hyperparameter_tuning_job.py | 2 +- .../example_vertex_ai_list_custom_jobs.py | 2 +- .../vision/example_vision_annotate_image.py | 2 +- .../vision/example_vision_autogenerated.py | 2 +- .../cloud/vision/example_vision_explicit.py | 2 +- tests/system/providers/http/example_http.py | 2 +- .../providers/jdbc/example_jdbc_queries.py | 2 +- tests/system/providers/mysql/example_mysql.py | 2 +- tests/system/providers/neo4j/example_neo4j.py | 2 +- .../providers/sftp/example_sftp_sensor.py | 2 +- .../providers/slack/example_sql_to_slack.py | 2 +- .../slack/example_sql_to_slack_webhook.py | 2 +- .../providers/trino/example_gcs_to_trino.py | 2 +- tests/system/providers/trino/example_trino.py | 2 +- .../weaviate/example_weaviate_operator.py | 2 +- tests/timetables/test_continuous_timetable.py | 3 +- tests/timetables/test_events_timetable.py | 3 +- tests/timetables/test_interval_timetable.py | 19 +++---- tests/timetables/test_trigger_timetable.py | 9 ++-- tests/timetables/test_workday_timetable.py | 3 +- 302 files changed, 443 insertions(+), 396 deletions(-) create mode 100644 airflow/utils/catchup.py diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index b9b5cf3f70f06..48a8e1d82583c 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -3579,10 +3579,7 @@ components: timezone: $ref: '#/components/schemas/Timezone' catchup: - type: boolean - readOnly: true - ignore_first_catchup: - type: boolean + type: string readOnly: true orientation: type: string diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index c3916ac47754c..e372e00e9c1f6 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -2148,16 +2148,17 @@ scheduler: default: "10.0" catchup_by_default: description: | - Turn off scheduler catchup by setting this to ``False``. + Turn off scheduler catchup by setting this to ``disable``. Default behavior is unchanged and Command Line Backfills still work, but the scheduler - will not do scheduler catchup if this is ``False``, + will not do scheduler catchup if this is ``disable``, however it can be set on a per DAG basis in the - DAG definition (catchup) + DAG definition (catchup). You can set it to ``ignore_first`` + to disable backfilling since the start date of the DAG. version_added: ~ - type: boolean + type: string example: ~ - default: "True" + default: "enable" ignore_first_depends_on_past_by_default: description: | Setting this to True will make first task instance of a task @@ -2169,14 +2170,6 @@ scheduler: type: boolean example: ~ default: "True" - ignore_first_catchup: - description: | - Setting this to True will make the scheduler ignore the catchup - configuration on the first DAG run. - version_added: 2.8.0 - type: boolean - example: ~ - default: "False" max_tis_per_query: description: | This changes the batch size of queries in the scheduling main loop. diff --git a/airflow/example_dags/example_bash_operator.py b/airflow/example_dags/example_bash_operator.py index 2988479e3692a..b3611cb6c9a96 100644 --- a/airflow/example_dags/example_bash_operator.py +++ b/airflow/example_dags/example_bash_operator.py @@ -30,7 +30,7 @@ dag_id="example_bash_operator", schedule="0 0 * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", dagrun_timeout=datetime.timedelta(minutes=60), tags=["example", "example2"], params={"example_key": "example_value"}, diff --git a/airflow/example_dags/example_branch_datetime_operator.py b/airflow/example_dags/example_branch_datetime_operator.py index 5b13c007f583f..cfeacd6377bb1 100644 --- a/airflow/example_dags/example_branch_datetime_operator.py +++ b/airflow/example_dags/example_branch_datetime_operator.py @@ -30,7 +30,7 @@ dag1 = DAG( dag_id="example_branch_datetime_operator", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], schedule="@daily", ) @@ -56,7 +56,7 @@ dag2 = DAG( dag_id="example_branch_datetime_operator_2", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], schedule="@daily", ) @@ -81,7 +81,7 @@ dag3 = DAG( dag_id="example_branch_datetime_operator_3", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], schedule="@daily", ) 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 1699e3c01ea7b..04840a5f82bda 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 @@ with DAG( dag_id="example_weekday_branch_operator", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], schedule="@daily", ) as dag: diff --git a/airflow/example_dags/example_branch_labels.py b/airflow/example_dags/example_branch_labels.py index 512fc4244dea7..d67fb0b20e996 100644 --- a/airflow/example_dags/example_branch_labels.py +++ b/airflow/example_dags/example_branch_labels.py @@ -30,7 +30,7 @@ "example_branch_labels", schedule="@daily", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", ) as dag: ingest = EmptyOperator(task_id="ingest") analyse = EmptyOperator(task_id="analyze") diff --git a/airflow/example_dags/example_branch_operator.py b/airflow/example_dags/example_branch_operator.py index 594c6a4cb1495..b8b0593a633f8 100644 --- a/airflow/example_dags/example_branch_operator.py +++ b/airflow/example_dags/example_branch_operator.py @@ -46,7 +46,7 @@ with DAG( dag_id="example_branch_operator", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule="@daily", tags=["example", "example2"], orientation="TB", diff --git a/airflow/example_dags/example_branch_operator_decorator.py b/airflow/example_dags/example_branch_operator_decorator.py index 5d42ff6b2726f..38c5b912c8c3f 100644 --- a/airflow/example_dags/example_branch_operator_decorator.py +++ b/airflow/example_dags/example_branch_operator_decorator.py @@ -40,7 +40,7 @@ with DAG( dag_id="example_branch_python_operator_decorator", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule="@daily", tags=["example", "example2"], orientation="TB", 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 af28a9b9e10b4..4dfb3390e57e6 100644 --- a/airflow/example_dags/example_branch_python_dop_operator_3.py +++ b/airflow/example_dags/example_branch_python_dop_operator_3.py @@ -49,7 +49,7 @@ def should_run(**kwargs) -> str: dag_id="example_branch_dop_operator_v3", schedule="*/1 * * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", default_args={"depends_on_past": True}, tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_complex.py b/airflow/example_dags/example_complex.py index 1e7ab78faa31d..0db6bcb2ff8c9 100644 --- a/airflow/example_dags/example_complex.py +++ b/airflow/example_dags/example_complex.py @@ -30,7 +30,7 @@ dag_id="example_complex", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example", "example2", "example3"], ) as dag: # Create diff --git a/airflow/example_dags/example_dag_decorator.py b/airflow/example_dags/example_dag_decorator.py index 447b4471b97e4..893ee6aa9a94b 100644 --- a/airflow/example_dags/example_dag_decorator.py +++ b/airflow/example_dags/example_dag_decorator.py @@ -45,7 +45,7 @@ def execute(self, context: Context): @dag( schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) def example_dag_decorator(email: str = "example@example.com"): diff --git a/airflow/example_dags/example_datasets.py b/airflow/example_dags/example_datasets.py index ba3ed927f1d92..45365ca47e10e 100644 --- a/airflow/example_dags/example_datasets.py +++ b/airflow/example_dags/example_datasets.py @@ -51,7 +51,7 @@ with DAG( dag_id="dataset_produces_1", - catchup=False, + catchup="disable", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule="@daily", tags=["produces", "dataset-scheduled"], @@ -62,7 +62,7 @@ with DAG( dag_id="dataset_produces_2", - catchup=False, + catchup="disable", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule=None, tags=["produces", "dataset-scheduled"], @@ -72,7 +72,7 @@ # [START dag_dep] with DAG( dag_id="dataset_consumes_1", - catchup=False, + catchup="disable", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule=[dag1_dataset], tags=["consumes", "dataset-scheduled"], @@ -86,7 +86,7 @@ with DAG( dag_id="dataset_consumes_1_and_2", - catchup=False, + catchup="disable", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule=[dag1_dataset, dag2_dataset], tags=["consumes", "dataset-scheduled"], @@ -99,7 +99,7 @@ with DAG( dag_id="dataset_consumes_1_never_scheduled", - catchup=False, + catchup="disable", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule=[ dag1_dataset, @@ -115,7 +115,7 @@ with DAG( dag_id="dataset_consumes_unknown_never_scheduled", - catchup=False, + catchup="disable", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule=[ Dataset("s3://unrelated/dataset3.txt"), diff --git a/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py b/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py index ca68a1cf2943d..7146191a5a9a9 100644 --- a/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py +++ b/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py @@ -53,7 +53,7 @@ def execute(self, context): with DAG( dag_id="example_dynamic_task_mapping_with_no_taskflow_operators", start_date=datetime(2022, 3, 4), - catchup=False, + catchup="disable", ): # map the task to a list of values add_one_task = AddOneOperator.partial(task_id="add_one").expand(value=[1, 2, 3]) diff --git a/airflow/example_dags/example_external_task_marker_dag.py b/airflow/example_dags/example_external_task_marker_dag.py index e3b6b7fc19907..b9ab58adcc9b0 100644 --- a/airflow/example_dags/example_external_task_marker_dag.py +++ b/airflow/example_dags/example_external_task_marker_dag.py @@ -50,7 +50,7 @@ with DAG( dag_id="example_external_task_marker_parent", start_date=start_date, - catchup=False, + catchup="disable", schedule=None, tags=["example2"], ) as parent_dag: @@ -66,7 +66,7 @@ dag_id="example_external_task_marker_child", start_date=start_date, schedule=None, - catchup=False, + catchup="disable", tags=["example2"], ) as child_dag: # [START howto_operator_external_task_sensor] diff --git a/airflow/example_dags/example_kubernetes_executor.py b/airflow/example_dags/example_kubernetes_executor.py index a82abcf8a3d2e..eee085af0ccfb 100644 --- a/airflow/example_dags/example_kubernetes_executor.py +++ b/airflow/example_dags/example_kubernetes_executor.py @@ -47,7 +47,7 @@ dag_id="example_kubernetes_executor", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example3"], ) as dag: # You can use annotations on your kubernetes pods! diff --git a/airflow/example_dags/example_latest_only.py b/airflow/example_dags/example_latest_only.py index 59c879cfc28ca..837d74d986aa4 100644 --- a/airflow/example_dags/example_latest_only.py +++ b/airflow/example_dags/example_latest_only.py @@ -28,7 +28,7 @@ dag_id="latest_only", schedule=datetime.timedelta(hours=4), start_date=datetime.datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example2", "example3"], ) as dag: latest_only = LatestOnlyOperator(task_id="latest_only") diff --git a/airflow/example_dags/example_latest_only_with_trigger.py b/airflow/example_dags/example_latest_only_with_trigger.py index 3f6c0ffbd1e81..1d4c46fc65b5c 100644 --- a/airflow/example_dags/example_latest_only_with_trigger.py +++ b/airflow/example_dags/example_latest_only_with_trigger.py @@ -34,7 +34,7 @@ dag_id="latest_only_with_trigger", schedule=datetime.timedelta(hours=4), start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example3"], ) as dag: latest_only = LatestOnlyOperator(task_id="latest_only") diff --git a/airflow/example_dags/example_local_kubernetes_executor.py b/airflow/example_dags/example_local_kubernetes_executor.py index 18341fabe32f1..e6921b97b881b 100644 --- a/airflow/example_dags/example_local_kubernetes_executor.py +++ b/airflow/example_dags/example_local_kubernetes_executor.py @@ -45,7 +45,7 @@ dag_id="example_local_kubernetes_executor", schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example3"], ) as dag: # You can use annotations on your kubernetes pods! diff --git a/airflow/example_dags/example_nested_branch_dag.py b/airflow/example_dags/example_nested_branch_dag.py index 624a66dab6b3c..040bdb3e328c0 100644 --- a/airflow/example_dags/example_nested_branch_dag.py +++ b/airflow/example_dags/example_nested_branch_dag.py @@ -32,7 +32,7 @@ with DAG( dag_id="example_nested_branch_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule="@daily", tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_params_trigger_ui.py b/airflow/example_dags/example_params_trigger_ui.py index cc46406521be7..33843607212ad 100644 --- a/airflow/example_dags/example_params_trigger_ui.py +++ b/airflow/example_dags/example_params_trigger_ui.py @@ -40,7 +40,7 @@ doc_md=__doc__, schedule=None, start_date=datetime.datetime(2022, 3, 4), - catchup=False, + catchup="disable", tags=["example_ui"], params={ "names": Param( diff --git a/airflow/example_dags/example_params_ui_tutorial.py b/airflow/example_dags/example_params_ui_tutorial.py index 12992c545c25a..374960be1af66 100644 --- a/airflow/example_dags/example_params_ui_tutorial.py +++ b/airflow/example_dags/example_params_ui_tutorial.py @@ -36,7 +36,7 @@ doc_md=__doc__, schedule=None, start_date=datetime.datetime(2022, 3, 4), - catchup=False, + catchup="disable", tags=["example_ui"], params={ # Let's start simple: Standard dict values are detected from type and offered as entry form fields. 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 9e73d30d1f8e7..e36175e7539c2 100644 --- a/airflow/example_dags/example_passing_params_via_test_command.py +++ b/airflow/example_dags/example_passing_params_via_test_command.py @@ -61,7 +61,7 @@ def print_env_vars(test_mode=None): "example_passing_params_via_test_command", schedule="*/1 * * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", dagrun_timeout=datetime.timedelta(minutes=4), tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_python_operator.py b/airflow/example_dags/example_python_operator.py index 7518206b41a2f..65f8b553caa13 100644 --- a/airflow/example_dags/example_python_operator.py +++ b/airflow/example_dags/example_python_operator.py @@ -45,7 +45,7 @@ def x(): dag_id="example_python_operator", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_python] diff --git a/airflow/example_dags/example_sensor_decorator.py b/airflow/example_dags/example_sensor_decorator.py index db3059398c0d1..33527ac0a67d8 100644 --- a/airflow/example_dags/example_sensor_decorator.py +++ b/airflow/example_dags/example_sensor_decorator.py @@ -34,7 +34,7 @@ @dag( schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) def example_sensor_decorator(): diff --git a/airflow/example_dags/example_sensors.py b/airflow/example_dags/example_sensors.py index 9e3fc02f9c55c..babdd84d1e094 100644 --- a/airflow/example_dags/example_sensors.py +++ b/airflow/example_dags/example_sensors.py @@ -49,7 +49,7 @@ def failure_callable(): dag_id="example_sensors", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START example_time_delta_sensor] diff --git a/airflow/example_dags/example_setup_teardown.py b/airflow/example_dags/example_setup_teardown.py index 59aba9753a119..7c91c4fd0ddc2 100644 --- a/airflow/example_dags/example_setup_teardown.py +++ b/airflow/example_dags/example_setup_teardown.py @@ -27,7 +27,7 @@ with DAG( dag_id="example_setup_teardown", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: root_setup = BashOperator(task_id="root_setup", bash_command="echo 'Hello from root_setup'").as_setup() diff --git a/airflow/example_dags/example_setup_teardown_taskflow.py b/airflow/example_dags/example_setup_teardown_taskflow.py index b1bdbccf7fadc..00d15675bfa41 100644 --- a/airflow/example_dags/example_setup_teardown_taskflow.py +++ b/airflow/example_dags/example_setup_teardown_taskflow.py @@ -26,7 +26,7 @@ with DAG( dag_id="example_setup_teardown_taskflow", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_short_circuit_decorator.py b/airflow/example_dags/example_short_circuit_decorator.py index 79a8c6904b17e..40019dad79559 100644 --- a/airflow/example_dags/example_short_circuit_decorator.py +++ b/airflow/example_dags/example_short_circuit_decorator.py @@ -25,7 +25,7 @@ from airflow.utils.trigger_rule import TriggerRule -@dag(start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"]) +@dag(start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup="disable", tags=["example"]) def example_short_circuit_decorator(): # [START howto_operator_short_circuit] @task.short_circuit() diff --git a/airflow/example_dags/example_short_circuit_operator.py b/airflow/example_dags/example_short_circuit_operator.py index 18a0a8d8509f9..0a8866f05a193 100644 --- a/airflow/example_dags/example_short_circuit_operator.py +++ b/airflow/example_dags/example_short_circuit_operator.py @@ -29,7 +29,7 @@ with DAG( dag_id="example_short_circuit_operator", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: cond_true = ShortCircuitOperator( diff --git a/airflow/example_dags/example_skip_dag.py b/airflow/example_dags/example_skip_dag.py index e30b462b8c7b7..a82a65870c65c 100644 --- a/airflow/example_dags/example_skip_dag.py +++ b/airflow/example_dags/example_skip_dag.py @@ -63,7 +63,7 @@ def create_test_pipeline(suffix, trigger_rule): with DAG( dag_id="example_skip_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_test_pipeline("1", TriggerRule.ALL_SUCCESS) diff --git a/airflow/example_dags/example_sla_dag.py b/airflow/example_dags/example_sla_dag.py index e76f40f10ddad..8e32bded393ac 100644 --- a/airflow/example_dags/example_sla_dag.py +++ b/airflow/example_dags/example_sla_dag.py @@ -42,7 +42,7 @@ def sla_callback(dag, task_list, blocking_task_list, slas, blocking_tis): @dag( schedule="*/2 * * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", sla_miss_callback=sla_callback, default_args={"email": "email@example.com"}, ) diff --git a/airflow/example_dags/example_task_group.py b/airflow/example_dags/example_task_group.py index 9d7a9f2e74d59..c916a25282099 100644 --- a/airflow/example_dags/example_task_group.py +++ b/airflow/example_dags/example_task_group.py @@ -29,7 +29,7 @@ with DAG( dag_id="example_task_group", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: start = EmptyOperator(task_id="start") diff --git a/airflow/example_dags/example_task_group_decorator.py b/airflow/example_dags/example_task_group_decorator.py index db8d7b302529c..6b3ec4e4c7897 100644 --- a/airflow/example_dags/example_task_group_decorator.py +++ b/airflow/example_dags/example_task_group_decorator.py @@ -67,7 +67,7 @@ def task_group_function(value: int) -> None: with DAG( dag_id="example_task_group_decorator", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: start_task = task_start() diff --git a/airflow/example_dags/example_time_delta_sensor_async.py b/airflow/example_dags/example_time_delta_sensor_async.py index 1666f29f3ac7d..f399f0606ffcc 100644 --- a/airflow/example_dags/example_time_delta_sensor_async.py +++ b/airflow/example_dags/example_time_delta_sensor_async.py @@ -33,7 +33,7 @@ dag_id="example_time_delta_sensor_async", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: wait = TimeDeltaSensorAsync(task_id="wait", delta=datetime.timedelta(seconds=30)) diff --git a/airflow/example_dags/example_trigger_controller_dag.py b/airflow/example_dags/example_trigger_controller_dag.py index 9d0d642223372..335cf2ef1ab50 100644 --- a/airflow/example_dags/example_trigger_controller_dag.py +++ b/airflow/example_dags/example_trigger_controller_dag.py @@ -30,7 +30,7 @@ with DAG( dag_id="example_trigger_controller_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule="@once", tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_trigger_target_dag.py b/airflow/example_dags/example_trigger_target_dag.py index d31b4b869a041..1db893023f034 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): with DAG( dag_id="example_trigger_target_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule=None, tags=["example"], ) as dag: diff --git a/airflow/example_dags/example_xcom.py b/airflow/example_dags/example_xcom.py index 655c424755423..dfaaf81964440 100644 --- a/airflow/example_dags/example_xcom.py +++ b/airflow/example_dags/example_xcom.py @@ -67,7 +67,7 @@ def pull_value_from_bash_push(ti=None): "example_xcom", schedule="@once", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: bash_push = BashOperator( diff --git a/airflow/example_dags/example_xcomargs.py b/airflow/example_dags/example_xcomargs.py index c2be83ea08ac5..7784ad86466c2 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): with DAG( dag_id="example_xcom_args", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule=None, tags=["example"], ) as dag: @@ -53,7 +53,7 @@ def print_value(value, ts=None): with DAG( "example_xcom_args_with_operators", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule=None, tags=["example"], ) as dag2: diff --git a/airflow/example_dags/plugins/workday.py b/airflow/example_dags/plugins/workday.py index ed5d51eaca3be..2608aaac1c271 100644 --- a/airflow/example_dags/plugins/workday.py +++ b/airflow/example_dags/plugins/workday.py @@ -27,6 +27,7 @@ from airflow.plugins_manager import AirflowPlugin from airflow.timetables.base import DagRunInfo, DataInterval, Timetable +from airflow.utils.catchup import Catchup if TYPE_CHECKING: from airflow.timetables.base import TimeRestriction @@ -71,8 +72,8 @@ def next_dagrun_info( last_automated_data_interval: DataInterval | None, restriction: TimeRestriction, ) -> DagRunInfo | None: - should_catchup = restriction.catchup and ( - not restriction.ignore_first_catchup or last_automated_data_interval is not None + should_catchup = (restriction.catchup == Catchup.ENABLE) or ( + restriction.catchup == Catchup.IGNORE_FIRST and last_automated_data_interval is not None ) if last_automated_data_interval is not None: # There was a previous run on the regular schedule. last_start = last_automated_data_interval.start @@ -84,7 +85,7 @@ def next_dagrun_info( if next_start is None: # No start_date. Don't schedule. return None if not should_catchup: - # If the DAG has catchup=False, or it doesn't have any run and has ignore_first_dag=True, + # If the DAG has catchup="disabled", or catchup="ignore_first" and it doesn't have any run, # today is the earliest to consider. next_start = max(next_start, DateTime.combine(Date.today(), Time.min).replace(tzinfo=UTC)) elif next_start.time() != Time.min: diff --git a/airflow/example_dags/subdags/subdag.py b/airflow/example_dags/subdags/subdag.py index 9745fb9e2d91f..110872a5238fb 100644 --- a/airflow/example_dags/subdags/subdag.py +++ b/airflow/example_dags/subdags/subdag.py @@ -38,7 +38,7 @@ def subdag(parent_dag_name, child_dag_name, args) -> DAG: dag_id=f"{parent_dag_name}.{child_dag_name}", default_args=args, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", schedule="@daily", ) diff --git a/airflow/example_dags/tutorial.py b/airflow/example_dags/tutorial.py index 4656f69c0b655..46899cf6fdbea 100644 --- a/airflow/example_dags/tutorial.py +++ b/airflow/example_dags/tutorial.py @@ -66,7 +66,7 @@ description="A simple tutorial DAG", schedule=timedelta(days=1), start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [END instantiate_dag] diff --git a/airflow/example_dags/tutorial_dag.py b/airflow/example_dags/tutorial_dag.py index f0420a85f850b..d26f103106344 100644 --- a/airflow/example_dags/tutorial_dag.py +++ b/airflow/example_dags/tutorial_dag.py @@ -47,7 +47,7 @@ description="DAG tutorial", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [END instantiate_dag] diff --git a/airflow/example_dags/tutorial_objectstorage.py b/airflow/example_dags/tutorial_objectstorage.py index 5394238c7ad2b..14f3da6dfcba4 100644 --- a/airflow/example_dags/tutorial_objectstorage.py +++ b/airflow/example_dags/tutorial_objectstorage.py @@ -51,7 +51,7 @@ @dag( schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) def tutorial_objectstorage(): diff --git a/airflow/example_dags/tutorial_taskflow_api.py b/airflow/example_dags/tutorial_taskflow_api.py index f41f729af8870..f5716617a1694 100644 --- a/airflow/example_dags/tutorial_taskflow_api.py +++ b/airflow/example_dags/tutorial_taskflow_api.py @@ -32,7 +32,7 @@ @dag( schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) def tutorial_taskflow_api(): diff --git a/airflow/example_dags/tutorial_taskflow_api_virtualenv.py b/airflow/example_dags/tutorial_taskflow_api_virtualenv.py index 44134e445891d..ecf4f192bb80e 100644 --- a/airflow/example_dags/tutorial_taskflow_api_virtualenv.py +++ b/airflow/example_dags/tutorial_taskflow_api_virtualenv.py @@ -29,7 +29,7 @@ log.warning("The tutorial_taskflow_api_virtualenv example DAG requires virtualenv, please install it.") else: - @dag(schedule=None, start_date=datetime(2021, 1, 1), catchup=False, tags=["example"]) + @dag(schedule=None, start_date=datetime(2021, 1, 1), catchup="disable", tags=["example"]) def tutorial_taskflow_api_virtualenv(): """ ### TaskFlow API example using virtualenv diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 7a11bfd9a09ab..35d3e5f0bdb34 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -115,6 +115,7 @@ OnceTimetable, ) from airflow.utils import timezone +from airflow.utils.catchup import Catchup from airflow.utils.dag_cycle_tester import check_cycle from airflow.utils.dates import cron_presets, date_range as utils_date_range from airflow.utils.decorators import fixup_decorator_warning_stack @@ -365,9 +366,7 @@ class DAG(LoggingMixin): :param default_view: Specify DAG default view (grid, graph, duration, gantt, landing_times), default grid :param orientation: Specify DAG orientation in graph view (LR, TB, RL, BT), default LR - :param catchup: Perform scheduler catchup (or only run latest)? Defaults to True - :param ignore_first_catchup: Whether to ignore the catchup for the first DAG run - if catchup is set to True. + :param catchup: Perform scheduler catchup (or only run latest)? Defaults to enable. :param on_failure_callback: A function or list of functions to be called when a DagRun of this dag fails. A context dictionary is passed as a single parameter to this function. :param on_success_callback: Much like the ``on_failure_callback`` except @@ -453,8 +452,7 @@ def __init__( sla_miss_callback: None | SLAMissCallback | list[SLAMissCallback] = None, default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), - catchup: bool = airflow_conf.getboolean("scheduler", "catchup_by_default"), - ignore_first_catchup: bool = airflow_conf.getboolean("scheduler", "ignore_first_catchup"), + catchup: str | bool = airflow_conf.get("scheduler", "catchup_by_default"), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, @@ -630,8 +628,7 @@ def __init__( f"Invalid values of dag.orientation: only support " f"{ORIENTATION_PRESETS}, but get {orientation}" ) - self.catchup: bool = catchup - self.ignore_first_catchup: bool = ignore_first_catchup + self.catchup: Catchup = Catchup(catchup) self.partial: bool = False self.on_success_callback = on_success_callback @@ -1004,7 +1001,9 @@ def next_dagrun_info( restriction = self._time_restriction else: restriction = TimeRestriction( - earliest=None, latest=None, catchup=True, ignore_first_catchup=False + earliest=None, + latest=None, + catchup=Catchup.ENABLE, ) try: info = self.timetable.next_dagrun_info( @@ -1050,7 +1049,7 @@ def _time_restriction(self) -> TimeRestriction: end_dates.append(self.end_date) if end_dates: latest = timezone.coerce_datetime(max(end_dates)) - return TimeRestriction(earliest, latest, self.catchup, self.ignore_first_catchup) + return TimeRestriction(earliest, latest, self.catchup) def iter_dagrun_infos_between( self, @@ -1082,7 +1081,7 @@ def iter_dagrun_infos_between( earliest = timezone.coerce_datetime(earliest) latest = timezone.coerce_datetime(latest) - restriction = TimeRestriction(earliest, latest, catchup=True, ignore_first_catchup=False) + restriction = TimeRestriction(earliest, latest, catchup=Catchup.ENABLE) # HACK: Sub-DAGs are currently scheduled differently. For example, say # the schedule is @daily and start is 2021-06-03 22:16:00, a top-level @@ -3839,8 +3838,7 @@ def dag( sla_miss_callback: None | SLAMissCallback | list[SLAMissCallback] = None, default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), - catchup: bool = airflow_conf.getboolean("scheduler", "catchup_by_default"), - ignore_first_catchup: bool = airflow_conf.getboolean("scheduler", "ignore_first_catchup"), + catchup: str | bool = airflow_conf.get("scheduler", "catchup_by_default"), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, @@ -3895,7 +3893,6 @@ def factory(*args, **kwargs): default_view=default_view, orientation=orientation, catchup=catchup, - ignore_first_catchup=ignore_first_catchup, on_success_callback=on_success_callback, on_failure_callback=on_failure_callback, doc_md=doc_md, diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 02b6b9e71dfb0..89b4b890e82ac 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -947,7 +947,7 @@ def _get_previous_dagrun( # We always ignore schedule in dagrun lookup when `state` is given # or the DAG is never scheduled. For legacy reasons, when - # `catchup=True`, we use `get_previous_scheduled_dagrun` unless + # `catchup="enable"`, we use `get_previous_scheduled_dagrun` unless # `ignore_schedule` is `True`. ignore_schedule = state is not None or not dag.timetable.can_be_scheduled if dag.catchup is True and not ignore_schedule: diff --git a/airflow/providers/arangodb/example_dags/example_arangodb.py b/airflow/providers/arangodb/example_dags/example_arangodb.py index 71c6346ef6073..f48c980cb6ace 100644 --- a/airflow/providers/arangodb/example_dags/example_arangodb.py +++ b/airflow/providers/arangodb/example_dags/example_arangodb.py @@ -26,7 +26,7 @@ "example_arangodb_operator", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) # [START howto_aql_sensor_arangodb] 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 040ce91515cfb..4e418ad1fe669 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 @@ -268,7 +268,7 @@ def get_absolute_path(path): with DAG( dag_id="example_gcp_sql_query", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: prev_task = None diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_task.py b/airflow/providers/google/cloud/example_dags/example_cloud_task.py index 662199c23abc0..2191a12f08f4a 100644 --- a/airflow/providers/google/cloud/example_dags/example_cloud_task.py +++ b/airflow/providers/google/cloud/example_dags/example_cloud_task.py @@ -40,7 +40,7 @@ with DAG( "example_gcp_cloud_tasks_sensor", start_date=datetime(2022, 8, 8), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START cloud_tasks_empty_sensor] 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 a8b17ddeccac4..31626dda34973 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 @@ -60,7 +60,7 @@ with DAG( "example_facebook_ads_to_gcs", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_bucket = GCSCreateBucketOperator( task_id="create_bucket", diff --git a/airflow/providers/google/cloud/example_dags/example_looker.py b/airflow/providers/google/cloud/example_dags/example_looker.py index 1925d0fc2b78c..14ebaa6e41535 100644 --- a/airflow/providers/google/cloud/example_dags/example_looker.py +++ b/airflow/providers/google/cloud/example_dags/example_looker.py @@ -30,7 +30,7 @@ with DAG( dag_id="example_gcp_looker", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START cloud_looker_async_start_pdt_sensor] start_pdt_task_async = LookerStartPdtBuildOperator( 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 3589976b46285..af5dedbfd5213 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 @@ -51,7 +51,7 @@ def safe_name(s: str) -> str: with DAG( dag_id="example_presto_to_gcs", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create-dataset", dataset_id=DATASET_NAME) 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 74a6b91cbe329..942a9ad5fe10b 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 @@ -46,7 +46,7 @@ with DAG( "example_salesforce_to_gcs", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_bucket = GCSCreateBucketOperator( task_id="create_bucket", 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 a54d7e575be11..20864d49d6f3c 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 @@ -90,7 +90,7 @@ with DAG( "example_display_video_misc", start_date=START_DATE, - catchup=False, + catchup="disable", ) as dag2: # [START howto_google_display_video_upload_multiple_entity_read_files_to_big_query] upload_erf_to_bq = GCSToBigQueryOperator( @@ -123,7 +123,7 @@ with DAG( "example_display_video_sdf", start_date=START_DATE, - catchup=False, + catchup="disable", ) as dag3: # [START howto_google_display_video_create_sdf_download_task_operator] create_sdf_download_task = GoogleDisplayVideo360CreateSDFDownloadTaskOperator( @@ -171,7 +171,7 @@ with DAG( "example_display_video_v2", start_date=START_DATE, - catchup=False, + catchup="disable", ) as dag: # [START howto_google_display_video_create_query_operator] create_query_v2 = GoogleDisplayVideo360CreateQueryOperator(body=REPORT_V2, task_id="create_query") diff --git a/airflow/providers/microsoft/azure/CHANGELOG.rst b/airflow/providers/microsoft/azure/CHANGELOG.rst index 866ad0f962d80..f817368bb0a05 100644 --- a/airflow/providers/microsoft/azure/CHANGELOG.rst +++ b/airflow/providers/microsoft/azure/CHANGELOG.rst @@ -729,7 +729,7 @@ Bug Fixes .. Below changes are excluded from the changelog. Move them to appropriate section above if needed. Do not delete the lines(!): - * ``Ensure ''catchup=False'' is used in example dags (#19396)`` + * ``Ensure ''catchup="disable"'' is used in example dags (#19396)`` 3.3.0 ..... diff --git a/airflow/providers/oracle/example_dags/example_oracle.py b/airflow/providers/oracle/example_dags/example_oracle.py index a68c0be6c882d..194ea8d5861b3 100644 --- a/airflow/providers/oracle/example_dags/example_oracle.py +++ b/airflow/providers/oracle/example_dags/example_oracle.py @@ -25,7 +25,7 @@ with DAG( max_active_runs=1, max_active_tasks=3, - catchup=False, + catchup="disable", start_date=datetime(2023, 1, 1), dag_id="example_oracle", ) as dag: diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index f2ff42641378c..9e0d876968b65 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -130,8 +130,7 @@ "value": { "$ref": "#/definitions/dict" } } }, - "catchup": { "type": "boolean" }, - "ignore_first_catchup": { "type": "boolean" }, + "catchup": { "type": "string" }, "is_subdag": { "type": "boolean" }, "fileloc": { "type" : "string"}, "_processor_dags_folder": { diff --git a/airflow/timetables/base.py b/airflow/timetables/base.py index 28623069bc891..7e2a7a46197ec 100644 --- a/airflow/timetables/base.py +++ b/airflow/timetables/base.py @@ -24,6 +24,7 @@ if TYPE_CHECKING: from pendulum import DateTime + from airflow.utils.catchup import Catchup from airflow.utils.types import DagRunType @@ -47,14 +48,14 @@ class TimeRestriction(NamedTuple): """Restriction on when a DAG can be scheduled for a run. Specifically, the run must not be earlier than ``earliest``, nor later than - ``latest``. If ``catchup`` is *False*, the run must also not be earlier than + ``latest``. If ``catchup`` is *disabled*, the run must also not be earlier than the current time, i.e. "missed" schedules are not backfilled. If - ``ignore_first_catchup`` is *True*, the run must also not be earlier than the + ``catchup`` is *ignore_first*, the run must also not be earlier than the first DAG run's ``execution_date`` if there is any, and not earlier than the current time otherwise. These values are generally set on the DAG or task's ``start_date``, - ``end_date``, ``catchup`` and ``ignore_first_catchup`` arguments. + ``end_date`` and ``catchup`` Both ``earliest`` and ``latest``, if not *None*, are inclusive; a DAG run can happen exactly at either point of time. They are guaranteed to be aware @@ -64,8 +65,7 @@ class TimeRestriction(NamedTuple): earliest: DateTime | None latest: DateTime | None - catchup: bool - ignore_first_catchup: bool = False + catchup: Catchup class DagRunInfo(NamedTuple): diff --git a/airflow/timetables/interval.py b/airflow/timetables/interval.py index 0190ea3efa916..252c685bcd3dd 100644 --- a/airflow/timetables/interval.py +++ b/airflow/timetables/interval.py @@ -25,6 +25,7 @@ from airflow.exceptions import AirflowTimetableInvalid from airflow.timetables._cron import CronMixin from airflow.timetables.base import DagRunInfo, DataInterval, Timetable +from airflow.utils.catchup import Catchup from airflow.utils.timezone import convert_to_utc if TYPE_CHECKING: @@ -44,7 +45,7 @@ class _DataIntervalTimetable(Timetable): def _skip_to_latest(self, earliest: DateTime | None) -> DateTime: """Bound the earliest time a run can be scheduled. - This is called when ``catchup=False``. See docstring of subclasses for + This is called when ``catchup`` is disabled. See docstring of subclasses for exact skipping behaviour of a schedule. """ raise NotImplementedError() @@ -85,8 +86,8 @@ def next_dagrun_info( last_automated_data_interval: DataInterval | None, restriction: TimeRestriction, ) -> DagRunInfo | None: - should_catchup = restriction.catchup and ( - not restriction.ignore_first_catchup or last_automated_data_interval is not None + should_catchup = (restriction.catchup == Catchup.ENABLE) or ( + restriction.catchup == Catchup.IGNORE_FIRST and last_automated_data_interval is not None ) earliest = restriction.earliest if not should_catchup: @@ -103,7 +104,7 @@ def next_dagrun_info( # Alignment is needed when DAG has new schedule interval. align_last_data_interval_end = self._align_to_prev(last_automated_data_interval.end) if earliest is not None: - # Catchup is False or DAG has new start date in the future. + # Catchup is disabled or DAG has new start date in the future. # Make sure we get the later one. start = max(align_last_data_interval_end, earliest) else: diff --git a/airflow/timetables/trigger.py b/airflow/timetables/trigger.py index 34ae80ef0496a..de9583f23289b 100644 --- a/airflow/timetables/trigger.py +++ b/airflow/timetables/trigger.py @@ -23,6 +23,7 @@ from airflow.timetables._cron import CronMixin from airflow.timetables.base import DagRunInfo, DataInterval, Timetable +from airflow.utils.catchup import Catchup if TYPE_CHECKING: from dateutil.relativedelta import relativedelta @@ -85,8 +86,8 @@ def next_dagrun_info( last_automated_data_interval: DataInterval | None, restriction: TimeRestriction, ) -> DagRunInfo | None: - should_catchup = restriction.catchup and ( - not restriction.ignore_first_catchup or last_automated_data_interval is not None + should_catchup = (restriction.catchup == Catchup.ENABLE) or ( + restriction.catchup == Catchup.IGNORE_FIRST and last_automated_data_interval is not None ) if should_catchup: if last_automated_data_interval is not None: diff --git a/airflow/utils/catchup.py b/airflow/utils/catchup.py new file mode 100644 index 0000000000000..0086000008e61 --- /dev/null +++ b/airflow/utils/catchup.py @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import warnings +from enum import Enum + + +class Catchup(str, Enum): + """Enum for catchup policy.""" + + ENABLE = "enable" + DISABLE = "disable" + IGNORE_FIRST = "ignore_first" + + +def _catchup_backwards_compatibility(cls, value: str | bool) -> Catchup: + should_warn = False + if isinstance(value, bool): + should_warn = True + value = "enabled" if value else "disabled" + if value in ("True", "False"): + should_warn = True + value = "enabled" if value == "True" else "disabled" + if should_warn: + warnings.warn( + "Passing a boolean to Catchup is deprecated. " + "Please pass one of ['enable', 'disable', 'ignore_first'] instead.", + DeprecationWarning, + stacklevel=2, + ) + return Enum.__new__(cls, value) + + +Catchup.__new__ = _catchup_backwards_compatibility # type: ignore + + +__all__ = ["Catchup"] diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index 06e8d55ce162f..904fe7d99db2b 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -1471,8 +1471,7 @@ export interface components { */ DAGDetail: components["schemas"]["DAG"] & { timezone?: components["schemas"]["Timezone"]; - catchup?: boolean; - ignore_first_catchup?: boolean; + catchup?: string; orientation?: string; concurrency?: number; /** diff --git a/dev/perf/dags/sql_perf_dag.py b/dev/perf/dags/sql_perf_dag.py index 09ac9b0a53c2c..7459d9432bdb6 100644 --- a/dev/perf/dags/sql_perf_dag.py +++ b/dev/perf/dags/sql_perf_dag.py @@ -61,7 +61,7 @@ def generate_parallel_tasks(name_prefix, num_of_tasks, deps): with DAG( DAG_ID, default_args=default_args, - catchup=True, + catchup="enable", schedule=timedelta(minutes=1), is_paused_upon_creation=False, ): diff --git a/docs/apache-airflow-providers-amazon/notifications/chime_notifier_howto_guide.rst b/docs/apache-airflow-providers-amazon/notifications/chime_notifier_howto_guide.rst index c10b8cbae4142..4310f6dea2691 100644 --- a/docs/apache-airflow-providers-amazon/notifications/chime_notifier_howto_guide.rst +++ b/docs/apache-airflow-providers-amazon/notifications/chime_notifier_howto_guide.rst @@ -45,7 +45,7 @@ Example Code: on_success_callback=[ send_chime_notification(chime_conn_id="my_chime_conn", message="The DAG {{ dag.dag_id }} succeeded") ], - catchup=False, + catchup="disable", ): BashOperator( task_id="mytask", diff --git a/docs/apache-airflow-providers-amazon/notifications/sns.rst b/docs/apache-airflow-providers-amazon/notifications/sns.rst index 337e82cf62eb4..79a740530d535 100644 --- a/docs/apache-airflow-providers-amazon/notifications/sns.rst +++ b/docs/apache-airflow-providers-amazon/notifications/sns.rst @@ -59,7 +59,7 @@ Example Code: schedule="@once", start_date=datetime(2023, 1, 1), on_failure_callback=[dag_failure_sns_notification], - catchup=False, + catchup="disable", ): BashOperator( task_id="mytask", diff --git a/docs/apache-airflow-providers-amazon/notifications/sqs.rst b/docs/apache-airflow-providers-amazon/notifications/sqs.rst index 4a2232b006a03..379ad6c13db39 100644 --- a/docs/apache-airflow-providers-amazon/notifications/sqs.rst +++ b/docs/apache-airflow-providers-amazon/notifications/sqs.rst @@ -58,6 +58,6 @@ Example Code: schedule="@once", start_date=datetime(2023, 1, 1, tzinfo=timezone.utc), on_failure_callback=[dag_failure_sqs_notification], - catchup=False, + catchup="disable", ): BashOperator(task_id="mytask", on_failure_callback=[task_failure_sqs_notification], bash_command="fail") diff --git a/docs/apache-airflow-providers-apprise/notifications/apprise_notifier_howto_guide.rst b/docs/apache-airflow-providers-apprise/notifications/apprise_notifier_howto_guide.rst index 0169fe9c4b675..4c23971a80b54 100644 --- a/docs/apache-airflow-providers-apprise/notifications/apprise_notifier_howto_guide.rst +++ b/docs/apache-airflow-providers-apprise/notifications/apprise_notifier_howto_guide.rst @@ -38,7 +38,7 @@ Example Code: dag_id="apprise_notifier_testing", schedule_interval=None, start_date=datetime(2023, 1, 1), - catchup=False, + catchup="disable", on_success_callback=[ send_apprise_notification(body="The dag {{ dag.dag_id }} failed", notify_type=NotifyType.FAILURE) ], diff --git a/docs/apache-airflow-providers-slack/notifications/slackwebhook_notifier_howto_guide.rst b/docs/apache-airflow-providers-slack/notifications/slackwebhook_notifier_howto_guide.rst index bb9e85c67466f..ec7a365d0fda8 100644 --- a/docs/apache-airflow-providers-slack/notifications/slackwebhook_notifier_howto_guide.rst +++ b/docs/apache-airflow-providers-slack/notifications/slackwebhook_notifier_howto_guide.rst @@ -52,7 +52,7 @@ Example Code: schedule="@once", start_date=datetime(2023, 1, 1, tzinfo=timezone.utc), on_failure_callback=[dag_failure_slack_webhook_notification], - catchup=False, + catchup="disable", ): BashOperator( task_id="mytask", on_failure_callback=[task_failure_slack_webhook_notification], bash_command="fail" diff --git a/docs/apache-airflow-providers-smtp/notifications/smtp_notifier_howto_guide.rst b/docs/apache-airflow-providers-smtp/notifications/smtp_notifier_howto_guide.rst index 99895aef3c212..30e387faca1e5 100644 --- a/docs/apache-airflow-providers-smtp/notifications/smtp_notifier_howto_guide.rst +++ b/docs/apache-airflow-providers-smtp/notifications/smtp_notifier_howto_guide.rst @@ -42,7 +42,7 @@ Example Code: dag_id="smtp_notifier", schedule_interval=None, start_date=datetime(2023, 1, 1), - catchup=False, + catchup="disable", on_failure_callback=[ send_smtp_notification( from_email="someone@mail.com", diff --git a/docs/apache-airflow/administration-and-deployment/lineage.rst b/docs/apache-airflow/administration-and-deployment/lineage.rst index 7b967d3f10e61..5bdcd1ae46e29 100644 --- a/docs/apache-airflow/administration-and-deployment/lineage.rst +++ b/docs/apache-airflow/administration-and-deployment/lineage.rst @@ -45,7 +45,7 @@ works. dag_id="example_lineage", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule="0 0 * * *", - catchup=False, + catchup="disable", dagrun_timeout=datetime.timedelta(minutes=60), ) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst index c21752ab79e8f..89ff260898e75 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst @@ -79,7 +79,7 @@ In the following example, failures in any task call the ``task_failure_alert`` f schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), dagrun_timeout=datetime.timedelta(minutes=60), - catchup=False, + catchup="disable", on_success_callback=None, on_failure_callback=task_failure_alert, tags=["example"], diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index 81af9367f4ece..ae8b7af0976af 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -133,7 +133,7 @@ Bad example: dag_id="example_python_operator", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: @@ -159,7 +159,7 @@ Good example: dag_id="example_python_operator", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: @@ -212,7 +212,7 @@ Imagine this code: dag_id="example_python_operator", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: operator = PythonOperator( @@ -246,7 +246,7 @@ What you can do check it, add to your code you want to check some print statemen dag_id="example_python_operator", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) as dag: operator = PythonOperator( @@ -461,7 +461,7 @@ It's easier to grab the concept with an example. Let's say that we have the foll dag_id="watcher_example", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: failing_task = BashOperator(task_id="failing_task", bash_command="exit 1", retries=0) passing_task = BashOperator(task_id="passing_task", bash_command="echo passing_task") diff --git a/docs/apache-airflow/core-concepts/dag-run.rst b/docs/apache-airflow/core-concepts/dag-run.rst index e477561e0f953..160a836b6f4ef 100644 --- a/docs/apache-airflow/core-concepts/dag-run.rst +++ b/docs/apache-airflow/core-concepts/dag-run.rst @@ -127,7 +127,7 @@ 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.), -then you will want to turn catchup off. This can be done by setting ``catchup=False`` in DAG or ``catchup_by_default=False`` +then you will want to turn catchup off. This can be done by setting ``catchup="disable"`` in DAG or ``catchup_by_default=False`` in the configuration file. When turned off, the scheduler creates a DAG run only for the latest interval. .. code-block:: python @@ -152,7 +152,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="@daily", - catchup=False, + catchup="disable", ) In the example above, if the DAG is picked up by the scheduler daemon on @@ -266,7 +266,7 @@ Example of a parameterized DAG: "example_parameterized_dag", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", ) parameterized_task = BashOperator( diff --git a/docs/apache-airflow/core-concepts/dags.rst b/docs/apache-airflow/core-concepts/dags.rst index 86f4aaad2eda9..e38dfb24db3c3 100644 --- a/docs/apache-airflow/core-concepts/dags.rst +++ b/docs/apache-airflow/core-concepts/dags.rst @@ -662,7 +662,7 @@ This is especially useful if your tasks are built dynamically from configuration "my_dag", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule="@daily", - catchup=False, + catchup="disable", ) dag.doc_md = __doc__ diff --git a/docs/apache-airflow/core-concepts/executor/kubernetes.rst b/docs/apache-airflow/core-concepts/executor/kubernetes.rst index e19264c96e5e5..e9fee08bbc4aa 100644 --- a/docs/apache-airflow/core-concepts/executor/kubernetes.rst +++ b/docs/apache-airflow/core-concepts/executor/kubernetes.rst @@ -176,7 +176,7 @@ Here is an example of a task with both features: dag_id="example_pod_template_file", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example3"], ) as dag: executor_config_template = { diff --git a/docs/apache-airflow/core-concepts/operators.rst b/docs/apache-airflow/core-concepts/operators.rst index ddbcf7da7ab2d..27402d5e6834f 100644 --- a/docs/apache-airflow/core-concepts/operators.rst +++ b/docs/apache-airflow/core-concepts/operators.rst @@ -218,7 +218,7 @@ you can pass ``render_template_as_native_obj=True`` to the DAG as follows: dag_id="example_template_as_python_object", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", render_template_as_native_obj=True, ) diff --git a/docs/apache-airflow/core-concepts/taskflow.rst b/docs/apache-airflow/core-concepts/taskflow.rst index c1de77d68cfc8..b6e681db282af 100644 --- a/docs/apache-airflow/core-concepts/taskflow.rst +++ b/docs/apache-airflow/core-concepts/taskflow.rst @@ -118,7 +118,7 @@ a ``Dataset``, which is ``@attr.define`` decorated, together with TaskFlow. now = pendulum.now() - @dag(start_date=now, schedule="@daily", catchup=False) + @dag(start_date=now, schedule="@daily", catchup="disable") def etl(): @task() def retrieve(src: Dataset) -> dict: diff --git a/docs/apache-airflow/howto/dynamic-dag-generation.rst b/docs/apache-airflow/howto/dynamic-dag-generation.rst index 5d542a29320b7..7131c213a54f7 100644 --- a/docs/apache-airflow/howto/dynamic-dag-generation.rst +++ b/docs/apache-airflow/howto/dynamic-dag-generation.rst @@ -84,7 +84,7 @@ Then you can import and use the ``ALL_TASKS`` constant in all your DAGs like tha dag_id="my_dag", schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ): for task in ALL_TASKS: # create your operators and relations here diff --git a/docs/apache-airflow/howto/usage-cli.rst b/docs/apache-airflow/howto/usage-cli.rst index abf17ea7d1899..d2550e22e0333 100644 --- a/docs/apache-airflow/howto/usage-cli.rst +++ b/docs/apache-airflow/howto/usage-cli.rst @@ -250,7 +250,7 @@ Keep in mind that some tables have foreign key relationships defined with ``ON D Special handling for DAG runs ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Commonly, Airflow determines which DagRun to run next by looking up the latest DagRun. If you delete all DAG runs, Airflow may schedule an old DAG run that was already completed, e.g. if you have set ``catchup=True``. So the ``db clean`` command will preserve the latest non-manually-triggered DAG run to preserve continuity in scheduling. +Commonly, Airflow determines which DagRun to run next by looking up the latest DagRun. If you delete all DAG runs, Airflow may schedule an old DAG run that was already completed, e.g. if you have set ``catchup="enable"``. So the ``db clean`` command will preserve the latest non-manually-triggered DAG run to preserve continuity in scheduling. Considerations for backfillable DAGs ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/docs/apache-airflow/tutorial/pipeline.rst b/docs/apache-airflow/tutorial/pipeline.rst index fca2418a701c3..4723d3bc78ea3 100644 --- a/docs/apache-airflow/tutorial/pipeline.rst +++ b/docs/apache-airflow/tutorial/pipeline.rst @@ -224,7 +224,7 @@ Putting all of the pieces together, we have our completed DAG. dag_id="process-employees", schedule_interval="0 0 * * *", start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", dagrun_timeout=datetime.timedelta(minutes=60), ) def ProcessEmployees(): 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 54f59331e0c33..0c1b0c7fec4ba 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 @@ -36,7 +36,7 @@ schedule="*/10 * * * *", default_args={"depends_on_past": True}, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", ) run_this_1 = EmptyOperator(task_id="run_this_1", dag=dag) diff --git a/tests/dags/test_dags_folder.py b/tests/dags/test_dags_folder.py index f8402bfe71faf..ffd3b84762684 100644 --- a/tests/dags/test_dags_folder.py +++ b/tests/dags/test_dags_folder.py @@ -26,7 +26,7 @@ dag_id="test_dags_folder", schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", ) as dag: @task(task_id="task") diff --git a/tests/dags/test_datasets.py b/tests/dags/test_datasets.py index 4bdef9f6978cb..fc006b7e5e987 100644 --- a/tests/dags/test_datasets.py +++ b/tests/dags/test_datasets.py @@ -35,7 +35,7 @@ def raise_skip_exc(): dag_with_skip_task = DAG( dag_id="dag_with_skip_task", - catchup=False, + catchup="disable", start_date=datetime(2020, 1, 1), schedule="@daily", tags=["upstream-skipping"], @@ -49,7 +49,7 @@ def raise_skip_exc(): with DAG( dag_id="dag_that_follows_dag_with_skip", - catchup=False, + catchup="disable", start_date=datetime(2020, 1, 1), schedule=[skip_task_dag_dataset], tags=["downstream-skipped"], @@ -66,7 +66,7 @@ def raise_fail_exc(): dag_with_fail_task = DAG( dag_id="dag_with_fail_task", - catchup=False, + catchup="disable", start_date=datetime(2020, 1, 1), schedule="@daily", tags=["upstream-skipping"], @@ -80,7 +80,7 @@ def raise_fail_exc(): with DAG( dag_id="dag_that_follows_dag_with_fail", - catchup=False, + catchup="disable", start_date=datetime(2020, 1, 1), schedule=[fail_task_dag_dataset], tags=["downstream-failed"], diff --git a/tests/dags/test_sensor.py b/tests/dags/test_sensor.py index fd8dbe6634a49..5673ec7bd269b 100644 --- a/tests/dags/test_sensor.py +++ b/tests/dags/test_sensor.py @@ -24,7 +24,7 @@ from airflow.utils import timezone with DAG( - dag_id="test_sensor", start_date=datetime.datetime(2022, 1, 1), catchup=False, schedule="@once" + dag_id="test_sensor", start_date=datetime.datetime(2022, 1, 1), catchup="disable", schedule="@once" ) as dag: @task diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index 8e745e1e083f1..92b743ee5864d 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -3736,7 +3736,7 @@ def test_bulk_write_to_db_external_trigger_dont_skip_scheduled_run(self, dag_mak dag_id="test_bulk_write_to_db_external_trigger_dont_skip_scheduled_run", schedule="*/1 * * * *", max_active_runs=5, - catchup=True, + catchup="enable", ) as dag: EmptyOperator(task_id="dummy") @@ -3845,7 +3845,7 @@ def test_do_schedule_max_active_runs_dag_timed_out(self, dag_maker): dag_id="test_max_active_run_with_dag_timed_out", schedule="@once", max_active_runs=1, - catchup=True, + catchup="enable", dagrun_timeout=datetime.timedelta(seconds=1), ) as dag: task1 = BashOperator( @@ -4964,7 +4964,7 @@ def test_catchup_works_correctly(self, dag_maker): dag_id="test_catchup_schedule_dag", schedule=timedelta(days=1), start_date=DEFAULT_DATE, - catchup=True, + catchup="enable", max_active_runs=1, session=session, ) as dag: diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 5d9c68b9e6b7a..ce23c83e5c525 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -78,6 +78,7 @@ OnceTimetable, ) from airflow.utils import timezone +from airflow.utils.catchup import Catchup from airflow.utils.file import list_py_file_paths from airflow.utils.session import create_session, provide_session from airflow.utils.state import DagRunState, State, TaskInstanceState @@ -2218,7 +2219,7 @@ def make_dag(dag_id, schedule, start_date, catchup): dag_id="dag_without_catchup_ten_minute", schedule="*/10 * * * *", start_date=six_hours_ago_to_the_hour, - catchup=False, + catchup="disable", ) next_date, _ = dag1.next_dagrun_info(None) # The DR should be scheduled in the last half an hour, not 6 hours ago @@ -2229,7 +2230,7 @@ def make_dag(dag_id, schedule, start_date, catchup): dag_id="dag_without_catchup_hourly", schedule="@hourly", start_date=six_hours_ago_to_the_hour, - catchup=False, + catchup="disable", ) next_date, _ = dag2.next_dagrun_info(None) @@ -2242,7 +2243,7 @@ def make_dag(dag_id, schedule, start_date, catchup): dag_id="dag_without_catchup_once", schedule="@once", start_date=six_hours_ago_to_the_hour, - catchup=False, + catchup="disable", ) next_date, _ = dag3.next_dagrun_info(None) @@ -2254,13 +2255,13 @@ def make_dag(dag_id, schedule, start_date, catchup): def test_next_dagrun_info_timedelta_schedule_and_catchup_false(self, schedule): """ Test that the dag file processor does not create multiple dagruns - if a dag is scheduled with 'timedelta' and catchup=False + if a dag is scheduled with 'timedelta' and catchup="disable" """ dag = DAG( "test_scheduler_dagrun_once_with_timedelta_and_catchup_false", start_date=timezone.datetime(2015, 1, 1), schedule=schedule, - catchup=False, + catchup="disable", ) next_info = dag.next_dagrun_info(None) @@ -2274,13 +2275,13 @@ def test_next_dagrun_info_timedelta_schedule_and_catchup_false(self, schedule): def test_next_dagrun_info_timedelta_schedule_and_catchup_true(self): """ Test that the dag file processor creates multiple dagruns - if a dag is scheduled with 'timedelta' and catchup=True + if a dag is scheduled with 'timedelta' and catchup="enable" """ dag = DAG( "test_scheduler_dagrun_once_with_timedelta_and_catchup_true", start_date=timezone.datetime(2020, 5, 1), schedule=timedelta(days=1), - catchup=True, + catchup="enable", ) next_info = dag.next_dagrun_info(None) @@ -2297,26 +2298,25 @@ def test_next_dagrun_info_timedelta_schedule_and_catchup_true(self): assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 4) @time_machine.travel(timezone.datetime(2020, 5, 4)) - def test_next_dagrun_info_timedelta_schedule_and_catchup_true_and_ignore_first_catchup_true(self): + def test_next_dagrun_info_timedelta_schedule_and_catchup_ignore_first(self): """ Test that the dag file processor creates multiple dagruns - if a dag is scheduled with 'timedelta' and catchup=True and ignore_first_depends_on_past=True + if a dag is scheduled with 'timedelta' and catchup='ignore_first' """ dag = DAG( "test_scheduler_dagrun_once_with_timedelta_and_catchup_true", start_date=timezone.datetime(2020, 5, 1), schedule=timedelta(days=1), - catchup=True, - ignore_first_catchup=True, + catchup="ignore_first", ) next_info = dag.next_dagrun_info(None) - # # if it is the first dagrun, the scheduling should be the same as catchup=False + # # if it is the first dagrun, the scheduling should be the same as catchup="disable" # for the first dagrun, the dagrun date is the nearest data interval to current time (<= current time) assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 3) next_info = dag.next_dagrun_info(timezone.datetime(2020, 5, 1)) - # if it is not the first dagrun, the scheduling should be the same as catchup=True + # if it is not the first dagrun, the scheduling should be the same as catchup="enable" assert next_info and next_info.logical_date == timezone.datetime(2020, 5, 2) next_info = dag.next_dagrun_info(next_info.data_interval) @@ -2337,7 +2337,7 @@ def next_dagrun_info(self, last_automated_data_interval, restriction): "test_next_dagrun_info_timetable_exception", start_date=timezone.datetime(2020, 5, 1), timetable=FailingTimetable(), - catchup=True, + catchup="enable", ) def _check_logs(records: list[logging.LogRecord], data_interval: DataInterval) -> None: @@ -3441,12 +3441,12 @@ def test_get_next_data_interval( (DEFAULT_DATE + timedelta(days=1), DEFAULT_DATE + timedelta(days=2)), (DEFAULT_DATE + timedelta(days=3), DEFAULT_DATE + timedelta(days=4)), ], - TimeRestriction(DEFAULT_DATE, DEFAULT_DATE + timedelta(days=4), True), + TimeRestriction(DEFAULT_DATE, DEFAULT_DATE + timedelta(days=4), Catchup.ENABLE), ], [ (DEFAULT_DATE, None), [(DEFAULT_DATE, DEFAULT_DATE + timedelta(days=1)), (DEFAULT_DATE, None)], - TimeRestriction(DEFAULT_DATE, None, True), + TimeRestriction(DEFAULT_DATE, None, Catchup.ENABLE), ], ], ) @@ -4059,7 +4059,7 @@ def test_clearing_behavior_just_teardown(self): def test_validate_setup_teardown_trigger_rule(self): with DAG( - dag_id="direct_setup_trigger_rule", start_date=pendulum.now(), schedule=None, catchup=False + dag_id="direct_setup_trigger_rule", start_date=pendulum.now(), schedule=None, catchup="disable" ) as dag: s1, w1 = self.make_tasks(dag, "s1, w1") s1 >> w1 diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index 27ce80df1ab76..aa1f417fae2e8 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -2236,7 +2236,7 @@ def test_task2(): @staticmethod def _test_previous_dates_setup( schedule_interval: str | datetime.timedelta | None, - catchup: bool, + catchup: str, scenario: list[TaskInstanceState], dag_maker, ) -> list: @@ -2269,12 +2269,12 @@ def get_test_ti(execution_date: pendulum.DateTime, state: str) -> TI: return ret _prev_dates_param_list = [ - pytest.param("0 0 * * * ", True, id="cron/catchup"), - pytest.param("0 0 * * *", False, id="cron/no-catchup"), - pytest.param(None, True, id="no-sched/catchup"), - pytest.param(None, False, id="no-sched/no-catchup"), - pytest.param(datetime.timedelta(days=1), True, id="timedelta/catchup"), - pytest.param(datetime.timedelta(days=1), False, id="timedelta/no-catchup"), + pytest.param("0 0 * * * ", "enable", id="cron/catchup"), + pytest.param("0 0 * * *", "disable", id="cron/no-catchup"), + pytest.param(None, "enable", id="no-sched/catchup"), + pytest.param(None, "disable", id="no-sched/no-catchup"), + pytest.param(datetime.timedelta(days=1), "enable", id="timedelta/catchup"), + pytest.param(datetime.timedelta(days=1), "disable", id="timedelta/no-catchup"), ] @pytest.mark.parametrize("schedule_interval, catchup", _prev_dates_param_list) diff --git a/tests/system/providers/amazon/aws/example_appflow.py b/tests/system/providers/amazon/aws/example_appflow.py index 0fb2764c0b71c..a3bbc8ebbb72c 100644 --- a/tests/system/providers/amazon/aws/example_appflow.py +++ b/tests/system/providers/amazon/aws/example_appflow.py @@ -38,7 +38,7 @@ DAG_ID, schedule="@once", start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_appflow_run.py b/tests/system/providers/amazon/aws/example_appflow_run.py index d73a426127808..3af5a94f39d4f 100644 --- a/tests/system/providers/amazon/aws/example_appflow_run.py +++ b/tests/system/providers/amazon/aws/example_appflow_run.py @@ -143,7 +143,7 @@ def delete_flow(flow_name: str): dag_id=DAG_ID, schedule="@once", start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_athena.py b/tests/system/providers/amazon/aws/example_athena.py index 92c56c24ee7fc..bc1fdf5a1eb02 100644 --- a/tests/system/providers/amazon/aws/example_athena.py +++ b/tests/system/providers/amazon/aws/example_athena.py @@ -66,7 +66,7 @@ def read_results_from_s3(bucket_name, query_execution_id): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py b/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py index 0109b8813063d..2e4575211166d 100644 --- a/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py +++ b/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py @@ -34,7 +34,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_batch.py b/tests/system/providers/amazon/aws/example_batch.py index 77c5d21997364..62c24b49be001 100644 --- a/tests/system/providers/amazon/aws/example_batch.py +++ b/tests/system/providers/amazon/aws/example_batch.py @@ -156,7 +156,7 @@ def delete_job_queue(job_queue_name): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_cloudformation.py b/tests/system/providers/amazon/aws/example_cloudformation.py index a376addd3489b..ad1377b6aa4b9 100644 --- a/tests/system/providers/amazon/aws/example_cloudformation.py +++ b/tests/system/providers/amazon/aws/example_cloudformation.py @@ -52,7 +52,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_datasync.py b/tests/system/providers/amazon/aws/example_datasync.py index 8f57d1c656068..e2b3a0c46e70b 100644 --- a/tests/system/providers/amazon/aws/example_datasync.py +++ b/tests/system/providers/amazon/aws/example_datasync.py @@ -120,7 +120,7 @@ def delete_locations(locations): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_dms.py b/tests/system/providers/amazon/aws/example_dms.py index 32506a0268bfb..2dd1fdee1c060 100644 --- a/tests/system/providers/amazon/aws/example_dms.py +++ b/tests/system/providers/amazon/aws/example_dms.py @@ -228,7 +228,7 @@ def delete_security_group(security_group_id: str, security_group_name: str): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_dynamodb.py b/tests/system/providers/amazon/aws/example_dynamodb.py index 8316cab0e6d1a..5e9086d16b63e 100644 --- a/tests/system/providers/amazon/aws/example_dynamodb.py +++ b/tests/system/providers/amazon/aws/example_dynamodb.py @@ -70,7 +70,7 @@ def delete_table(table_name: str): dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_dynamodb_to_s3.py b/tests/system/providers/amazon/aws/example_dynamodb_to_s3.py index dc08e2d5b9dcc..9b891751a66af 100644 --- a/tests/system/providers/amazon/aws/example_dynamodb_to_s3.py +++ b/tests/system/providers/amazon/aws/example_dynamodb_to_s3.py @@ -110,7 +110,7 @@ def delete_dynamodb_table(table_name: str): dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_ec2.py b/tests/system/providers/amazon/aws/example_ec2.py index 506d73908b224..a66589f4f8e01 100644 --- a/tests/system/providers/amazon/aws/example_ec2.py +++ b/tests/system/providers/amazon/aws/example_ec2.py @@ -86,7 +86,7 @@ def parse_response(instance_ids: list): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_ecs.py b/tests/system/providers/amazon/aws/example_ecs.py index 98617dcbcb6d4..a8ab118a9a4f5 100644 --- a/tests/system/providers/amazon/aws/example_ecs.py +++ b/tests/system/providers/amazon/aws/example_ecs.py @@ -77,7 +77,7 @@ def clean_logs(group_name: str): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_ecs_fargate.py b/tests/system/providers/amazon/aws/example_ecs_fargate.py index 6a299c788ca19..59a4ea02c3007 100644 --- a/tests/system/providers/amazon/aws/example_ecs_fargate.py +++ b/tests/system/providers/amazon/aws/example_ecs_fargate.py @@ -90,7 +90,7 @@ def delete_cluster(cluster_name: str) -> None: schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_eks_templated.py b/tests/system/providers/amazon/aws/example_eks_templated.py index 7ad8bc77c2ece..d6904ea0096d8 100644 --- a/tests/system/providers/amazon/aws/example_eks_templated.py +++ b/tests/system/providers/amazon/aws/example_eks_templated.py @@ -54,7 +54,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example", "templated"], - catchup=False, + catchup="disable", # render_template_as_native_obj=True is what converts the Jinja to Python objects, instead of a string. render_template_as_native_obj=True, ) as dag: diff --git a/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py b/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py index ae67a26588bdc..0cc5fcfbf0f4a 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py +++ b/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py @@ -53,7 +53,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py b/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py index 9cce50b9bac46..2ff2630ab7740 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py +++ b/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py @@ -57,7 +57,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py b/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py index 9bfce2cc1696f..32cfffacac7ee 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py +++ b/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py @@ -66,7 +66,7 @@ def delete_launch_template(template_name: str): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_eks_with_nodegroups.py b/tests/system/providers/amazon/aws/example_eks_with_nodegroups.py index 78a72b68d108b..006cc784cf71b 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_nodegroups.py +++ b/tests/system/providers/amazon/aws/example_eks_with_nodegroups.py @@ -68,7 +68,7 @@ def delete_launch_template(template_name: str): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_emr.py b/tests/system/providers/amazon/aws/example_emr.py index 32f10f6f1bc73..b909c7c832d6c 100644 --- a/tests/system/providers/amazon/aws/example_emr.py +++ b/tests/system/providers/amazon/aws/example_emr.py @@ -129,7 +129,7 @@ def get_step_id(step_ids: list): dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_emr_eks.py b/tests/system/providers/amazon/aws/example_emr_eks.py index 28dc7ac3c27e1..8116f519452ef 100644 --- a/tests/system/providers/amazon/aws/example_emr_eks.py +++ b/tests/system/providers/amazon/aws/example_emr_eks.py @@ -192,7 +192,7 @@ def delete_virtual_cluster(virtual_cluster_id): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_emr_notebook_execution.py b/tests/system/providers/amazon/aws/example_emr_notebook_execution.py index 86fc3055a1624..01f67a0d726c3 100644 --- a/tests/system/providers/amazon/aws/example_emr_notebook_execution.py +++ b/tests/system/providers/amazon/aws/example_emr_notebook_execution.py @@ -42,7 +42,7 @@ dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_emr_serverless.py b/tests/system/providers/amazon/aws/example_emr_serverless.py index 0c0a19e17bffb..03ca7c3f3b253 100644 --- a/tests/system/providers/amazon/aws/example_emr_serverless.py +++ b/tests/system/providers/amazon/aws/example_emr_serverless.py @@ -46,7 +46,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_eventbridge.py b/tests/system/providers/amazon/aws/example_eventbridge.py index 9b3bb922e1669..583f3540a08a8 100644 --- a/tests/system/providers/amazon/aws/example_eventbridge.py +++ b/tests/system/providers/amazon/aws/example_eventbridge.py @@ -46,7 +46,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_ftp_to_s3.py b/tests/system/providers/amazon/aws/example_ftp_to_s3.py index ef62bdc06a733..6a746d05bee49 100644 --- a/tests/system/providers/amazon/aws/example_ftp_to_s3.py +++ b/tests/system/providers/amazon/aws/example_ftp_to_s3.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_gcs_to_s3.py b/tests/system/providers/amazon/aws/example_gcs_to_s3.py index 198f9729feb3f..de33205c4960e 100644 --- a/tests/system/providers/amazon/aws/example_gcs_to_s3.py +++ b/tests/system/providers/amazon/aws/example_gcs_to_s3.py @@ -46,7 +46,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_glacier_to_gcs.py b/tests/system/providers/amazon/aws/example_glacier_to_gcs.py index c4eae47f009ce..be1e6982a6dc7 100644 --- a/tests/system/providers/amazon/aws/example_glacier_to_gcs.py +++ b/tests/system/providers/amazon/aws/example_glacier_to_gcs.py @@ -52,7 +52,7 @@ def delete_vault(vault_name): schedule="@once", start_date=datetime(2021, 1, 1), # Override to match your needs tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_glue.py b/tests/system/providers/amazon/aws/example_glue.py index bc1a80e9535ad..83b501c196cc6 100644 --- a/tests/system/providers/amazon/aws/example_glue.py +++ b/tests/system/providers/amazon/aws/example_glue.py @@ -88,7 +88,7 @@ def glue_cleanup(crawler_name: str, job_name: str, db_name: str) -> None: schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_google_api_sheets_to_s3.py b/tests/system/providers/amazon/aws/example_google_api_sheets_to_s3.py index 401fb86634865..82d63ec7f52c0 100644 --- a/tests/system/providers/amazon/aws/example_google_api_sheets_to_s3.py +++ b/tests/system/providers/amazon/aws/example_google_api_sheets_to_s3.py @@ -42,7 +42,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_google_api_youtube_to_s3.py b/tests/system/providers/amazon/aws/example_google_api_youtube_to_s3.py index cb2447cd7cdfd..f5ed6029c982f 100644 --- a/tests/system/providers/amazon/aws/example_google_api_youtube_to_s3.py +++ b/tests/system/providers/amazon/aws/example_google_api_youtube_to_s3.py @@ -118,7 +118,7 @@ def transform_video_ids(**kwargs): schedule="@once", start_date=datetime(2021, 1, 1), # Override to match your needs tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_hive_to_dynamodb.py b/tests/system/providers/amazon/aws/example_hive_to_dynamodb.py index bdc5c26b646af..cee6cb1383d8a 100644 --- a/tests/system/providers/amazon/aws/example_hive_to_dynamodb.py +++ b/tests/system/providers/amazon/aws/example_hive_to_dynamodb.py @@ -116,7 +116,7 @@ def configure_hive_connection(connection_id, hostname): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_imap_attachment_to_s3.py b/tests/system/providers/amazon/aws/example_imap_attachment_to_s3.py index 6453e3fdff554..4ecf015b170c8 100644 --- a/tests/system/providers/amazon/aws/example_imap_attachment_to_s3.py +++ b/tests/system/providers/amazon/aws/example_imap_attachment_to_s3.py @@ -47,7 +47,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_lambda.py b/tests/system/providers/amazon/aws/example_lambda.py index 4ba74a26d5f22..aa71be7b3482a 100644 --- a/tests/system/providers/amazon/aws/example_lambda.py +++ b/tests/system/providers/amazon/aws/example_lambda.py @@ -71,7 +71,7 @@ def delete_lambda(function_name: str): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_local_to_s3.py b/tests/system/providers/amazon/aws/example_local_to_s3.py index da979bcd37854..7b27bfe7eb4bc 100644 --- a/tests/system/providers/amazon/aws/example_local_to_s3.py +++ b/tests/system/providers/amazon/aws/example_local_to_s3.py @@ -52,7 +52,7 @@ def delete_temp_file(): schedule="@once", start_date=datetime(2021, 1, 1), # Override to match your needs tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_mongo_to_s3.py b/tests/system/providers/amazon/aws/example_mongo_to_s3.py index 33bcd171c0d82..7aa0b8f8f6cab 100644 --- a/tests/system/providers/amazon/aws/example_mongo_to_s3.py +++ b/tests/system/providers/amazon/aws/example_mongo_to_s3.py @@ -38,7 +38,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_quicksight.py b/tests/system/providers/amazon/aws/example_quicksight.py index ccf4746122f65..f8015b28513a0 100644 --- a/tests/system/providers/amazon/aws/example_quicksight.py +++ b/tests/system/providers/amazon/aws/example_quicksight.py @@ -129,7 +129,7 @@ def delete_ingestion(aws_account_id: str, dataset_name: str, ingestion_name: str schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() account_id = get_aws_account_id() diff --git a/tests/system/providers/amazon/aws/example_rds_event.py b/tests/system/providers/amazon/aws/example_rds_event.py index 0d397f43066e4..cd94da4c6e2b8 100644 --- a/tests/system/providers/amazon/aws/example_rds_event.py +++ b/tests/system/providers/amazon/aws/example_rds_event.py @@ -53,7 +53,7 @@ def delete_sns_topic(topic_arn) -> None: schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_rds_export.py b/tests/system/providers/amazon/aws/example_rds_export.py index 385a1871e7e27..27e89cfdc3c9f 100644 --- a/tests/system/providers/amazon/aws/example_rds_export.py +++ b/tests/system/providers/amazon/aws/example_rds_export.py @@ -57,7 +57,7 @@ def get_snapshot_arn(snapshot_name: str) -> str: schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_rds_instance.py b/tests/system/providers/amazon/aws/example_rds_instance.py index f917fb128486c..9175a5bbc0781 100644 --- a/tests/system/providers/amazon/aws/example_rds_instance.py +++ b/tests/system/providers/amazon/aws/example_rds_instance.py @@ -44,7 +44,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() rds_db_identifier = f"{test_context[ENV_ID_KEY]}-database" diff --git a/tests/system/providers/amazon/aws/example_rds_snapshot.py b/tests/system/providers/amazon/aws/example_rds_snapshot.py index 5585f339feec2..364dcb3c8039e 100644 --- a/tests/system/providers/amazon/aws/example_rds_snapshot.py +++ b/tests/system/providers/amazon/aws/example_rds_snapshot.py @@ -41,7 +41,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_redshift.py b/tests/system/providers/amazon/aws/example_redshift.py index 70da271fdf5f2..b4df552d0d752 100644 --- a/tests/system/providers/amazon/aws/example_redshift.py +++ b/tests/system/providers/amazon/aws/example_redshift.py @@ -102,7 +102,7 @@ def delete_security_group(sec_group_id: str, sec_group_name: str): dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_redshift_s3_transfers.py b/tests/system/providers/amazon/aws/example_redshift_s3_transfers.py index 8d453908601b2..06756c592a07e 100644 --- a/tests/system/providers/amazon/aws/example_redshift_s3_transfers.py +++ b/tests/system/providers/amazon/aws/example_redshift_s3_transfers.py @@ -123,7 +123,7 @@ def delete_security_group(sec_group_id: str, sec_group_name: str): dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_s3.py b/tests/system/providers/amazon/aws/example_s3.py index 87806a0bf0a5a..d6b5772f46862 100644 --- a/tests/system/providers/amazon/aws/example_s3.py +++ b/tests/system/providers/amazon/aws/example_s3.py @@ -59,7 +59,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_s3_to_ftp.py b/tests/system/providers/amazon/aws/example_s3_to_ftp.py index bf909ddc8a219..73ee48bdae1e9 100644 --- a/tests/system/providers/amazon/aws/example_s3_to_ftp.py +++ b/tests/system/providers/amazon/aws/example_s3_to_ftp.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_s3_to_sftp.py b/tests/system/providers/amazon/aws/example_s3_to_sftp.py index 78825d7151468..a7f57915b906b 100644 --- a/tests/system/providers/amazon/aws/example_s3_to_sftp.py +++ b/tests/system/providers/amazon/aws/example_s3_to_sftp.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_s3_to_sql.py b/tests/system/providers/amazon/aws/example_s3_to_sql.py index 42eb46d1b4b0e..b19d30906d265 100644 --- a/tests/system/providers/amazon/aws/example_s3_to_sql.py +++ b/tests/system/providers/amazon/aws/example_s3_to_sql.py @@ -107,7 +107,7 @@ def delete_security_group(sec_group_id: str, sec_group_name: str): dag_id=DAG_ID, start_date=datetime(2023, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_sagemaker.py b/tests/system/providers/amazon/aws/example_sagemaker.py index 445e95410a683..39b75496deb76 100644 --- a/tests/system/providers/amazon/aws/example_sagemaker.py +++ b/tests/system/providers/amazon/aws/example_sagemaker.py @@ -466,7 +466,7 @@ def delete_docker_image(image_name): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py b/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py index 226b690cbf409..dbc2aafee898f 100644 --- a/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py +++ b/tests/system/providers/amazon/aws/example_sagemaker_endpoint.py @@ -193,7 +193,7 @@ def set_up(env_id, role_arn, ti=None): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_sagemaker_notebook.py b/tests/system/providers/amazon/aws/example_sagemaker_notebook.py index b61bfa822bb04..463a5057cd6d6 100644 --- a/tests/system/providers/amazon/aws/example_sagemaker_notebook.py +++ b/tests/system/providers/amazon/aws/example_sagemaker_notebook.py @@ -40,7 +40,7 @@ schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_sagemaker_pipeline.py b/tests/system/providers/amazon/aws/example_sagemaker_pipeline.py index 48bf92fa7fc33..560d8cc2fbae7 100644 --- a/tests/system/providers/amazon/aws/example_sagemaker_pipeline.py +++ b/tests/system/providers/amazon/aws/example_sagemaker_pipeline.py @@ -65,7 +65,7 @@ def delete_pipeline(name: str): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context[ENV_ID_KEY] diff --git a/tests/system/providers/amazon/aws/example_salesforce_to_s3.py b/tests/system/providers/amazon/aws/example_salesforce_to_s3.py index 96c585661c5a7..79fb95f7c4512 100644 --- a/tests/system/providers/amazon/aws/example_salesforce_to_s3.py +++ b/tests/system/providers/amazon/aws/example_salesforce_to_s3.py @@ -37,7 +37,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 7, 8), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_sftp_to_s3.py b/tests/system/providers/amazon/aws/example_sftp_to_s3.py index c5cae1ebfa68c..93f5b2ff50559 100644 --- a/tests/system/providers/amazon/aws/example_sftp_to_s3.py +++ b/tests/system/providers/amazon/aws/example_sftp_to_s3.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_sns.py b/tests/system/providers/amazon/aws/example_sns.py index 4cfb2b89153c8..4304d270e0e8f 100644 --- a/tests/system/providers/amazon/aws/example_sns.py +++ b/tests/system/providers/amazon/aws/example_sns.py @@ -47,7 +47,7 @@ def delete_topic(topic_arn) -> None: schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_sql_to_s3.py b/tests/system/providers/amazon/aws/example_sql_to_s3.py index cff8072d74fdd..b71249f2a1c63 100644 --- a/tests/system/providers/amazon/aws/example_sql_to_s3.py +++ b/tests/system/providers/amazon/aws/example_sql_to_s3.py @@ -112,7 +112,7 @@ def delete_security_group(sec_group_id: str, sec_group_name: str): dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/amazon/aws/example_sqs.py b/tests/system/providers/amazon/aws/example_sqs.py index 0a14d5c446408..b664d1e292f7f 100644 --- a/tests/system/providers/amazon/aws/example_sqs.py +++ b/tests/system/providers/amazon/aws/example_sqs.py @@ -47,7 +47,7 @@ def delete_queue(queue_url): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] diff --git a/tests/system/providers/amazon/aws/example_step_functions.py b/tests/system/providers/amazon/aws/example_step_functions.py index 1c0e90979d694..7e4eadb377451 100644 --- a/tests/system/providers/amazon/aws/example_step_functions.py +++ b/tests/system/providers/amazon/aws/example_step_functions.py @@ -68,7 +68,7 @@ def delete_state_machine(state_machine_arn): schedule="@once", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # This context contains the ENV_ID and any env variables requested when the # task was built above. Access the info as you would any other TaskFlow task. diff --git a/tests/system/providers/apache/beam/example_beam.py b/tests/system/providers/apache/beam/example_beam.py index b89e7187b14b0..663a405d5f236 100644 --- a/tests/system/providers/apache/beam/example_beam.py +++ b/tests/system/providers/apache/beam/example_beam.py @@ -34,7 +34,7 @@ "example_beam_native_java_direct_runner", schedule=None, # Override to match your needs start_date=START_DATE, - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_start_java_direct_runner_pipeline] 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 70792624df6e1..7cb7c88fdac00 100644 --- a/tests/system/providers/apache/beam/example_beam_java_flink.py +++ b/tests/system/providers/apache/beam/example_beam_java_flink.py @@ -34,7 +34,7 @@ "example_beam_native_java_flink_runner", schedule=None, # Override to match your needs start_date=START_DATE, - catchup=False, + catchup="disable", tags=["example"], ) as dag: jar_to_local_flink_runner = GCSToLocalFilesystemOperator( 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 715c22021b0c3..6a1cc167744f1 100644 --- a/tests/system/providers/apache/beam/example_beam_java_spark.py +++ b/tests/system/providers/apache/beam/example_beam_java_spark.py @@ -34,7 +34,7 @@ "example_beam_native_java_spark_runner", schedule=None, # Override to match your needs start_date=START_DATE, - catchup=False, + catchup="disable", tags=["example"], ) as dag: jar_to_local_spark_runner = GCSToLocalFilesystemOperator( diff --git a/tests/system/providers/apache/beam/example_go.py b/tests/system/providers/apache/beam/example_go.py index 6c9a734c26c1f..22c4867aafb61 100644 --- a/tests/system/providers/apache/beam/example_go.py +++ b/tests/system/providers/apache/beam/example_go.py @@ -37,7 +37,7 @@ "example_beam_native_go", start_date=START_DATE, schedule="@once", - catchup=False, + catchup="disable", default_args=DEFAULT_ARGS, tags=["example"], ) as dag: diff --git a/tests/system/providers/apache/beam/example_go_dataflow.py b/tests/system/providers/apache/beam/example_go_dataflow.py index 841bcb1c8956c..7e41892360e59 100644 --- a/tests/system/providers/apache/beam/example_go_dataflow.py +++ b/tests/system/providers/apache/beam/example_go_dataflow.py @@ -40,7 +40,7 @@ default_args=DEFAULT_ARGS, start_date=START_DATE, schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_start_go_dataflow_runner_pipeline_async_gcs_file] diff --git a/tests/system/providers/apache/beam/example_java_dataflow.py b/tests/system/providers/apache/beam/example_java_dataflow.py index 1c6405b4c17b4..5b9caecb92640 100644 --- a/tests/system/providers/apache/beam/example_java_dataflow.py +++ b/tests/system/providers/apache/beam/example_java_dataflow.py @@ -36,7 +36,7 @@ "example_beam_native_java_dataflow_runner", schedule=None, # Override to match your needs start_date=START_DATE, - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_start_java_dataflow_runner_pipeline] diff --git a/tests/system/providers/apache/beam/example_python.py b/tests/system/providers/apache/beam/example_python.py index 59e3a09326205..f98b2150776ab 100644 --- a/tests/system/providers/apache/beam/example_python.py +++ b/tests/system/providers/apache/beam/example_python.py @@ -37,7 +37,7 @@ "example_beam_native_python", start_date=START_DATE, schedule=None, # Override to match your needs - catchup=False, + catchup="disable", default_args=DEFAULT_ARGS, tags=["example"], ) as dag: diff --git a/tests/system/providers/apache/beam/example_python_async.py b/tests/system/providers/apache/beam/example_python_async.py index 62e2afe91a293..3329065506f0b 100644 --- a/tests/system/providers/apache/beam/example_python_async.py +++ b/tests/system/providers/apache/beam/example_python_async.py @@ -37,7 +37,7 @@ dag_id="dataflow_native_python_async", start_date=START_DATE, schedule=None, # Override to match your needs - catchup=False, + catchup="disable", default_args=DEFAULT_ARGS, tags=["example"], ) as dag: diff --git a/tests/system/providers/apache/beam/example_python_dataflow.py b/tests/system/providers/apache/beam/example_python_dataflow.py index 931711595b87e..96948e38102a6 100644 --- a/tests/system/providers/apache/beam/example_python_dataflow.py +++ b/tests/system/providers/apache/beam/example_python_dataflow.py @@ -40,7 +40,7 @@ default_args=DEFAULT_ARGS, start_date=START_DATE, schedule=None, # Override to match your needs - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_start_python_dataflow_runner_pipeline_async_gcs_file] diff --git a/tests/system/providers/apache/cassandra/example_cassandra_dag.py b/tests/system/providers/apache/cassandra/example_cassandra_dag.py index 1a305c5014715..e96d533339b3e 100644 --- a/tests/system/providers/apache/cassandra/example_cassandra_dag.py +++ b/tests/system/providers/apache/cassandra/example_cassandra_dag.py @@ -41,7 +41,7 @@ schedule=None, start_date=datetime(2021, 1, 1), default_args={"table": "keyspace_name.table_name"}, - catchup=False, + catchup="disable", tags=["example"], ) as dag: table_sensor = CassandraTableSensor(task_id="cassandra_table_sensor") diff --git a/tests/system/providers/apache/drill/example_drill_dag.py b/tests/system/providers/apache/drill/example_drill_dag.py index e9c602c9e1bdb..c7adf433b37f5 100644 --- a/tests/system/providers/apache/drill/example_drill_dag.py +++ b/tests/system/providers/apache/drill/example_drill_dag.py @@ -33,7 +33,7 @@ dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_drill] diff --git a/tests/system/providers/apache/druid/example_druid_dag.py b/tests/system/providers/apache/druid/example_druid_dag.py index 1c22f87948e0c..9839c85f643cc 100644 --- a/tests/system/providers/apache/druid/example_druid_dag.py +++ b/tests/system/providers/apache/druid/example_druid_dag.py @@ -33,7 +33,7 @@ dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_druid_submit] diff --git a/tests/system/providers/apache/hive/example_twitter_dag.py b/tests/system/providers/apache/hive/example_twitter_dag.py index 8db8319e1cf84..cb76a0258cd89 100644 --- a/tests/system/providers/apache/hive/example_twitter_dag.py +++ b/tests/system/providers/apache/hive/example_twitter_dag.py @@ -84,7 +84,7 @@ def transfer_to_db(): schedule="@daily", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: fetch = fetch_tweets() clean = clean_tweets() diff --git a/tests/system/providers/apache/kafka/example_dag_event_listener.py b/tests/system/providers/apache/kafka/example_dag_event_listener.py index 768673f62070c..ab47165b8306c 100644 --- a/tests/system/providers/apache/kafka/example_dag_event_listener.py +++ b/tests/system/providers/apache/kafka/example_dag_event_listener.py @@ -69,7 +69,7 @@ def _producer_function(): dag_id="fizzbuzz-load-topic", description="Load Data to fizz_buzz topic", start_date=datetime(2022, 11, 1), - catchup=False, + catchup="disable", tags=["fizz-buzz"], ) as dag: t0 = PythonOperator(task_id="load_connections", python_callable=load_connections) @@ -85,7 +85,7 @@ def _producer_function(): dag_id="fizzbuzz-listener-dag", description="listen for messages with mod 3 and mod 5 are zero", start_date=datetime(2022, 11, 1), - catchup=False, + catchup="disable", tags=["fizz", "buzz"], ): diff --git a/tests/system/providers/apache/kafka/example_dag_hello_kafka.py b/tests/system/providers/apache/kafka/example_dag_hello_kafka.py index 57e67ac81af1e..258253394867c 100644 --- a/tests/system/providers/apache/kafka/example_dag_hello_kafka.py +++ b/tests/system/providers/apache/kafka/example_dag_hello_kafka.py @@ -153,7 +153,7 @@ def hello_kafka(): description="Examples of Kafka Operators", schedule=timedelta(days=1), start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: t0 = PythonOperator(task_id="load_connections", python_callable=load_connections) diff --git a/tests/system/providers/apache/kylin/example_kylin_dag.py b/tests/system/providers/apache/kylin/example_kylin_dag.py index ad804900d38b0..efc65d9722fa8 100644 --- a/tests/system/providers/apache/kylin/example_kylin_dag.py +++ b/tests/system/providers/apache/kylin/example_kylin_dag.py @@ -34,7 +34,7 @@ dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", default_args={"project": "learn_kylin", "cube": "kylin_sales_cube"}, tags=["example"], ) as dag: diff --git a/tests/system/providers/apache/livy/example_livy.py b/tests/system/providers/apache/livy/example_livy.py index f8ff27f416edc..31daaa3a6fdac 100644 --- a/tests/system/providers/apache/livy/example_livy.py +++ b/tests/system/providers/apache/livy/example_livy.py @@ -35,7 +35,7 @@ default_args={"args": [10]}, schedule="@daily", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START create_livy] livy_java_task = LivyOperator( diff --git a/tests/system/providers/apache/pig/example_pig.py b/tests/system/providers/apache/pig/example_pig.py index 18842eb0fa079..e663be1724202 100644 --- a/tests/system/providers/apache/pig/example_pig.py +++ b/tests/system/providers/apache/pig/example_pig.py @@ -31,7 +31,7 @@ dag_id="example_pig_operator", schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START create_pig] diff --git a/tests/system/providers/apache/pinot/example_pinot_dag.py b/tests/system/providers/apache/pinot/example_pinot_dag.py index eecd6a4c2abed..16da4a22a1a17 100644 --- a/tests/system/providers/apache/pinot/example_pinot_dag.py +++ b/tests/system/providers/apache/pinot/example_pinot_dag.py @@ -30,7 +30,7 @@ schedule=None, start_date=datetime(2021, 1, 1), # Override to match your needs tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_pinot_admin_hook] @task diff --git a/tests/system/providers/apache/spark/example_spark_dag.py b/tests/system/providers/apache/spark/example_spark_dag.py index 3f9958058eabf..6d5c9af12f17f 100644 --- a/tests/system/providers/apache/spark/example_spark_dag.py +++ b/tests/system/providers/apache/spark/example_spark_dag.py @@ -36,7 +36,7 @@ dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_spark_submit] diff --git a/tests/system/providers/asana/example_asana.py b/tests/system/providers/asana/example_asana.py index 4a528ffcf2f82..057b00a149964 100644 --- a/tests/system/providers/asana/example_asana.py +++ b/tests/system/providers/asana/example_asana.py @@ -50,7 +50,7 @@ start_date=datetime(2021, 1, 1), default_args={"conn_id": CONN_ID}, tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START asana_example_dag] # [START run_asana_create_task_operator] diff --git a/tests/system/providers/cncf/kubernetes/example_kubernetes_decorator.py b/tests/system/providers/cncf/kubernetes/example_kubernetes_decorator.py index 2348f31bdf59a..de6d1c5d9e298 100644 --- a/tests/system/providers/cncf/kubernetes/example_kubernetes_decorator.py +++ b/tests/system/providers/cncf/kubernetes/example_kubernetes_decorator.py @@ -27,7 +27,7 @@ schedule=None, start_date=datetime(2021, 1, 1), tags=["example", "cncf", "kubernetes"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_kubernetes] @task.kubernetes( diff --git a/tests/system/providers/cncf/kubernetes/example_kubernetes_resource.py b/tests/system/providers/cncf/kubernetes/example_kubernetes_resource.py index 7be0a8099106b..7db4bde723b65 100644 --- a/tests/system/providers/cncf/kubernetes/example_kubernetes_resource.py +++ b/tests/system/providers/cncf/kubernetes/example_kubernetes_resource.py @@ -57,7 +57,7 @@ description="create and delete a PVC in a kubernetes", schedule=timedelta(days=1), start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: t1 = KubernetesCreateResourceOperator( task_id="create_pvc", diff --git a/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py b/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py index 03fd0a34e5325..5506ff5164465 100644 --- a/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py +++ b/tests/system/providers/cncf/kubernetes/example_spark_kubernetes.py @@ -52,7 +52,7 @@ description="submit spark-pi as sparkApplication on kubernetes", schedule=timedelta(days=1), start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START SparkKubernetesOperator_DAG] t1 = SparkKubernetesOperator( diff --git a/tests/system/providers/cohere/example_cohere_embedding_operator.py b/tests/system/providers/cohere/example_cohere_embedding_operator.py index ec97ee91e57cb..1f744a6835b39 100644 --- a/tests/system/providers/cohere/example_cohere_embedding_operator.py +++ b/tests/system/providers/cohere/example_cohere_embedding_operator.py @@ -21,7 +21,9 @@ from airflow import DAG from airflow.providers.cohere.operators.embedding import CohereEmbeddingOperator -with DAG("example_cohere_embedding", schedule=None, start_date=datetime(2023, 1, 1), catchup=False) as dag: +with DAG( + "example_cohere_embedding", schedule=None, start_date=datetime(2023, 1, 1), catchup="disable" +) as dag: # [START howto_operator_cohere_embedding] texts = [ "On Kernel-Target Alignment. We describe a family of global optimization procedures", 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 2790d092a2b47..64cab3096a5cf 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 @@ -38,7 +38,7 @@ default_args=connection_args, start_date=datetime(2021, 1, 1), schedule=None, - catchup=False, + catchup="disable", ) as dag: """ ### Example SQL Column and Table Check DAG diff --git a/tests/system/providers/common/sql/example_sql_execute_query.py b/tests/system/providers/common/sql/example_sql_execute_query.py index 694ca3f2ef99c..290f9aab4c2a5 100644 --- a/tests/system/providers/common/sql/example_sql_execute_query.py +++ b/tests/system/providers/common/sql/example_sql_execute_query.py @@ -38,7 +38,7 @@ default_args=connection_args, start_date=datetime(2021, 1, 1), schedule=None, - catchup=False, + catchup="disable", ) as dag: """ ### Example SQL execute query DAG diff --git a/tests/system/providers/core/example_external_task_child_deferrable.py b/tests/system/providers/core/example_external_task_child_deferrable.py index f75eb4f23479f..2aa26c7e0f515 100644 --- a/tests/system/providers/core/example_external_task_child_deferrable.py +++ b/tests/system/providers/core/example_external_task_child_deferrable.py @@ -25,7 +25,7 @@ dag_id="child_dag", start_date=datetime(2022, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "async", "core"], ) as dag: dummy_task = BashOperator( diff --git a/tests/system/providers/core/example_external_task_parent_deferrable.py b/tests/system/providers/core/example_external_task_parent_deferrable.py index 7cec2ce13815a..76d5d67d7fe71 100644 --- a/tests/system/providers/core/example_external_task_parent_deferrable.py +++ b/tests/system/providers/core/example_external_task_parent_deferrable.py @@ -26,7 +26,7 @@ dag_id="example_external_task", start_date=datetime(2022, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "async", "core"], ) as dag: start = DummyOperator(task_id="start") diff --git a/tests/system/providers/dbt/cloud/example_dbt_cloud.py b/tests/system/providers/dbt/cloud/example_dbt_cloud.py index 21f2f5dc3fdf8..6d94634ef7a86 100644 --- a/tests/system/providers/dbt/cloud/example_dbt_cloud.py +++ b/tests/system/providers/dbt/cloud/example_dbt_cloud.py @@ -42,7 +42,7 @@ default_args={"dbt_cloud_conn_id": "dbt", "account_id": 39151}, start_date=datetime(2021, 1, 1), schedule=None, - catchup=False, + catchup="disable", ) as dag: begin = EmptyOperator(task_id="begin") end = EmptyOperator(task_id="end") diff --git a/tests/system/providers/docker/example_docker.py b/tests/system/providers/docker/example_docker.py index 069f4794de632..8b5cbdcef7f97 100644 --- a/tests/system/providers/docker/example_docker.py +++ b/tests/system/providers/docker/example_docker.py @@ -31,7 +31,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "docker"], ) as dag: t1 = BashOperator(task_id="print_date", bash_command="date", dag=dag) diff --git a/tests/system/providers/docker/example_docker_copy_data.py b/tests/system/providers/docker/example_docker_copy_data.py index 1154790bdcbc9..81f6bf1e50168 100644 --- a/tests/system/providers/docker/example_docker_copy_data.py +++ b/tests/system/providers/docker/example_docker_copy_data.py @@ -42,7 +42,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "docker"], ) as dag: locate_file_cmd = """ diff --git a/tests/system/providers/docker/example_docker_swarm.py b/tests/system/providers/docker/example_docker_swarm.py index 88a200b0fd958..f9721625a40a1 100644 --- a/tests/system/providers/docker/example_docker_swarm.py +++ b/tests/system/providers/docker/example_docker_swarm.py @@ -30,7 +30,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "docker"], ) as dag: t1 = DockerSwarmOperator( diff --git a/tests/system/providers/docker/example_taskflow_api_docker_virtualenv.py b/tests/system/providers/docker/example_taskflow_api_docker_virtualenv.py index 95c502a221073..d68992ea264da 100644 --- a/tests/system/providers/docker/example_taskflow_api_docker_virtualenv.py +++ b/tests/system/providers/docker/example_taskflow_api_docker_virtualenv.py @@ -105,7 +105,7 @@ def load(total_order_value: float): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "docker"], ) as dag: # The try/except here is because Airflow versions less than 2.2.0 doesn't support diff --git a/tests/system/providers/ftp/example_ftp.py b/tests/system/providers/ftp/example_ftp.py index cfecccf61ca20..8459bdae435a7 100644 --- a/tests/system/providers/ftp/example_ftp.py +++ b/tests/system/providers/ftp/example_ftp.py @@ -36,7 +36,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "Ftp", "FtpFileTransmit", "Ftps", "FtpsFileTransmit"], ) as dag: # [START howto_operator_ftp_put] diff --git a/tests/system/providers/github/example_github.py b/tests/system/providers/github/example_github.py index 4667dddf9094c..a001b459d2ba9 100644 --- a/tests/system/providers/github/example_github.py +++ b/tests/system/providers/github/example_github.py @@ -36,7 +36,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_tag_sensor_github] diff --git a/tests/system/providers/google/ads/example_ads.py b/tests/system/providers/google/ads/example_ads.py index 5dd336e7df449..4af9c6c99f506 100644 --- a/tests/system/providers/google/ads/example_ads.py +++ b/tests/system/providers/google/ads/example_ads.py @@ -76,7 +76,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "ads"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/automl/example_automl_dataset.py b/tests/system/providers/google/cloud/automl/example_automl_dataset.py index 1b0c687a2d56c..3dfddf9e61e06 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_dataset.py +++ b/tests/system/providers/google/cloud/automl/example_automl_dataset.py @@ -76,7 +76,7 @@ def get_target_column_spec(columns_specs: list[dict], column_name: str) -> str: dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "automl", "dataset"], user_defined_macros={ "get_target_column_spec": get_target_column_spec, diff --git a/tests/system/providers/google/cloud/automl/example_automl_model.py b/tests/system/providers/google/cloud/automl/example_automl_model.py index fbcbcb1898cb2..59c7ba3ec00af 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_model.py +++ b/tests/system/providers/google/cloud/automl/example_automl_model.py @@ -114,7 +114,7 @@ def get_target_column_spec(columns_specs: list[dict], column_name: str) -> str: dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={ "get_target_column_spec": get_target_column_spec, "target": "Deposit", diff --git a/tests/system/providers/google/cloud/automl/example_automl_nl_text_classification.py b/tests/system/providers/google/cloud/automl/example_automl_nl_text_classification.py index e43463063cf8e..48ade675bf9c5 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_nl_text_classification.py +++ b/tests/system/providers/google/cloud/automl/example_automl_nl_text_classification.py @@ -79,7 +79,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "automl", "text-classification"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 3575eb6b38a7f..dec1b7b125de3 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 @@ -78,7 +78,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "text-extraction"], ) as dag: diff --git a/tests/system/providers/google/cloud/automl/example_automl_nl_text_sentiment.py b/tests/system/providers/google/cloud/automl/example_automl_nl_text_sentiment.py index f4e16c39088f9..fca7a1867e468 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_nl_text_sentiment.py +++ b/tests/system/providers/google/cloud/automl/example_automl_nl_text_sentiment.py @@ -79,7 +79,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "text-sentiment"], ) as dag: diff --git a/tests/system/providers/google/cloud/automl/example_automl_translation.py b/tests/system/providers/google/cloud/automl/example_automl_translation.py index d3e55a64bd7e4..582869bfe507b 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_translation.py +++ b/tests/system/providers/google/cloud/automl/example_automl_translation.py @@ -79,7 +79,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "translate"], ) as dag: diff --git a/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_classification.py b/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_classification.py index b88dcebe108fd..94d26ee4f53cd 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_classification.py +++ b/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_classification.py @@ -72,7 +72,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "video-clss"], ) as dag: diff --git a/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_tracking.py b/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_tracking.py index be0d07c7c45f9..f93302356937d 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_tracking.py +++ b/tests/system/providers/google/cloud/automl/example_automl_video_intelligence_tracking.py @@ -72,7 +72,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "video-tracking"], ) as dag: 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 39a93f254bac7..10614da1898b8 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 @@ -77,7 +77,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "vision-clss"], ) as dag: diff --git a/tests/system/providers/google/cloud/automl/example_automl_vision_object_detection.py b/tests/system/providers/google/cloud/automl/example_automl_vision_object_detection.py index e9a49c4b2403b..9ed8ea709636c 100644 --- a/tests/system/providers/google/cloud/automl/example_automl_vision_object_detection.py +++ b/tests/system/providers/google/cloud/automl/example_automl_vision_object_detection.py @@ -71,7 +71,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", user_defined_macros={"extract_object_id": extract_object_id}, tags=["example", "automl", "object-detection"], ) as dag: 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 c46287db08897..41efc47058cfc 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 @@ -45,7 +45,7 @@ }, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "azure"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 2f3357cafaf64..0abc59024bd70 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_dataset.py @@ -43,7 +43,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: # [START howto_operator_bigquery_create_dataset] diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_dts.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_dts.py index 5f81429abbc93..37ade32908e2c 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_dts.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_dts.py @@ -85,7 +85,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 b4e6306170bb0..c7f9d9455b1a3 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_operations.py @@ -47,7 +47,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME) 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 7c7b72643531f..0edfd3c7be4cf 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 @@ -42,7 +42,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_dataset_with_location = BigQueryCreateEmptyDatasetOperator( 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 7898088046651..d7f6902dd29a9 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_queries.py @@ -72,7 +72,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], user_defined_macros={"DATASET": DATASET, "TABLE": TABLE_1, "QUERY_SQL_PATH": QUERY_SQL_PATH}, ) as dag: diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_queries_async.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_queries_async.py index 4ec675de38b98..88451906d128f 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_queries_async.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_queries_async.py @@ -101,7 +101,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", default_args=default_args, tags=["example", "bigquery", "deferrable"], user_defined_macros={"DATASET": DATASET_NAME, "TABLE": TABLE_NAME_1}, 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 c04248c28d38a..b78e75968b288 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_sensors.py @@ -60,7 +60,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery", "sensors"], user_defined_macros={"DATASET": DATASET_NAME, "TABLE": TABLE_NAME}, default_args={"project_id": PROJECT_ID}, 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 83815c3fd2da5..032895781dd34 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_tables.py @@ -56,7 +56,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 716c89cd51948..d8359bfd1236e 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 @@ -45,7 +45,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_dataset = BigQueryCreateEmptyDatasetOperator( 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 bc86701a4cf22..38af1a65285e1 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 @@ -47,7 +47,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs_async.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs_async.py index e8a9f942d4df4..70de3ee9f25a8 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs_async.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_gcs_async.py @@ -47,7 +47,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 c02c7a36cbc19..91593b4c46eb1 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 @@ -50,7 +50,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: # [START howto_operator_bigquery_to_mssql] diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mysql.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mysql.py index c5e7fd2e218ae..5208a975ac8fe 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mysql.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mysql.py @@ -49,7 +49,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: # [START howto_operator_bigquery_to_mysql] diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_postgres.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_postgres.py index c0d365091a178..a2b770a891d18 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_postgres.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_postgres.py @@ -49,7 +49,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: # [START howto_operator_bigquery_to_postgres] 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 acbc312189375..245cfc50461c8 100644 --- a/tests/system/providers/google/cloud/bigquery/example_bigquery_transfer.py +++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_transfer.py @@ -48,7 +48,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "bigquery"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/bigtable/example_bigtable.py b/tests/system/providers/google/cloud/bigtable/example_bigtable.py index af06226637997..1c637c30e2c00 100644 --- a/tests/system/providers/google/cloud/bigtable/example_bigtable.py +++ b/tests/system/providers/google/cloud/bigtable/example_bigtable.py @@ -86,7 +86,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["bigtable", "example"], ) as dag: # [START howto_operator_gcp_bigtable_instance_create] diff --git a/tests/system/providers/google/cloud/cloud_run/example_cloud_run.py b/tests/system/providers/google/cloud/cloud_run/example_cloud_run.py index 4230e69cf7e35..9191e539e442e 100644 --- a/tests/system/providers/google/cloud/cloud_run/example_cloud_run.py +++ b/tests/system/providers/google/cloud/cloud_run/example_cloud_run.py @@ -137,7 +137,7 @@ def _create_job_with_label(): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_cloud_run_create_job] diff --git a/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_mysql.py b/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_mysql.py index f66242dd2a326..8e82dff8b3b91 100644 --- a/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_mysql.py +++ b/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_mysql.py @@ -165,7 +165,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "cloudsql", "mysql"], ) as dag: create_cloud_sql_instance = CloudSQLCreateInstanceOperator( diff --git a/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_postgres.py b/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_postgres.py index fae820ed590ab..78a1d83a29bd3 100644 --- a/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_postgres.py +++ b/tests/system/providers/google/cloud/cloud_sql/example_cloud_sql_query_postgres.py @@ -167,7 +167,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "cloudsql", "postgres"], ) as dag: create_cloud_sql_instance = CloudSQLCreateInstanceOperator( diff --git a/tests/system/providers/google/cloud/composer/example_cloud_composer.py b/tests/system/providers/google/cloud/composer/example_cloud_composer.py index ab9c6a0e1b8b8..57d7e1822e4e9 100644 --- a/tests/system/providers/google/cloud/composer/example_cloud_composer.py +++ b/tests/system/providers/google/cloud/composer/example_cloud_composer.py @@ -64,7 +64,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "composer"], ) as dag: # [START howto_operator_composer_image_list] diff --git a/tests/system/providers/google/cloud/composer/example_cloud_composer_deferrable.py b/tests/system/providers/google/cloud/composer/example_cloud_composer_deferrable.py index 1a69e12d7b35d..518fcb919f696 100644 --- a/tests/system/providers/google/cloud/composer/example_cloud_composer_deferrable.py +++ b/tests/system/providers/google/cloud/composer/example_cloud_composer_deferrable.py @@ -60,7 +60,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "composer"], ) as dag: # [START howto_operator_create_composer_environment_deferrable_mode] diff --git a/tests/system/providers/google/cloud/compute/example_compute.py b/tests/system/providers/google/cloud/compute/example_compute.py index eebc6f71e03b1..7f0dd489a6bd2 100644 --- a/tests/system/providers/google/cloud/compute/example_compute.py +++ b/tests/system/providers/google/cloud/compute/example_compute.py @@ -102,7 +102,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_gce_insert] diff --git a/tests/system/providers/google/cloud/compute/example_compute_igm.py b/tests/system/providers/google/cloud/compute/example_compute_igm.py index 68f4bd37f434c..294b4984d3659 100644 --- a/tests/system/providers/google/cloud/compute/example_compute_igm.py +++ b/tests/system/providers/google/cloud/compute/example_compute_igm.py @@ -112,7 +112,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_gce_igm_insert_template] diff --git a/tests/system/providers/google/cloud/compute/example_compute_ssh.py b/tests/system/providers/google/cloud/compute/example_compute_ssh.py index 26f6e89ff17ba..fe6edf039a3e8 100644 --- a/tests/system/providers/google/cloud/compute/example_compute_ssh.py +++ b/tests/system/providers/google/cloud/compute/example_compute_ssh.py @@ -73,7 +73,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "compute-ssh"], ) as dag: # [START howto_operator_gce_insert] diff --git a/tests/system/providers/google/cloud/compute/example_compute_ssh_os_login.py b/tests/system/providers/google/cloud/compute/example_compute_ssh_os_login.py index 88cdc8bcca082..f41213015d23e 100644 --- a/tests/system/providers/google/cloud/compute/example_compute_ssh_os_login.py +++ b/tests/system/providers/google/cloud/compute/example_compute_ssh_os_login.py @@ -81,7 +81,7 @@ DAG_ID, schedule_interval="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "compute-ssh", "os-login"], ) as dag: # [START howto_operator_gce_insert] diff --git a/tests/system/providers/google/cloud/compute/example_compute_ssh_parallel.py b/tests/system/providers/google/cloud/compute/example_compute_ssh_parallel.py index 60efada741ba3..c3ff60485a6a6 100644 --- a/tests/system/providers/google/cloud/compute/example_compute_ssh_parallel.py +++ b/tests/system/providers/google/cloud/compute/example_compute_ssh_parallel.py @@ -73,7 +73,7 @@ DAG_ID, schedule_interval="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "compute-ssh-parallel"], ) as dag: # [START howto_operator_gce_insert] diff --git a/tests/system/providers/google/cloud/dataflow/example_dataflow_go.py b/tests/system/providers/google/cloud/dataflow/example_dataflow_go.py index 71985a2f835f7..3e6739daba675 100644 --- a/tests/system/providers/google/cloud/dataflow/example_dataflow_go.py +++ b/tests/system/providers/google/cloud/dataflow/example_dataflow_go.py @@ -65,7 +65,7 @@ "example_beam_native_go", start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", default_args=default_args, tags=["example"], ) as dag: diff --git a/tests/system/providers/google/cloud/dataflow/example_dataflow_native_java.py b/tests/system/providers/google/cloud/dataflow/example_dataflow_native_java.py index 53b33b89e9571..3a1f87704bdbe 100644 --- a/tests/system/providers/google/cloud/dataflow/example_dataflow_native_java.py +++ b/tests/system/providers/google/cloud/dataflow/example_dataflow_native_java.py @@ -60,7 +60,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataflow"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python.py b/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python.py index 957fe9afbe85a..c95001d069515 100644 --- a/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python.py +++ b/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python.py @@ -58,7 +58,7 @@ default_args=default_args, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataflow"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python_async.py b/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python_async.py index 05695eab31c31..9e709a9f0e341 100644 --- a/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python_async.py +++ b/tests/system/providers/google/cloud/dataflow/example_dataflow_native_python_async.py @@ -66,7 +66,7 @@ default_args=default_args, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataflow"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/dataflow/example_dataflow_sql.py b/tests/system/providers/google/cloud/dataflow/example_dataflow_sql.py index 86afc9fa38b7d..72ed01d6fba10 100644 --- a/tests/system/providers/google/cloud/dataflow/example_dataflow_sql.py +++ b/tests/system/providers/google/cloud/dataflow/example_dataflow_sql.py @@ -55,7 +55,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataflow-sql"], ) as dag: create_bq_dataset = BigQueryCreateEmptyDatasetOperator( diff --git a/tests/system/providers/google/cloud/dataflow/example_dataflow_template.py b/tests/system/providers/google/cloud/dataflow/example_dataflow_template.py index 9182834fbb5c4..14af32e362930 100644 --- a/tests/system/providers/google/cloud/dataflow/example_dataflow_template.py +++ b/tests/system/providers/google/cloud/dataflow/example_dataflow_template.py @@ -77,7 +77,7 @@ default_args=default_args, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataflow"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/dataform/example_dataform.py b/tests/system/providers/google/cloud/dataform/example_dataform.py index 4e5053868b9f3..2ef203c297d41 100644 --- a/tests/system/providers/google/cloud/dataform/example_dataform.py +++ b/tests/system/providers/google/cloud/dataform/example_dataform.py @@ -62,7 +62,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataform"], ) as dag: # [START howto_operator_create_repository] diff --git a/tests/system/providers/google/cloud/dataprep/example_dataprep.py b/tests/system/providers/google/cloud/dataprep/example_dataprep.py index 1ac9d98082706..d42d15f39144e 100644 --- a/tests/system/providers/google/cloud/dataprep/example_dataprep.py +++ b/tests/system/providers/google/cloud/dataprep/example_dataprep.py @@ -82,7 +82,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), # Override to match your needs - catchup=False, + catchup="disable", tags=["example", "dataprep"], render_template_as_native_obj=True, ) as dag: 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 7dd5eff73aa73..fa4a6be0b53d8 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch.py @@ -57,7 +57,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc"], ) as dag: # [START how_to_cloud_dataproc_create_batch_operator] diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_deferrable.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_deferrable.py index 4306219ee8fd9..51639d553ad7d 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_deferrable.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_batch_deferrable.py @@ -49,7 +49,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "batch", "deferrable"], ) as dag: # [START how_to_cloud_dataproc_create_batch_operator_async] 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 5c639e5ede0f3..804ec6561dfb7 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 @@ -70,7 +70,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "batch", "persistent"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_deferrable.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_deferrable.py index a0c427b568757..c77413adc3c17 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_deferrable.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_deferrable.py @@ -80,7 +80,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "deferrable"], ) as dag: # [START how_to_cloud_dataproc_create_cluster_operator_async] 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 8f4ddb6a10560..9a9d60e57209b 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 @@ -72,7 +72,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_update.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_update.py index 643ac7bf82317..efbf5db73c3a2 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_update.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_cluster_update.py @@ -70,7 +70,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 564b33180a7fd..6eba6a62c43b3 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_gke.py @@ -87,7 +87,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "gke"], ) as dag: create_gke_cluster = GKECreateClusterOperator( 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 734dd5830b8da..593ea41e042fa 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_hadoop.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_hadoop.py @@ -74,7 +74,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "hadoop"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 af3c54636b141..8147bf7b0fd21 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_hive.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_hive.py @@ -79,7 +79,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "hive"], ) as dag: # [START how_to_cloud_dataproc_create_cluster_operator] 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 a7d27efdc8174..dbe853d02d478 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_pig.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_pig.py @@ -67,7 +67,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "pig"], ) as dag: create_cluster = DataprocCreateClusterOperator( diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_presto.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_presto.py index bd4503183053c..4459405e13e13 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_presto.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_presto.py @@ -74,7 +74,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "presto"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 93ee3e54ac93a..e1721374ecd44 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_pyspark.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_pyspark.py @@ -67,7 +67,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "pyspark"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 3acda02d3cb09..bf26a9fb8012c 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark.py @@ -69,7 +69,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "spark"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 80ef2388f3a33..caec15243c111 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 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "spark", "async"], ) as dag: create_cluster = DataprocCreateClusterOperator( diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_deferrable.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_deferrable.py index 336def4696b34..8831a395cff55 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_deferrable.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_spark_deferrable.py @@ -70,7 +70,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "spark", "deferrable"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 f50cc81b92dcb..104d98b80c18c 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 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 15c5aee0d3838..fb2d6238cdaec 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_sparkr.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_sparkr.py @@ -67,7 +67,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "sparkr"], ) as dag: create_cluster = DataprocCreateClusterOperator( diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_trino.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_trino.py index 04032f351330b..492dafcd0f52f 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_trino.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_trino.py @@ -76,7 +76,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "trino"], ) as dag: create_cluster = DataprocCreateClusterOperator( 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 0736d16bdab82..d51353f493b8f 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow.py @@ -65,7 +65,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "workflow"], ) as dag: # [START how_to_cloud_dataproc_create_workflow_template] diff --git a/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow_deferrable.py b/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow_deferrable.py index 6500af4366b70..9b29023134e78 100644 --- a/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow_deferrable.py +++ b/tests/system/providers/google/cloud/dataproc/example_dataproc_workflow_deferrable.py @@ -65,7 +65,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "dataproc", "workflow", "deferrable"], ) as dag: create_workflow_template = DataprocCreateWorkflowTemplateOperator( 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 6dba0709b1409..5cf2eb32503aa 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_commit.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_commit.py @@ -61,7 +61,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["datastore", "example"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 cc461da927e45..b4930db33c2b5 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_query.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_query.py @@ -50,7 +50,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["datastore", "example"], ) as dag: allocate_ids = CloudDatastoreAllocateIdsOperator( 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 57d33879c7869..75cdd5ef99c61 100644 --- a/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py +++ b/tests/system/providers/google/cloud/datastore/example_datastore_rollback.py @@ -42,7 +42,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["datastore", "example"], ) as dag: begin_transaction_to_rollback = CloudDatastoreBeginTransactionOperator( 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 dfd904a1b9c05..02e0f8ae1a609 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 @@ -45,7 +45,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "calendar"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/gcs/example_firestore.py b/tests/system/providers/google/cloud/gcs/example_firestore.py index b3eaac6c2392f..f5240cd6fcb7f 100644 --- a/tests/system/providers/google/cloud/gcs/example_firestore.py +++ b/tests/system/providers/google/cloud/gcs/example_firestore.py @@ -67,7 +67,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "firestore"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 01ab83e51ae35..d28c56fda054b 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_acl.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_acl.py @@ -52,7 +52,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "acl", "example"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 f89464b829ca5..093eacfa70b46 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 @@ -53,7 +53,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "example"], ) as dag: create_bucket_src = GCSCreateBucketOperator( 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 607c670cc2908..75e73216ef9d4 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_sensor.py @@ -71,7 +71,7 @@ def mode_setter(self, value): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "example"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 732034f699766..43762798d0f21 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 @@ -42,7 +42,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs"], ) as dag: create_test_dataset = BigQueryCreateEmptyDatasetOperator( diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery_async.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery_async.py index 92308f4a3987f..ac726d2c20a0b 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery_async.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_to_bigquery_async.py @@ -50,7 +50,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs"], ) as dag: create_test_dataset_for_string_fields = BigQueryCreateEmptyDatasetOperator( 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 46077894031c6..2250e7868e2fc 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 @@ -54,7 +54,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "example"], ) as dag: 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 8aef24ea91206..e0974d95716c0 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 @@ -63,7 +63,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs", "gdrive"], ) as dag: 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 e728eb668f8df..f67a1dc48e4e1 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 @@ -48,7 +48,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", # Override to match your needs - catchup=False, + catchup="disable", tags=["example", "gcs"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 355fa1e99b293..16c290fadba40 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_transform.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_transform.py @@ -50,7 +50,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "example"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 75812e63bb9aa..7ea9a920efd8a 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 @@ -56,7 +56,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "example"], ) as dag: create_bucket_src = GCSCreateBucketOperator( 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 437cb4f87f619..f40c13a8092c1 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 @@ -45,7 +45,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["gcs", "example"], ) as dag: # [START howto_operator_gcs_create_bucket] diff --git a/tests/system/providers/google/cloud/gcs/example_gdrive_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_gdrive_to_gcs.py index 65cbcf18a2bd4..cfd8659b59910 100644 --- a/tests/system/providers/google/cloud/gcs/example_gdrive_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_gdrive_to_gcs.py @@ -57,7 +57,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs", "gdrive"], ) as dag: 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 f7d7dd4a82deb..fbd9fbf7c945f 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 @@ -45,7 +45,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "mssql"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 94a77199cdb22..557f1e3e4109d 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 @@ -149,7 +149,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "mysql", "gcs"], ) as dag: create_instance = ComputeEngineInsertInstanceOperator( 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 571478c8333c0..dac1b2a66bb61 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 @@ -36,7 +36,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "oracle"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 437f09a7e6613..20f8ffb7e614b 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 @@ -50,7 +50,7 @@ def upload_file(): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "s3"], ) as dag: create_s3_bucket = S3CreateBucketOperator( diff --git a/tests/system/providers/google/cloud/gcs/example_s3_to_gcs_async.py b/tests/system/providers/google/cloud/gcs/example_s3_to_gcs_async.py index 4c3832a96ad85..7292f0a0ba7d5 100644 --- a/tests/system/providers/google/cloud/gcs/example_s3_to_gcs_async.py +++ b/tests/system/providers/google/cloud/gcs/example_s3_to_gcs_async.py @@ -48,7 +48,7 @@ def upload_file(): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "s3"], ) as dag: create_s3_bucket = S3CreateBucketOperator( diff --git a/tests/system/providers/google/cloud/gcs/example_sftp_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_sftp_to_gcs.py index 5b13ec79edd75..b2164e00613a9 100644 --- a/tests/system/providers/google/cloud/gcs/example_sftp_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_sftp_to_gcs.py @@ -55,7 +55,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/gcs/example_sheets.py b/tests/system/providers/google/cloud/gcs/example_sheets.py index 37b3441263ce6..7697869f194d4 100644 --- a/tests/system/providers/google/cloud/gcs/example_sheets.py +++ b/tests/system/providers/google/cloud/gcs/example_sheets.py @@ -49,7 +49,7 @@ DAG_ID, schedule="@once", # Override to match your needs, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "sheets"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 bb6264d2b5c1e..e30816f456798 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 @@ -47,7 +47,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "sheets"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 d2b661a643971..4916c995a82dc 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 @@ -57,7 +57,7 @@ def safe_name(s: str) -> str: dag_id=DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs"], ) as dag: create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create-dataset", dataset_id=DATASET_NAME) diff --git a/tests/system/providers/google/cloud/ml_engine/example_mlengine.py b/tests/system/providers/google/cloud/ml_engine/example_mlengine.py index 4ae63696f302b..ef084c5e98c7e 100644 --- a/tests/system/providers/google/cloud/ml_engine/example_mlengine.py +++ b/tests/system/providers/google/cloud/ml_engine/example_mlengine.py @@ -95,7 +95,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "ml_engine"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/pubsub/example_pubsub.py b/tests/system/providers/google/cloud/pubsub/example_pubsub.py index 3cb94f6d38489..501c58ddae135 100644 --- a/tests/system/providers/google/cloud/pubsub/example_pubsub.py +++ b/tests/system/providers/google/cloud/pubsub/example_pubsub.py @@ -57,7 +57,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_gcp_pubsub_create_topic] create_topic = PubSubCreateTopicOperator( diff --git a/tests/system/providers/google/cloud/pubsub/example_pubsub_deferrable.py b/tests/system/providers/google/cloud/pubsub/example_pubsub_deferrable.py index fb9dace2dba0f..6565be8f7eb99 100644 --- a/tests/system/providers/google/cloud/pubsub/example_pubsub_deferrable.py +++ b/tests/system/providers/google/cloud/pubsub/example_pubsub_deferrable.py @@ -47,7 +47,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_topic = PubSubCreateTopicOperator( task_id="create_topic", topic=TOPIC_ID, project_id=PROJECT_ID, fail_if_exists=False diff --git a/tests/system/providers/google/cloud/spanner/example_spanner.py b/tests/system/providers/google/cloud/spanner/example_spanner.py index cb945b7d05b8f..918897260abfd 100644 --- a/tests/system/providers/google/cloud/spanner/example_spanner.py +++ b/tests/system/providers/google/cloud/spanner/example_spanner.py @@ -52,7 +52,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "spanner"], ) as dag: # Create diff --git a/tests/system/providers/google/cloud/tasks/example_queue.py b/tests/system/providers/google/cloud/tasks/example_queue.py index 6030fc182ed04..dd2c977c0089e 100644 --- a/tests/system/providers/google/cloud/tasks/example_queue.py +++ b/tests/system/providers/google/cloud/tasks/example_queue.py @@ -58,7 +58,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "tasks"], ) as dag: diff --git a/tests/system/providers/google/cloud/tasks/example_tasks.py b/tests/system/providers/google/cloud/tasks/example_tasks.py index 24d3aff7f5cb2..ed0c7136aa9d4 100644 --- a/tests/system/providers/google/cloud/tasks/example_tasks.py +++ b/tests/system/providers/google/cloud/tasks/example_tasks.py @@ -65,7 +65,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "tasks"], ) as dag: 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 a1a58e0f3aa20..3aa5369f8168f 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 @@ -51,7 +51,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/transfers/example_gdrive_to_local.py b/tests/system/providers/google/cloud/transfers/example_gdrive_to_local.py index 12b491da5227d..25f4031f9528b 100644 --- a/tests/system/providers/google/cloud/transfers/example_gdrive_to_local.py +++ b/tests/system/providers/google/cloud/transfers/example_gdrive_to_local.py @@ -60,7 +60,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "gdrive"], ) as dag: diff --git a/tests/system/providers/google/cloud/translate/example_translate.py b/tests/system/providers/google/cloud/translate/example_translate.py index 5a5ad1e943317..6bbcea4269b6d 100644 --- a/tests/system/providers/google/cloud/translate/example_translate.py +++ b/tests/system/providers/google/cloud/translate/example_translate.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_translate_text] diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_tabular_training.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_tabular_training.py index 2a967cba295b5..6dcab2b422c56 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_tabular_training.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_tabular_training.py @@ -87,7 +87,7 @@ f"{DAG_ID}_tabular_training_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "auto_ml"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job_python_package.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job_python_package.py index e11337711348e..59811ece013d9 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job_python_package.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job_python_package.py @@ -87,7 +87,7 @@ def TABULAR_DATASET(bucket_name): f"{DAG_ID}_python_package", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "custom_job"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py index 52baa494b287b..d12b616f92d5b 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py @@ -78,7 +78,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "hyperparameter_tuning_job"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_list_custom_jobs.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_list_custom_jobs.py index 7fab55ca0d5e9..2c2c4d555c2ab 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_list_custom_jobs.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_list_custom_jobs.py @@ -37,7 +37,7 @@ f"{DAG_ID}_list_custom_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "custom_job"], ) as dag: # [START how_to_cloud_vertex_ai_list_custom_training_job_operator] diff --git a/tests/system/providers/google/cloud/vision/example_vision_annotate_image.py b/tests/system/providers/google/cloud/vision/example_vision_annotate_image.py index a72c296d904a3..588f94e4bcf3f 100644 --- a/tests/system/providers/google/cloud/vision/example_vision_annotate_image.py +++ b/tests/system/providers/google/cloud/vision/example_vision_annotate_image.py @@ -78,7 +78,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vision"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vision/example_vision_autogenerated.py b/tests/system/providers/google/cloud/vision/example_vision_autogenerated.py index fabc8f7e4322e..988c7ec971be4 100644 --- a/tests/system/providers/google/cloud/vision/example_vision_autogenerated.py +++ b/tests/system/providers/google/cloud/vision/example_vision_autogenerated.py @@ -110,7 +110,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vision"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vision/example_vision_explicit.py b/tests/system/providers/google/cloud/vision/example_vision_explicit.py index e7eb23a9a3d38..a6335f6a27780 100644 --- a/tests/system/providers/google/cloud/vision/example_vision_explicit.py +++ b/tests/system/providers/google/cloud/vision/example_vision_explicit.py @@ -96,7 +96,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vision"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/http/example_http.py b/tests/system/providers/http/example_http.py index 3cd52edf8474e..46b07ebae31c1 100644 --- a/tests/system/providers/http/example_http.py +++ b/tests/system/providers/http/example_http.py @@ -35,7 +35,7 @@ default_args={"retries": 1}, tags=["example"], start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) dag.doc_md = __doc__ diff --git a/tests/system/providers/jdbc/example_jdbc_queries.py b/tests/system/providers/jdbc/example_jdbc_queries.py index 39bb868d73b69..7b077dc5009c2 100644 --- a/tests/system/providers/jdbc/example_jdbc_queries.py +++ b/tests/system/providers/jdbc/example_jdbc_queries.py @@ -39,7 +39,7 @@ start_date=datetime(2021, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=["example"], - catchup=False, + catchup="disable", ) as dag: run_this_last = EmptyOperator(task_id="run_this_last") diff --git a/tests/system/providers/mysql/example_mysql.py b/tests/system/providers/mysql/example_mysql.py index 95845d922f414..1df20086e8099 100644 --- a/tests/system/providers/mysql/example_mysql.py +++ b/tests/system/providers/mysql/example_mysql.py @@ -34,7 +34,7 @@ start_date=datetime(2021, 1, 1), default_args={"mysql_conn_id": "mysql_conn_id"}, tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_mysql] diff --git a/tests/system/providers/neo4j/example_neo4j.py b/tests/system/providers/neo4j/example_neo4j.py index b84c2d690d3f6..a208694345225 100644 --- a/tests/system/providers/neo4j/example_neo4j.py +++ b/tests/system/providers/neo4j/example_neo4j.py @@ -33,7 +33,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START run_query_neo4j_operator] diff --git a/tests/system/providers/sftp/example_sftp_sensor.py b/tests/system/providers/sftp/example_sftp_sensor.py index 934556637b91b..4b7d95af4d3bf 100644 --- a/tests/system/providers/sftp/example_sftp_sensor.py +++ b/tests/system/providers/sftp/example_sftp_sensor.py @@ -42,7 +42,7 @@ def sleep_function(): "example_sftp_sensor", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "sftp"], ) as dag: # [START howto_operator_sftp_sensor_decorator] diff --git a/tests/system/providers/slack/example_sql_to_slack.py b/tests/system/providers/slack/example_sql_to_slack.py index 288bac418b9a1..2f489d927912d 100644 --- a/tests/system/providers/slack/example_sql_to_slack.py +++ b/tests/system/providers/slack/example_sql_to_slack.py @@ -34,7 +34,7 @@ dag_id=DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_sql_to_slack_api_file] diff --git a/tests/system/providers/slack/example_sql_to_slack_webhook.py b/tests/system/providers/slack/example_sql_to_slack_webhook.py index fc2e6ee2a9ebb..418859676ec42 100644 --- a/tests/system/providers/slack/example_sql_to_slack_webhook.py +++ b/tests/system/providers/slack/example_sql_to_slack_webhook.py @@ -35,7 +35,7 @@ dag_id=DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_sql_to_slack_webhook] diff --git a/tests/system/providers/trino/example_gcs_to_trino.py b/tests/system/providers/trino/example_gcs_to_trino.py index d4e475f7000f2..4adbeb82b5156 100644 --- a/tests/system/providers/trino/example_gcs_to_trino.py +++ b/tests/system/providers/trino/example_gcs_to_trino.py @@ -36,7 +36,7 @@ dag_id=DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START gcs_csv_to_trino_table] diff --git a/tests/system/providers/trino/example_trino.py b/tests/system/providers/trino/example_trino.py index d3c928af764e9..0bd7aafdbb708 100644 --- a/tests/system/providers/trino/example_trino.py +++ b/tests/system/providers/trino/example_trino.py @@ -36,7 +36,7 @@ dag_id="example_trino", schedule="@once", # Override to match your needs start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: trino_create_schema = TrinoOperator( diff --git a/tests/system/providers/weaviate/example_weaviate_operator.py b/tests/system/providers/weaviate/example_weaviate_operator.py index de16546d69efb..19208e34d8bb5 100644 --- a/tests/system/providers/weaviate/example_weaviate_operator.py +++ b/tests/system/providers/weaviate/example_weaviate_operator.py @@ -85,7 +85,7 @@ def get_data_without_vectors(*args, **kwargs): @dag( schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example", "weaviate"], ) def example_weaviate_using_operator(): diff --git a/tests/timetables/test_continuous_timetable.py b/tests/timetables/test_continuous_timetable.py index 9fc5130f70763..dc78d1c338769 100644 --- a/tests/timetables/test_continuous_timetable.py +++ b/tests/timetables/test_continuous_timetable.py @@ -23,6 +23,7 @@ from airflow.timetables.base import DataInterval, TimeRestriction from airflow.timetables.simple import ContinuousTimetable +from airflow.utils.catchup import Catchup BEFORE_DATE = pendulum.datetime(2023, 3, 1, tz="UTC") START_DATE = pendulum.datetime(2023, 3, 3, tz="UTC") @@ -33,7 +34,7 @@ @pytest.fixture() def restriction(): - return TimeRestriction(earliest=START_DATE, latest=END_DATE, catchup=True) + return TimeRestriction(earliest=START_DATE, latest=END_DATE, catchup=Catchup.ENABLE) @pytest.fixture() diff --git a/tests/timetables/test_events_timetable.py b/tests/timetables/test_events_timetable.py index eb28143de28ca..dc3abefe6b99d 100644 --- a/tests/timetables/test_events_timetable.py +++ b/tests/timetables/test_events_timetable.py @@ -23,6 +23,7 @@ from airflow.settings import TIMEZONE from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable from airflow.timetables.events import EventsTimetable +from airflow.utils.catchup import Catchup START_DATE = pendulum.DateTime(2021, 9, 4, tzinfo=TIMEZONE) # Precedes all events @@ -49,7 +50,7 @@ @pytest.fixture() def restriction(): - return TimeRestriction(earliest=START_DATE, latest=None, catchup=True) + return TimeRestriction(earliest=START_DATE, latest=None, catchup=Catchup.ENABLE) @pytest.fixture() diff --git a/tests/timetables/test_interval_timetable.py b/tests/timetables/test_interval_timetable.py index 596c274cf7152..fc3eff091f8f8 100644 --- a/tests/timetables/test_interval_timetable.py +++ b/tests/timetables/test_interval_timetable.py @@ -28,6 +28,7 @@ from airflow.settings import TIMEZONE from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable from airflow.timetables.interval import CronDataIntervalTimetable, DeltaDataIntervalTimetable +from airflow.utils.catchup import Catchup START_DATE = pendulum.DateTime(2021, 9, 4, tzinfo=TIMEZONE) @@ -56,10 +57,10 @@ def test_no_catchup_first_starts_at_current_time( last_automated_data_interval: DataInterval | None, ) -> None: - """If ``catchup=False`` and start_date is a day before""" + """If ``catchup="disable`` and start_date is a day before""" next_info = CRON_TIMETABLE.next_dagrun_info( last_automated_data_interval=last_automated_data_interval, - restriction=TimeRestriction(earliest=YESTERDAY, latest=None, catchup=False), + restriction=TimeRestriction(earliest=YESTERDAY, latest=None, catchup=Catchup.DISABLE), ) expected_start = YESTERDAY + DELTA_FROM_MIDNIGHT assert next_info == DagRunInfo.interval(start=expected_start, end=CURRENT_TIME + DELTA_FROM_MIDNIGHT) @@ -71,12 +72,12 @@ def test_no_catchup_first_starts_at_current_time( ) @pytest.mark.parametrize( "catchup", - [pytest.param(True, id="catchup_true"), pytest.param(False, id="catchup_false")], + [pytest.param(Catchup.ENABLE, id="catchup_enable"), pytest.param(Catchup.DISABLE, id="catchup_disable")], ) @time_machine.travel(CURRENT_TIME) def test_new_schedule_interval_next_info_starts_at_new_time( earliest: pendulum.DateTime | None, - catchup: bool, + catchup: Catchup, ) -> None: """First run after DAG has new schedule interval.""" next_info = CRON_TIMETABLE.next_dagrun_info( @@ -105,10 +106,10 @@ def test_no_catchup_next_info_starts_at_current_time( timetable: Timetable, last_automated_data_interval: DataInterval | None, ) -> None: - """If ``catchup=False``, the next data interval ends at the current time.""" + """If ``catchup="disable"``, the next data interval ends at the current time.""" next_info = timetable.next_dagrun_info( last_automated_data_interval=last_automated_data_interval, - restriction=TimeRestriction(earliest=START_DATE, latest=None, catchup=False), + restriction=TimeRestriction(earliest=START_DATE, latest=None, catchup=Catchup.DISABLE), ) expected_start = CURRENT_TIME - datetime.timedelta(hours=1) assert next_info == DagRunInfo.interval(start=expected_start, end=CURRENT_TIME) @@ -123,10 +124,10 @@ def test_no_catchup_next_info_starts_at_current_time( ], ) def test_catchup_next_info_starts_at_previous_interval_end(timetable: Timetable) -> None: - """If ``catchup=True``, the next interval starts at the previous's end.""" + """If ``catchup="enable"``, the next interval starts at the previous's end.""" next_info = timetable.next_dagrun_info( last_automated_data_interval=PREV_DATA_INTERVAL, - restriction=TimeRestriction(earliest=START_DATE, latest=None, catchup=True), + restriction=TimeRestriction(earliest=START_DATE, latest=None, catchup=Catchup.ENABLE), ) expected_end = PREV_DATA_INTERVAL_END + datetime.timedelta(hours=1) assert next_info == DagRunInfo.interval(start=PREV_DATA_INTERVAL_END, end=expected_end) @@ -250,6 +251,6 @@ def test_cron_next_dagrun_info_alignment(last_data_interval: DataInterval, expec timetable = CronDataIntervalTimetable("@daily", TIMEZONE) info = timetable.next_dagrun_info( last_automated_data_interval=last_data_interval, - restriction=TimeRestriction(None, None, True), + restriction=TimeRestriction(None, None, Catchup.ENABLE), ) assert info == expected_info diff --git a/tests/timetables/test_trigger_timetable.py b/tests/timetables/test_trigger_timetable.py index 58e1f49df9058..c07a146d719e8 100644 --- a/tests/timetables/test_trigger_timetable.py +++ b/tests/timetables/test_trigger_timetable.py @@ -28,6 +28,7 @@ from airflow.exceptions import AirflowTimetableInvalid from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction from airflow.timetables.trigger import CronTriggerTimetable +from airflow.utils.catchup import Catchup TIMEZONE = pendulum.tz.timezone("UTC") START_DATE = pendulum.DateTime(2021, 9, 4, tzinfo=TIMEZONE) @@ -72,7 +73,7 @@ def test_daily_cron_trigger_no_catchup_first_starts_at_next_schedule( timetable = CronTriggerTimetable("30 16 * * *", timezone=TIMEZONE) next_info = timetable.next_dagrun_info( last_automated_data_interval=last_automated_data_interval, - restriction=TimeRestriction(earliest=YESTERDAY, latest=None, catchup=False), + restriction=TimeRestriction(earliest=YESTERDAY, latest=None, catchup=Catchup.DISABLE), ) assert next_info == DagRunInfo.exact(next_start_time) @@ -120,7 +121,7 @@ def test_hourly_cron_trigger_no_catchup_next_info( with time_machine.travel(current_time): next_info = HOURLY_CRON_TRIGGER_TIMETABLE.next_dagrun_info( last_automated_data_interval=PREV_DATA_INTERVAL_EXACT, - restriction=TimeRestriction(earliest=earliest, latest=None, catchup=False), + restriction=TimeRestriction(earliest=earliest, latest=None, catchup=Catchup.DISABLE), ) assert next_info == expected @@ -167,7 +168,7 @@ def test_hourly_cron_trigger_catchup_next_info( ) -> None: next_info = HOURLY_CRON_TRIGGER_TIMETABLE.next_dagrun_info( last_automated_data_interval=last_automated_data_interval, - restriction=TimeRestriction(earliest=earliest, latest=None, catchup=True), + restriction=TimeRestriction(earliest=earliest, latest=None, catchup=Catchup.ENABLE), ) assert next_info == expected @@ -185,7 +186,7 @@ def test_cron_trigger_next_info_with_interval(): pendulum.DateTime(2022, 8, 1, tzinfo=TIMEZONE), pendulum.DateTime(2022, 8, 1, 16, 30, tzinfo=TIMEZONE), ), - restriction=TimeRestriction(earliest=START_DATE, latest=None, catchup=True), + restriction=TimeRestriction(earliest=START_DATE, latest=None, catchup=Catchup.ENABLE), ) assert next_info == DagRunInfo.interval( pendulum.DateTime(2022, 8, 8, tzinfo=TIMEZONE), diff --git a/tests/timetables/test_workday_timetable.py b/tests/timetables/test_workday_timetable.py index d20f45d44cf8a..c1b783f1b7e07 100644 --- a/tests/timetables/test_workday_timetable.py +++ b/tests/timetables/test_workday_timetable.py @@ -25,6 +25,7 @@ from airflow.example_dags.plugins.workday import AfterWorkdayTimetable from airflow.settings import TIMEZONE from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable +from airflow.utils.catchup import Catchup START_DATE = pendulum.DateTime(2021, 9, 4, tzinfo=TIMEZONE) # This is a Saturday. @@ -44,7 +45,7 @@ @pytest.fixture() def restriction(): - return TimeRestriction(earliest=START_DATE, latest=None, catchup=True) + return TimeRestriction(earliest=START_DATE, latest=None, catchup=Catchup.ENABLE) @pytest.fixture() From c538a07a6413ada460cfa34dce6303799221dd8c Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Tue, 7 Nov 2023 20:46:43 +0100 Subject: [PATCH 3/6] Update all examples --- airflow/models/dag.py | 8 ++++++-- airflow/utils/catchup.py | 4 ++-- airflow/www/views.py | 3 ++- tests/plugins/workday.py | 2 +- .../marketing_platform/operators/test_campaign_manager.py | 2 +- .../marketing_platform/operators/test_display_video.py | 2 +- .../marketing_platform/operators/test_search_ads.py | 2 +- .../openlineage/extractors/test_bash_extractor.py | 2 +- .../openlineage/extractors/test_python_extractor.py | 2 +- tests/providers/openlineage/plugins/test_utils.py | 5 ++++- tests/serialization/test_dag_serialization.py | 2 +- tests/serialization/test_serialized_objects.py | 2 +- .../providers/airbyte/example_airbyte_trigger_job.py | 2 +- tests/system/providers/alibaba/example_adb_spark_batch.py | 2 +- tests/system/providers/alibaba/example_adb_spark_sql.py | 2 +- tests/system/providers/alibaba/example_oss_bucket.py | 2 +- tests/system/providers/alibaba/example_oss_object.py | 2 +- .../system/providers/amazon/aws/example_glue_databrew.py | 4 +++- tests/system/providers/amazon/aws/example_http_to_s3.py | 2 +- tests/system/providers/apache/spark/example_pyspark.py | 2 +- .../common/io/example_file_transfer_local_to_s3.py | 2 +- tests/system/providers/databricks/example_databricks.py | 2 +- .../providers/databricks/example_databricks_repos.py | 2 +- .../providers/databricks/example_databricks_sensors.py | 2 +- .../system/providers/databricks/example_databricks_sql.py | 2 +- tests/system/providers/dingding/example_dingding.py | 2 +- .../elasticsearch/example_elasticsearch_query.py | 2 +- .../google/cloud/cloud_batch/example_cloud_batch.py | 2 +- .../google/cloud/cloud_build/example_cloud_build.py | 2 +- .../cloud/cloud_build/example_cloud_build_trigger.py | 2 +- .../google/cloud/cloud_functions/example_functions.py | 2 +- .../example_cloud_memorystore_memcached.py | 2 +- .../cloud_memorystore/example_cloud_memorystore_redis.py | 2 +- .../providers/google/cloud/cloud_sql/example_cloud_sql.py | 2 +- .../example_dlp_deidentify_content.py | 2 +- .../cloud/data_loss_prevention/example_dlp_info_types.py | 2 +- .../data_loss_prevention/example_dlp_inspect_template.py | 2 +- .../google/cloud/data_loss_prevention/example_dlp_job.py | 2 +- .../cloud/data_loss_prevention/example_dlp_job_trigger.py | 2 +- .../google/cloud/datafusion/example_datafusion.py | 2 +- .../google/cloud/datapipelines/example_datapipeline.py | 2 +- .../dataproc_metastore/example_dataproc_metastore.py | 2 +- .../example_dataproc_metastore_backup.py | 2 +- .../example_dataproc_metastore_hive_partition_sensor.py | 2 +- .../cloud/kubernetes_engine/example_kubernetes_engine.py | 2 +- .../kubernetes_engine/example_kubernetes_engine_async.py | 2 +- .../google/cloud/life_sciences/example_life_sciences.py | 2 +- .../cloud/natural_language/example_natural_language.py | 2 +- .../google/cloud/speech_to_text/example_speech_to_text.py | 2 +- .../google/cloud/sql_to_sheets/example_sql_to_sheets.py | 2 +- .../google/cloud/stackdriver/example_stackdriver.py | 2 +- .../example_cloud_storage_transfer_service_aws.py | 2 +- .../example_cloud_storage_transfer_service_gcp.py | 2 +- .../example_cloud_storage_transfer_service_gcs_to_gcs.py | 2 +- .../google/cloud/text_to_speech/example_text_to_speech.py | 2 +- .../google/cloud/transfers/example_postgres_to_gcs.py | 2 +- .../cloud/translate_speech/example_translate_speech.py | 2 +- .../example_vertex_ai_auto_ml_forecasting_training.py | 2 +- .../vertex_ai/example_vertex_ai_auto_ml_image_training.py | 2 +- .../vertex_ai/example_vertex_ai_auto_ml_list_training.py | 2 +- .../vertex_ai/example_vertex_ai_auto_ml_text_training.py | 2 +- .../vertex_ai/example_vertex_ai_auto_ml_video_training.py | 2 +- .../vertex_ai/example_vertex_ai_batch_prediction_job.py | 2 +- .../cloud/vertex_ai/example_vertex_ai_custom_container.py | 2 +- .../cloud/vertex_ai/example_vertex_ai_custom_job.py | 2 +- .../google/cloud/vertex_ai/example_vertex_ai_dataset.py | 2 +- .../google/cloud/vertex_ai/example_vertex_ai_endpoint.py | 2 +- .../cloud/vertex_ai/example_vertex_ai_model_service.py | 2 +- .../cloud/vertex_ai/example_vertex_ai_pipeline_job.py | 2 +- .../video_intelligence/example_video_intelligence.py | 2 +- .../providers/google/cloud/workflows/example_workflows.py | 2 +- .../google/datacatalog/example_datacatalog_entries.py | 2 +- .../datacatalog/example_datacatalog_search_catalog.py | 2 +- .../datacatalog/example_datacatalog_tag_templates.py | 2 +- .../google/datacatalog/example_datacatalog_tags.py | 2 +- tests/system/providers/google/leveldb/example_leveldb.py | 2 +- .../google/marketing_platform/example_analytics.py | 2 +- .../google/marketing_platform/example_campaign_manager.py | 2 +- .../google/marketing_platform/example_search_ads.py | 2 +- .../providers/google/suite/example_local_to_drive.py | 2 +- tests/system/providers/influxdb/example_influxdb_query.py | 2 +- .../providers/microsoft/azure/example_adf_run_pipeline.py | 2 +- .../microsoft/azure/example_azure_batch_operator.py | 2 +- .../microsoft/azure/example_azure_container_instances.py | 2 +- .../providers/microsoft/azure/example_azure_cosmosdb.py | 2 +- .../microsoft/azure/example_azure_service_bus.py | 2 +- .../providers/microsoft/azure/example_azure_synapse.py | 2 +- .../system/providers/microsoft/azure/example_fileshare.py | 2 +- .../providers/microsoft/azure/example_local_to_adls.py | 2 +- .../providers/microsoft/azure/example_local_to_wasb.py | 2 +- .../providers/microsoft/azure/example_sftp_to_wasb.py | 2 +- .../providers/microsoft/azure/example_wasb_sensors.py | 2 +- tests/system/providers/microsoft/mssql/example_mssql.py | 2 +- tests/system/providers/microsoft/winrm/example_winrm.py | 2 +- tests/system/providers/opensearch/example_opensearch.py | 2 +- tests/system/providers/opsgenie/example_opsgenie_alert.py | 2 +- tests/system/providers/papermill/example_papermill.py | 2 +- .../providers/papermill/example_papermill_verify.py | 2 +- tests/system/providers/plexus/example_plexus.py | 2 +- tests/system/providers/postgres/example_postgres.py | 2 +- tests/system/providers/presto/example_gcs_to_presto.py | 2 +- tests/system/providers/salesforce/example_bulk.py | 2 +- .../providers/salesforce/example_salesforce_apex_rest.py | 2 +- tests/system/providers/samba/example_gcs_to_samba.py | 2 +- tests/system/providers/singularity/example_singularity.py | 2 +- .../providers/snowflake/example_copy_into_snowflake.py | 2 +- tests/system/providers/snowflake/example_snowflake.py | 2 +- tests/system/providers/sqlite/example_sqlite.py | 2 +- tests/system/providers/tabular/example_tabular.py | 2 +- .../providers/zendesk/example_zendesk_custom_get.py | 2 +- tests/timetables/test_continuous_timetable.py | 2 +- tests/timetables/test_trigger_timetable.py | 2 +- 112 files changed, 124 insertions(+), 114 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 35d3e5f0bdb34..71bb6c1d4df46 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -452,7 +452,9 @@ def __init__( sla_miss_callback: None | SLAMissCallback | list[SLAMissCallback] = None, default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), - catchup: str | bool = airflow_conf.get("scheduler", "catchup_by_default"), + catchup: Literal["enable", "disable", "ignore_first"] | bool = airflow_conf.get_default_value( + "scheduler", "catchup_by_default", fallback="enable" + ), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, @@ -3838,7 +3840,9 @@ def dag( sla_miss_callback: None | SLAMissCallback | list[SLAMissCallback] = None, default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), - catchup: str | bool = airflow_conf.get("scheduler", "catchup_by_default"), + catchup: Literal["enable", "disable", "ignore_first"] | bool = airflow_conf.get_default_value( + "scheduler", "catchup_by_default", fallback="enable" + ), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, diff --git a/airflow/utils/catchup.py b/airflow/utils/catchup.py index 0086000008e61..eacffd59eae13 100644 --- a/airflow/utils/catchup.py +++ b/airflow/utils/catchup.py @@ -32,10 +32,10 @@ def _catchup_backwards_compatibility(cls, value: str | bool) -> Catchup: should_warn = False if isinstance(value, bool): should_warn = True - value = "enabled" if value else "disabled" + value = "enable" if value else "disable" if value in ("True", "False"): should_warn = True - value = "enabled" if value == "True" else "disabled" + value = "enable" if value == "True" else "disable" if should_warn: warnings.warn( "Passing a boolean to Catchup is deprecated. " diff --git a/airflow/www/views.py b/airflow/www/views.py index edaed24a436d3..d86ed0f89c7b2 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -117,6 +117,7 @@ from airflow.timetables.base import DataInterval, TimeRestriction from airflow.utils import json as utils_json, timezone, yaml from airflow.utils.airflow_flask_app import get_airflow_app +from airflow.utils.catchup import Catchup from airflow.utils.dag_edges import dag_edges from airflow.utils.dates import infer_time_unit, scale_time_units from airflow.utils.db import get_query_count @@ -2897,7 +2898,7 @@ def _convert_to_date(session, column): ) year = last_automated_data_interval.end.year - restriction = TimeRestriction(dag.start_date, dag.end_date, False, False) + restriction = TimeRestriction(dag.start_date, dag.end_date, Catchup.DISABLE) dates: dict[datetime.date, int] = collections.Counter() if isinstance(dag.timetable, CronMixin): diff --git a/tests/plugins/workday.py b/tests/plugins/workday.py index 20363a69e7a4b..a7ab992d47004 100644 --- a/tests/plugins/workday.py +++ b/tests/plugins/workday.py @@ -78,7 +78,7 @@ def next_dagrun_info( if next_start is None: # No start_date. Don't schedule. return None if not restriction.catchup: - # If the DAG has catchup=False, today is the earliest to consider. + # If the DAG has catchup="disable", today is the earliest to consider. next_start = max(next_start, DateTime.combine(Date.today(), Time.min).replace(tzinfo=UTC)) elif next_start.time() != Time.min: # If earliest does not fall on midnight, skip to the next day. 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 258ee92f91b1d..e76262e799cd7 100644 --- a/tests/providers/google/marketing_platform/operators/test_campaign_manager.py +++ b/tests/providers/google/marketing_platform/operators/test_campaign_manager.py @@ -182,7 +182,7 @@ def test_set_bucket_name( dag_id="test_set_bucket_name", start_date=DEFAULT_DATE, schedule=None, - catchup=False, + catchup="disable", ) if BUCKET_NAME not in test_bucket_name: 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 66b3ddd80932d..ffcf2a8df15a9 100644 --- a/tests/providers/google/marketing_platform/operators/test_display_video.py +++ b/tests/providers/google/marketing_platform/operators/test_display_video.py @@ -164,7 +164,7 @@ def test_set_bucket_name( dag_id="test_set_bucket_name", start_date=DEFAULT_DATE, schedule=None, - catchup=False, + catchup="disable", ) if BUCKET_NAME not in test_bucket_name: 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 a168854d0a34b..67968601f3b39 100644 --- a/tests/providers/google/marketing_platform/operators/test_search_ads.py +++ b/tests/providers/google/marketing_platform/operators/test_search_ads.py @@ -145,7 +145,7 @@ def test_set_bucket_name(self, hook_mock, gcs_hook_mock, tempfile_mock, test_buc dag_id="test_set_bucket_name", start_date=DEFAULT_DATE, schedule=None, - catchup=False, + catchup="disable", ) if BUCKET_NAME not in test_bucket_name: diff --git a/tests/providers/openlineage/extractors/test_bash_extractor.py b/tests/providers/openlineage/extractors/test_bash_extractor.py index 201583c0ed2aa..c0d004acef79a 100644 --- a/tests/providers/openlineage/extractors/test_bash_extractor.py +++ b/tests/providers/openlineage/extractors/test_bash_extractor.py @@ -37,7 +37,7 @@ description="Test dummy DAG", schedule="*/2 * * * *", start_date=datetime(2020, 1, 8), - catchup=False, + catchup="disable", max_active_runs=1, ) as dag: bash_task = BashOperator(task_id="bash-task", bash_command="ls -halt && exit 0", dag=dag) diff --git a/tests/providers/openlineage/extractors/test_python_extractor.py b/tests/providers/openlineage/extractors/test_python_extractor.py index 45cb35288f0fb..6ed5367afb78b 100644 --- a/tests/providers/openlineage/extractors/test_python_extractor.py +++ b/tests/providers/openlineage/extractors/test_python_extractor.py @@ -39,7 +39,7 @@ description="Test dummy DAG", schedule="*/2 * * * *", start_date=datetime(2020, 1, 8), - catchup=False, + catchup="disable", max_active_runs=1, ) diff --git a/tests/providers/openlineage/plugins/test_utils.py b/tests/providers/openlineage/plugins/test_utils.py index 19f647fcfaf48..dedbf48f97d33 100644 --- a/tests/providers/openlineage/plugins/test_utils.py +++ b/tests/providers/openlineage/plugins/test_utils.py @@ -107,7 +107,10 @@ def test_parse_version(): def test_to_json_encodable(): dag = AIRFLOW_DAG( - dag_id="test_dag", schedule_interval="*/2 * * * *", start_date=datetime.datetime.now(), catchup=False + dag_id="test_dag", + schedule_interval="*/2 * * * *", + start_date=datetime.datetime.now(), + catchup="disable", ) task = EmptyOperator(task_id="test_task", dag=dag) diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py index 4aecebcd1de84..f94f198d9b180 100644 --- a/tests/serialization/test_dag_serialization.py +++ b/tests/serialization/test_dag_serialization.py @@ -292,7 +292,7 @@ def compute_next_execution_date(dag, execution_date): "next_execution_date": compute_next_execution_date, }, user_defined_filters={"hello": lambda name: f"Hello {name}"}, - catchup=False, + catchup="disable", ) BashOperator( task_id="echo", diff --git a/tests/serialization/test_serialized_objects.py b/tests/serialization/test_serialized_objects.py index e05f69114c6da..78d9ece98053e 100644 --- a/tests/serialization/test_serialized_objects.py +++ b/tests/serialization/test_serialized_objects.py @@ -161,7 +161,7 @@ def equal_time(a: datetime, b: datetime) -> bool: schedule="*/10 * * * *", default_args={"depends_on_past": True}, start_date=datetime.utcnow(), - catchup=False, + catchup="disable", ), DAT.DAG, lambda a, b: a.dag_id == b.dag_id and equal_time(a.start_date, b.start_date), diff --git a/tests/system/providers/airbyte/example_airbyte_trigger_job.py b/tests/system/providers/airbyte/example_airbyte_trigger_job.py index 84238f4ca1855..d30f74aced3b6 100644 --- a/tests/system/providers/airbyte/example_airbyte_trigger_job.py +++ b/tests/system/providers/airbyte/example_airbyte_trigger_job.py @@ -35,7 +35,7 @@ start_date=datetime(2021, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_airbyte_synchronous] sync_source_destination = AirbyteTriggerSyncOperator( diff --git a/tests/system/providers/alibaba/example_adb_spark_batch.py b/tests/system/providers/alibaba/example_adb_spark_batch.py index 3deb1c94731ce..4919411eded3a 100644 --- a/tests/system/providers/alibaba/example_adb_spark_batch.py +++ b/tests/system/providers/alibaba/example_adb_spark_batch.py @@ -33,7 +33,7 @@ start_date=datetime(2021, 1, 1), default_args={"cluster_id": "your cluster", "rg_name": "your resource group", "region": "your region"}, max_active_runs=1, - catchup=False, + catchup="disable", ) as dag: spark_pi = AnalyticDBSparkBatchOperator( task_id="task1", diff --git a/tests/system/providers/alibaba/example_adb_spark_sql.py b/tests/system/providers/alibaba/example_adb_spark_sql.py index beff440608bb9..365e1431300fc 100644 --- a/tests/system/providers/alibaba/example_adb_spark_sql.py +++ b/tests/system/providers/alibaba/example_adb_spark_sql.py @@ -33,7 +33,7 @@ start_date=datetime(2021, 1, 1), default_args={"cluster_id": "your cluster", "rg_name": "your resource group", "region": "your region"}, max_active_runs=1, - catchup=False, + catchup="disable", ) as dag: show_databases = AnalyticDBSparkSQLOperator(task_id="task1", sql="SHOE DATABASES;") diff --git a/tests/system/providers/alibaba/example_oss_bucket.py b/tests/system/providers/alibaba/example_oss_bucket.py index 6a48f05e9587e..1883aef893c27 100644 --- a/tests/system/providers/alibaba/example_oss_bucket.py +++ b/tests/system/providers/alibaba/example_oss_bucket.py @@ -32,7 +32,7 @@ default_args={"bucket_name": "your bucket", "region": "your region"}, max_active_runs=1, tags=["example"], - catchup=False, + catchup="disable", ) as dag: create_bucket = OSSCreateBucketOperator(task_id="task1", region=REGION) diff --git a/tests/system/providers/alibaba/example_oss_object.py b/tests/system/providers/alibaba/example_oss_object.py index 002b23d9436c7..e23e132a9e134 100644 --- a/tests/system/providers/alibaba/example_oss_object.py +++ b/tests/system/providers/alibaba/example_oss_object.py @@ -38,7 +38,7 @@ default_args={"bucket_name": "your bucket", "region": "your region"}, max_active_runs=1, tags=["example"], - catchup=False, + catchup="disable", ) as dag: create_object = OSSUploadObjectOperator( file="your local file", diff --git a/tests/system/providers/amazon/aws/example_glue_databrew.py b/tests/system/providers/amazon/aws/example_glue_databrew.py index d44ebe383fd63..ef9038ac22105 100644 --- a/tests/system/providers/amazon/aws/example_glue_databrew.py +++ b/tests/system/providers/amazon/aws/example_glue_databrew.py @@ -90,7 +90,9 @@ def delete_job(job_name: str): client.delete_job(Name=job_name) -with DAG(DAG_ID, schedule="@once", start_date=pendulum.datetime(2023, 1, 1, tz="UTC"), catchup=False) as dag: +with DAG( + DAG_ID, schedule="@once", start_date=pendulum.datetime(2023, 1, 1, tz="UTC"), catchup="disable" +) as dag: test_context = sys_test_context_task() env_id = test_context["ENV_ID"] role_arn = test_context[ROLE_ARN_KEY] diff --git a/tests/system/providers/amazon/aws/example_http_to_s3.py b/tests/system/providers/amazon/aws/example_http_to_s3.py index b5d65abe69b1d..e283cf4224fa4 100644 --- a/tests/system/providers/amazon/aws/example_http_to_s3.py +++ b/tests/system/providers/amazon/aws/example_http_to_s3.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: test_context = sys_test_context_task() diff --git a/tests/system/providers/apache/spark/example_pyspark.py b/tests/system/providers/apache/spark/example_pyspark.py index cc18911a38c72..daa277d0ca716 100644 --- a/tests/system/providers/apache/spark/example_pyspark.py +++ b/tests/system/providers/apache/spark/example_pyspark.py @@ -32,7 +32,7 @@ @dag( schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, + catchup="disable", tags=["example"], ) def example_pyspark(): diff --git a/tests/system/providers/common/io/example_file_transfer_local_to_s3.py b/tests/system/providers/common/io/example_file_transfer_local_to_s3.py index ff5bfa0471ae7..14c5ce4262cf7 100644 --- a/tests/system/providers/common/io/example_file_transfer_local_to_s3.py +++ b/tests/system/providers/common/io/example_file_transfer_local_to_s3.py @@ -65,7 +65,7 @@ def remove_bucket(): schedule="@once", start_date=datetime(2021, 1, 1), # Override to match your needs tags=["example"], - catchup=False, + catchup="disable", ) as dag: temp_file = create_temp_file() temp_file_path = cast(ObjectStoragePath, temp_file) diff --git a/tests/system/providers/databricks/example_databricks.py b/tests/system/providers/databricks/example_databricks.py index 3a7ed3e53b2e0..f402174f8926c 100644 --- a/tests/system/providers/databricks/example_databricks.py +++ b/tests/system/providers/databricks/example_databricks.py @@ -50,7 +50,7 @@ schedule="@daily", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_databricks_jobs_create_json] # Example of using the JSON parameter to initialize the operator. diff --git a/tests/system/providers/databricks/example_databricks_repos.py b/tests/system/providers/databricks/example_databricks_repos.py index dd583c67b9c90..7e86694b5cc5d 100644 --- a/tests/system/providers/databricks/example_databricks_repos.py +++ b/tests/system/providers/databricks/example_databricks_repos.py @@ -41,7 +41,7 @@ start_date=datetime(2021, 1, 1), default_args=default_args, tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_databricks_repo_create] # Example of creating a Databricks Repo diff --git a/tests/system/providers/databricks/example_databricks_sensors.py b/tests/system/providers/databricks/example_databricks_sensors.py index fd572a6bd9055..e5a100d001203 100644 --- a/tests/system/providers/databricks/example_databricks_sensors.py +++ b/tests/system/providers/databricks/example_databricks_sensors.py @@ -35,7 +35,7 @@ schedule="@daily", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: dag.doc_md = textwrap.dedent( """ diff --git a/tests/system/providers/databricks/example_databricks_sql.py b/tests/system/providers/databricks/example_databricks_sql.py index 636a263ddcd9c..c739140c87172 100644 --- a/tests/system/providers/databricks/example_databricks_sql.py +++ b/tests/system/providers/databricks/example_databricks_sql.py @@ -44,7 +44,7 @@ schedule="@daily", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: connection_id = "my_connection" sql_endpoint_name = "My Endpoint" diff --git a/tests/system/providers/dingding/example_dingding.py b/tests/system/providers/dingding/example_dingding.py index 9165f5e370c08..856e2c12f2d62 100644 --- a/tests/system/providers/dingding/example_dingding.py +++ b/tests/system/providers/dingding/example_dingding.py @@ -51,7 +51,7 @@ def failure_callback(context): dagrun_timeout=timedelta(minutes=60), start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_dingding] text_msg_remind_none = DingdingOperator( diff --git a/tests/system/providers/elasticsearch/example_elasticsearch_query.py b/tests/system/providers/elasticsearch/example_elasticsearch_query.py index 595bdeb269dea..0ebc9c572f45f 100644 --- a/tests/system/providers/elasticsearch/example_elasticsearch_query.py +++ b/tests/system/providers/elasticsearch/example_elasticsearch_query.py @@ -69,7 +69,7 @@ def use_elasticsearch_hook(): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "elasticsearch"], ) as dag: execute_query = show_tables() diff --git a/tests/system/providers/google/cloud/cloud_batch/example_cloud_batch.py b/tests/system/providers/google/cloud/cloud_batch/example_cloud_batch.py index 362b0cc1bab96..240e0a38833ac 100644 --- a/tests/system/providers/google/cloud/cloud_batch/example_cloud_batch.py +++ b/tests/system/providers/google/cloud/cloud_batch/example_cloud_batch.py @@ -119,7 +119,7 @@ def _create_job(): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "batch"], ) as dag: # [START howto_operator_batch_submit_job] diff --git a/tests/system/providers/google/cloud/cloud_build/example_cloud_build.py b/tests/system/providers/google/cloud/cloud_build/example_cloud_build.py index ab57e23923e20..e63fd33e56d23 100644 --- a/tests/system/providers/google/cloud/cloud_build/example_cloud_build.py +++ b/tests/system/providers/google/cloud/cloud_build/example_cloud_build.py @@ -73,7 +73,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: diff --git a/tests/system/providers/google/cloud/cloud_build/example_cloud_build_trigger.py b/tests/system/providers/google/cloud/cloud_build/example_cloud_build_trigger.py index a6c5718f180a0..7bc48f15b9ce3 100644 --- a/tests/system/providers/google/cloud/cloud_build/example_cloud_build_trigger.py +++ b/tests/system/providers/google/cloud/cloud_build/example_cloud_build_trigger.py @@ -87,7 +87,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_create_build_trigger] diff --git a/tests/system/providers/google/cloud/cloud_functions/example_functions.py b/tests/system/providers/google/cloud/cloud_functions/example_functions.py index d8d13eb1c3483..1fe108e86c9d7 100644 --- a/tests/system/providers/google/cloud/cloud_functions/example_functions.py +++ b/tests/system/providers/google/cloud/cloud_functions/example_functions.py @@ -80,7 +80,7 @@ DAG_ID, default_args=default_args, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcp-functions"], ) as dag: # [START howto_operator_gcf_deploy] 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 7a6830e6dc121..1e0d15b4544bc 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 @@ -110,7 +110,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_private_service_connection = BashOperator( 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 3b6c1f46415d7..f216a6365a86d 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 @@ -72,7 +72,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 b11208c022fdd..1f835167060d8 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 @@ -150,7 +150,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "cloud_sql"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_deidentify_content.py b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_deidentify_content.py index 6c3d2cee9c6dc..3535ef4db7950 100644 --- a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_deidentify_content.py +++ b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_deidentify_content.py @@ -88,7 +88,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["dlp", "example"], ) as dag: # [START _howto_operator_dlp_deidentify_content] diff --git a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_info_types.py b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_info_types.py index 8bced99ff8f37..bece75c1ea31f 100644 --- a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_info_types.py +++ b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_info_types.py @@ -84,7 +84,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["dlp", "example"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_inspect_template.py b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_inspect_template.py index 23d09aae03285..d0d402e6427ca 100644 --- a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_inspect_template.py +++ b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_inspect_template.py @@ -57,7 +57,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["dlp", "example"], ) as dag: # [START howto_operator_dlp_create_inspect_template] diff --git a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job.py b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job.py index 86906c144505c..2a710d4697a4c 100644 --- a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job.py +++ b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job.py @@ -56,7 +56,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["dlp", "example"], ) as dag: create_job = CloudDLPCreateDLPJobOperator( diff --git a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job_trigger.py b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job_trigger.py index 73308f115c6d6..838fb9efa8580 100644 --- a/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job_trigger.py +++ b/tests/system/providers/google/cloud/data_loss_prevention/example_dlp_job_trigger.py @@ -55,7 +55,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["dlp", "example"], ) as dag: # [START howto_operator_dlp_create_job_trigger] diff --git a/tests/system/providers/google/cloud/datafusion/example_datafusion.py b/tests/system/providers/google/cloud/datafusion/example_datafusion.py index 38f443a706528..bc0784497e03f 100644 --- a/tests/system/providers/google/cloud/datafusion/example_datafusion.py +++ b/tests/system/providers/google/cloud/datafusion/example_datafusion.py @@ -168,7 +168,7 @@ with DAG( DAG_ID, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "datafusion"], ) as dag: create_bucket1 = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/datapipelines/example_datapipeline.py b/tests/system/providers/google/cloud/datapipelines/example_datapipeline.py index 519087734a646..f53e7a083c35b 100644 --- a/tests/system/providers/google/cloud/datapipelines/example_datapipeline.py +++ b/tests/system/providers/google/cloud/datapipelines/example_datapipeline.py @@ -60,7 +60,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "datapipeline"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore.py b/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore.py index 8b7208dd1e6c7..49ce5c82ecd5f 100644 --- a/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore.py +++ b/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore.py @@ -90,7 +90,7 @@ DAG_ID, start_date=datetime.datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "dataproc", "metastore"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_backup.py b/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_backup.py index c2423feaae294..40fb890139959 100644 --- a/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_backup.py +++ b/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_backup.py @@ -59,7 +59,7 @@ DAG_ID, start_date=datetime.datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "dataproc", "metastore"], ) as dag: create_service = DataprocMetastoreCreateServiceOperator( diff --git a/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_hive_partition_sensor.py b/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_hive_partition_sensor.py index 24fec2c5eee01..99c8a86a487b1 100644 --- a/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_hive_partition_sensor.py +++ b/tests/system/providers/google/cloud/dataproc_metastore/example_dataproc_metastore_hive_partition_sensor.py @@ -110,7 +110,7 @@ DAG_ID, start_date=datetime.datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example", "dataproc", "metastore", "partition", "hive", "sensor"], ) as dag: create_metastore_service = DataprocMetastoreCreateServiceOperator( 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 4f53fe5183a5c..b208cc7a844ef 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 @@ -46,7 +46,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_gke_create_cluster] diff --git a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py index 70c465a0c1992..7973582a4cb96 100644 --- a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py +++ b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py @@ -44,7 +44,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_gke_create_cluster_async] 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 e2361df2f858a..9c0ae4f76af05 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 @@ -89,7 +89,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) 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 ad8fb72014f10..55e056a4c678d 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 @@ -58,7 +58,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_gcp_natural_language_analyze_entities] 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 76ec51881ad7f..6204d7f0eab6e 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 @@ -53,7 +53,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "speech_to_text"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) 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 f166dadfafd62..4c4ed0d2bf8e4 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 @@ -176,7 +176,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", # Override to match your needs - catchup=False, + catchup="disable", tags=["example", "sql"], ) as dag: create_instance = ComputeEngineInsertInstanceOperator( diff --git a/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py b/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py index d4c6fcfbf4107..5e4388a995ac0 100644 --- a/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py +++ b/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py @@ -127,7 +127,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "stackdriver"], ) as dag: # [START howto_operator_gcp_stackdriver_upsert_notification_channel] diff --git a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py index c2fb4e59457e8..01c74b53975b2 100644 --- a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py +++ b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py @@ -103,7 +103,7 @@ with DAG( dag_id=DAG_ID, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "aws", "gcs", "transfer"], ) as dag: create_bucket_s3 = S3CreateBucketOperator( diff --git a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcp.py b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcp.py index 67159b8747ea6..a232108eb5d59 100644 --- a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcp.py +++ b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcp.py @@ -105,7 +105,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "transfer", "gcp"], ) as dag: create_bucket_src = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcs_to_gcs.py b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcs_to_gcs.py index 53752d2079133..9f456629bed71 100644 --- a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcs_to_gcs.py +++ b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_gcs_to_gcs.py @@ -54,7 +54,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "transfer", "gcs-to-gcs"], ) as dag: create_bucket_src = GCSCreateBucketOperator( 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 b185787ac04c6..4fc02af018d20 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 @@ -45,7 +45,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "text_to_speech"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/transfers/example_postgres_to_gcs.py b/tests/system/providers/google/cloud/transfers/example_postgres_to_gcs.py index 33e70c51ff9e4..2133cdd532e60 100644 --- a/tests/system/providers/google/cloud/transfers/example_postgres_to_gcs.py +++ b/tests/system/providers/google/cloud/transfers/example_postgres_to_gcs.py @@ -145,7 +145,7 @@ dag_id=DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "postgres", "gcs"], ) as dag: create_instance = ComputeEngineInsertInstanceOperator( 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 ff85ddc841177..1d92c6c72935d 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 @@ -59,7 +59,7 @@ DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_forecasting_training.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_forecasting_training.py index de75840bf5a21..ded6f2f092e28 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_forecasting_training.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_forecasting_training.py @@ -85,7 +85,7 @@ f"{DAG_ID}_forecasting_training_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "auto_ml"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_image_training.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_image_training.py index e204cf8f06449..16b8be7480bce 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_image_training.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_image_training.py @@ -72,7 +72,7 @@ f"{DAG_ID}_image_training_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "auto_ml"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_list_training.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_list_training.py index f8cfc87324155..1d6eacde2233d 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_list_training.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_list_training.py @@ -38,7 +38,7 @@ f"{DAG_ID}_list_training_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "auto_ml", "list_operation"], ) as dag: # [START how_to_cloud_vertex_ai_list_auto_ml_training_job_operator] diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_text_training.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_text_training.py index e5336b694315c..89d40fe30bbb3 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_text_training.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_text_training.py @@ -71,7 +71,7 @@ f"{DAG_ID}_text_training_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "auto_ml"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_video_training.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_video_training.py index 9ffe674045884..054527d9b50a4 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_video_training.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_auto_ml_video_training.py @@ -71,7 +71,7 @@ f"{DAG_ID}_video_training_job", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "auto_ml"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_batch_prediction_job.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_batch_prediction_job.py index 6bf932dafb08f..3f9d5a81e8f96 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_batch_prediction_job.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_batch_prediction_job.py @@ -96,7 +96,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", render_template_as_native_obj=True, tags=["example", "vertex_ai", "batch_prediction_job"], ) as dag: diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_container.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_container.py index 9e1459c07278c..8a100a65e61ad 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_container.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_container.py @@ -85,7 +85,7 @@ def TABULAR_DATASET(bucket_name): f"{DAG_ID}_custom_container", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "custom_job"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job.py index 7d19a26604faa..6e110f5afd15e 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_custom_job.py @@ -81,7 +81,7 @@ def TABULAR_DATASET(bucket_name): f"{DAG_ID}_custom", schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "custom_job"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_dataset.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_dataset.py index 21b369f7c366f..68e397ced5413 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_dataset.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_dataset.py @@ -113,7 +113,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "dataset"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_endpoint.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_endpoint.py index 1e2c348ee4799..abf61d60a34a1 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_endpoint.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_endpoint.py @@ -83,7 +83,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", render_template_as_native_obj=True, tags=["example", "vertex_ai", "endpoint_service"], ) as dag: diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_model_service.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_model_service.py index 5ddd6b6c295f0..17f76dcfc3d0e 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_model_service.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_model_service.py @@ -114,7 +114,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", render_template_as_native_obj=True, tags=["example", "vertex_ai", "model_service"], ) as dag: diff --git a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_pipeline_job.py b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_pipeline_job.py index 2175d0b33e988..9b6231b601850 100644 --- a/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_pipeline_job.py +++ b/tests/system/providers/google/cloud/vertex_ai/example_vertex_ai_pipeline_job.py @@ -72,7 +72,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "vertex_ai", "pipeline_job"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/google/cloud/video_intelligence/example_video_intelligence.py b/tests/system/providers/google/cloud/video_intelligence/example_video_intelligence.py index 7dbfd933548c0..e476471e2f034 100644 --- a/tests/system/providers/google/cloud/video_intelligence/example_video_intelligence.py +++ b/tests/system/providers/google/cloud/video_intelligence/example_video_intelligence.py @@ -66,7 +66,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME_DST) diff --git a/tests/system/providers/google/cloud/workflows/example_workflows.py b/tests/system/providers/google/cloud/workflows/example_workflows.py index c2c313d363272..5a901aa195713 100644 --- a/tests/system/providers/google/cloud/workflows/example_workflows.py +++ b/tests/system/providers/google/cloud/workflows/example_workflows.py @@ -91,7 +91,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "workflows"], ) as dag: # [START how_to_create_workflow] diff --git a/tests/system/providers/google/datacatalog/example_datacatalog_entries.py b/tests/system/providers/google/datacatalog/example_datacatalog_entries.py index cf4276d3a2dd5..91bae5be10e3e 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_entries.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_entries.py @@ -54,7 +54,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) 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 2beeebe54f568..9dd7b7a7ddd9b 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_search_catalog.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_search_catalog.py @@ -59,7 +59,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) 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 24ba00b630fe4..8df67ba4deee0 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_tag_templates.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_tag_templates.py @@ -53,7 +53,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # Create # [START howto_operator_gcp_datacatalog_create_tag_template] diff --git a/tests/system/providers/google/datacatalog/example_datacatalog_tags.py b/tests/system/providers/google/datacatalog/example_datacatalog_tags.py index 5d7f1cff3ed0c..b95a73015cbc4 100644 --- a/tests/system/providers/google/datacatalog/example_datacatalog_tags.py +++ b/tests/system/providers/google/datacatalog/example_datacatalog_tags.py @@ -60,7 +60,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=BUCKET_NAME) diff --git a/tests/system/providers/google/leveldb/example_leveldb.py b/tests/system/providers/google/leveldb/example_leveldb.py index 31dd363f2c955..9ecd498338dfa 100644 --- a/tests/system/providers/google/leveldb/example_leveldb.py +++ b/tests/system/providers/google/leveldb/example_leveldb.py @@ -42,7 +42,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_operator_leveldb_get_key] diff --git a/tests/system/providers/google/marketing_platform/example_analytics.py b/tests/system/providers/google/marketing_platform/example_analytics.py index ce95f27297fe7..4c5aff596ea11 100644 --- a/tests/system/providers/google/marketing_platform/example_analytics.py +++ b/tests/system/providers/google/marketing_platform/example_analytics.py @@ -47,7 +47,7 @@ DAG_ID, schedule="@once", # Override to match your needs, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "analytics"], ) as dag: # [START howto_marketing_platform_list_accounts_operator] 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 61e4253d75ac4..7a09f0d0d74ff 100644 --- a/tests/system/providers/google/marketing_platform/example_campaign_manager.py +++ b/tests/system/providers/google/marketing_platform/example_campaign_manager.py @@ -111,7 +111,7 @@ DAG_ID, schedule="@once", # Override to match your needs, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "campaign"], ) as dag: create_bucket = GCSCreateBucketOperator( 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 0f16e816c9cdc..003debcb7966b 100644 --- a/tests/system/providers/google/marketing_platform/example_search_ads.py +++ b/tests/system/providers/google/marketing_platform/example_search_ads.py @@ -55,7 +55,7 @@ DAG_ID, schedule="@once", # Override to match your needs, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START howto_search_ads_generate_report_operator] diff --git a/tests/system/providers/google/suite/example_local_to_drive.py b/tests/system/providers/google/suite/example_local_to_drive.py index 7a94fdfa6465c..e2faf4f34d50e 100644 --- a/tests/system/providers/google/suite/example_local_to_drive.py +++ b/tests/system/providers/google/suite/example_local_to_drive.py @@ -59,7 +59,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gdrive"], ) as dag: diff --git a/tests/system/providers/influxdb/example_influxdb_query.py b/tests/system/providers/influxdb/example_influxdb_query.py index 57275f63a0b34..c463c9aa1c7e7 100644 --- a/tests/system/providers/influxdb/example_influxdb_query.py +++ b/tests/system/providers/influxdb/example_influxdb_query.py @@ -29,7 +29,7 @@ DAG_ID, start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_influxdb] 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 a90f8c261b59a..01ee0c9214ff4 100644 --- a/tests/system/providers/microsoft/azure/example_adf_run_pipeline.py +++ b/tests/system/providers/microsoft/azure/example_adf_run_pipeline.py @@ -42,7 +42,7 @@ dag_id=DAG_ID, start_date=datetime(2021, 8, 13), schedule="@daily", - catchup=False, + catchup="disable", default_args={ "retries": 1, "retry_delay": timedelta(minutes=3), diff --git a/tests/system/providers/microsoft/azure/example_azure_batch_operator.py b/tests/system/providers/microsoft/azure/example_azure_batch_operator.py index 435ef05060ab1..3cd91b3ddd5f3 100644 --- a/tests/system/providers/microsoft/azure/example_azure_batch_operator.py +++ b/tests/system/providers/microsoft/azure/example_azure_batch_operator.py @@ -35,7 +35,7 @@ with DAG( dag_id="example_azure_batch", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", doc_md=__doc__, tags=["example"], ) 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 9a7d8abac1cd4..902fa09236a98 100644 --- a/tests/system/providers/microsoft/azure/example_azure_container_instances.py +++ b/tests/system/providers/microsoft/azure/example_azure_container_instances.py @@ -37,7 +37,7 @@ default_args={"retries": 1}, schedule=timedelta(days=1), start_date=datetime(2018, 11, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: t1 = AzureContainerInstancesOperator( diff --git a/tests/system/providers/microsoft/azure/example_azure_cosmosdb.py b/tests/system/providers/microsoft/azure/example_azure_cosmosdb.py index b33186355457c..3264f58b005a7 100644 --- a/tests/system/providers/microsoft/azure/example_azure_cosmosdb.py +++ b/tests/system/providers/microsoft/azure/example_azure_cosmosdb.py @@ -40,7 +40,7 @@ dag_id=DAG_ID, default_args={"database_name": "airflow_example_db"}, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", doc_md=__doc__, tags=["example"], ) as dag: 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 e0b8558ffec10..ed958b062b011 100644 --- a/tests/system/providers/microsoft/azure/example_azure_service_bus.py +++ b/tests/system/providers/microsoft/azure/example_azure_service_bus.py @@ -53,7 +53,7 @@ dag_id="example_azure_service_bus", start_date=datetime(2021, 8, 13), schedule=None, - catchup=False, + catchup="disable", default_args={ "execution_timeout": timedelta(hours=EXECUTION_TIMEOUT), "azure_service_bus_conn_id": "azure_service_bus_default", diff --git a/tests/system/providers/microsoft/azure/example_azure_synapse.py b/tests/system/providers/microsoft/azure/example_azure_synapse.py index 7465b71677deb..6321fab3e3a2a 100644 --- a/tests/system/providers/microsoft/azure/example_azure_synapse.py +++ b/tests/system/providers/microsoft/azure/example_azure_synapse.py @@ -57,7 +57,7 @@ dag_id="example_synapse_spark_job", start_date=datetime(2022, 1, 1), schedule=None, - catchup=False, + catchup="disable", default_args=default_args, tags=["example", "synapse"], ) as dag: diff --git a/tests/system/providers/microsoft/azure/example_fileshare.py b/tests/system/providers/microsoft/azure/example_fileshare.py index f306a0188efa9..10e1f108d397f 100644 --- a/tests/system/providers/microsoft/azure/example_fileshare.py +++ b/tests/system/providers/microsoft/azure/example_fileshare.py @@ -51,7 +51,7 @@ def delete_fileshare(): DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: create_fileshare() >> delete_fileshare() 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 f5a75e7ce414b..080a6f777c3a5 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 @@ with models.DAG( DAG_ID, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", schedule=None, tags=["example"], ) as dag: 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 b03c11e6b3673..45ca65bda2641 100644 --- a/tests/system/providers/microsoft/azure/example_local_to_wasb.py +++ b/tests/system/providers/microsoft/azure/example_local_to_wasb.py @@ -33,7 +33,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_local_to_wasb] upload = LocalFilesystemToWasbOperator( 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 d80539cb4a238..44c752c55e817 100644 --- a/tests/system/providers/microsoft/azure/example_sftp_to_wasb.py +++ b/tests/system/providers/microsoft/azure/example_sftp_to_wasb.py @@ -47,7 +47,7 @@ def delete_sftp_file(): with DAG( DAG_ID, schedule=None, - catchup=False, + catchup="disable", start_date=datetime(2021, 1, 1), # Override to match your needs ) as dag: transfer_files_to_sftp_step = SFTPOperator( diff --git a/tests/system/providers/microsoft/azure/example_wasb_sensors.py b/tests/system/providers/microsoft/azure/example_wasb_sensors.py index a0961113f297d..99be2b6133d1b 100644 --- a/tests/system/providers/microsoft/azure/example_wasb_sensors.py +++ b/tests/system/providers/microsoft/azure/example_wasb_sensors.py @@ -40,7 +40,7 @@ with DAG( "example_wasb_sensors", start_date=datetime(2022, 8, 8), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START wasb_blob_sensor] diff --git a/tests/system/providers/microsoft/mssql/example_mssql.py b/tests/system/providers/microsoft/mssql/example_mssql.py index dd87b41e07a00..f9250ea5d8b10 100644 --- a/tests/system/providers/microsoft/mssql/example_mssql.py +++ b/tests/system/providers/microsoft/mssql/example_mssql.py @@ -43,7 +43,7 @@ schedule="@daily", start_date=datetime(2021, 10, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_mssql] diff --git a/tests/system/providers/microsoft/winrm/example_winrm.py b/tests/system/providers/microsoft/winrm/example_winrm.py index f0ab204b32536..a718f1bde6174 100644 --- a/tests/system/providers/microsoft/winrm/example_winrm.py +++ b/tests/system/providers/microsoft/winrm/example_winrm.py @@ -51,7 +51,7 @@ start_date=datetime(2021, 1, 1), dagrun_timeout=timedelta(minutes=60), tags=["example"], - catchup=False, + catchup="disable", ) as dag: run_this_last = EmptyOperator(task_id="run_this_last") diff --git a/tests/system/providers/opensearch/example_opensearch.py b/tests/system/providers/opensearch/example_opensearch.py index 6a8660557cf78..3fb474e7f1480 100644 --- a/tests/system/providers/opensearch/example_opensearch.py +++ b/tests/system/providers/opensearch/example_opensearch.py @@ -76,7 +76,7 @@ def load_connections(): dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", tags=["example"], default_args=default_args, description="Examples of OpenSearch Operators", diff --git a/tests/system/providers/opsgenie/example_opsgenie_alert.py b/tests/system/providers/opsgenie/example_opsgenie_alert.py index 30fdabc0d5e95..5c19c19663d07 100644 --- a/tests/system/providers/opsgenie/example_opsgenie_alert.py +++ b/tests/system/providers/opsgenie/example_opsgenie_alert.py @@ -33,7 +33,7 @@ dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START howto_opsgenie_create_alert_operator] opsgenie_alert_operator = OpsgenieCreateAlertOperator(task_id="opsgenie_task", message="Hello World!") diff --git a/tests/system/providers/papermill/example_papermill.py b/tests/system/providers/papermill/example_papermill.py index 82d2e1b686a98..d00ce4fcd382b 100644 --- a/tests/system/providers/papermill/example_papermill.py +++ b/tests/system/providers/papermill/example_papermill.py @@ -40,7 +40,7 @@ start_date=START_DATE, dagrun_timeout=DAGRUN_TIMEOUT, tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_papermill] run_this = PapermillOperator( diff --git a/tests/system/providers/papermill/example_papermill_verify.py b/tests/system/providers/papermill/example_papermill_verify.py index 64665e64a2e1e..1628d7f327498 100644 --- a/tests/system/providers/papermill/example_papermill_verify.py +++ b/tests/system/providers/papermill/example_papermill_verify.py @@ -60,7 +60,7 @@ def check_notebook(inlets, execution_date): schedule=SCHEDULE_INTERVAL, start_date=START_DATE, dagrun_timeout=DAGRUN_TIMEOUT, - catchup=False, + catchup="disable", ) as dag: run_this = PapermillOperator( task_id="run_example_notebook", diff --git a/tests/system/providers/plexus/example_plexus.py b/tests/system/providers/plexus/example_plexus.py index ad957133b091c..89d9ceb1989e8 100644 --- a/tests/system/providers/plexus/example_plexus.py +++ b/tests/system/providers/plexus/example_plexus.py @@ -33,7 +33,7 @@ description="testing plexus operator", start_date=datetime(2021, 1, 1), schedule="@once", - catchup=False, + catchup="disable", ) as dag: # [START plexus_job_op] t1 = PlexusJobOperator( diff --git a/tests/system/providers/postgres/example_postgres.py b/tests/system/providers/postgres/example_postgres.py index fd0a80783c9fa..40617c7153a0c 100644 --- a/tests/system/providers/postgres/example_postgres.py +++ b/tests/system/providers/postgres/example_postgres.py @@ -35,7 +35,7 @@ dag_id=DAG_ID, start_date=datetime.datetime(2020, 2, 2), schedule="@once", - catchup=False, + catchup="disable", ) as dag: # [START postgres_operator_howto_guide_create_pet_table] create_pet_table = PostgresOperator( diff --git a/tests/system/providers/presto/example_gcs_to_presto.py b/tests/system/providers/presto/example_gcs_to_presto.py index f207f69871ce6..fc4ce08a7b638 100644 --- a/tests/system/providers/presto/example_gcs_to_presto.py +++ b/tests/system/providers/presto/example_gcs_to_presto.py @@ -36,7 +36,7 @@ dag_id=DAG_ID, schedule="@once", # Override to match your needs start_date=datetime(2022, 1, 1), - catchup=False, + catchup="disable", tags=["example"], ) as dag: # [START gcs_csv_to_presto_table] diff --git a/tests/system/providers/salesforce/example_bulk.py b/tests/system/providers/salesforce/example_bulk.py index 06764aaf54b42..c2ee58964d9da 100644 --- a/tests/system/providers/salesforce/example_bulk.py +++ b/tests/system/providers/salesforce/example_bulk.py @@ -29,7 +29,7 @@ dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START howto_salesforce_bulk_insert_operation] bulk_insert = SalesforceBulkOperator( diff --git a/tests/system/providers/salesforce/example_salesforce_apex_rest.py b/tests/system/providers/salesforce/example_salesforce_apex_rest.py index 9bfdb94839621..52924afcb1927 100644 --- a/tests/system/providers/salesforce/example_salesforce_apex_rest.py +++ b/tests/system/providers/salesforce/example_salesforce_apex_rest.py @@ -30,7 +30,7 @@ dag_id="salesforce_apex_rest_operator_dag", schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: # [START howto_salesforce_apex_rest_operator] payload = {"activity": [{"user": "12345", "action": "update page", "time": "2014-04-21T13:00:15Z"}]} diff --git a/tests/system/providers/samba/example_gcs_to_samba.py b/tests/system/providers/samba/example_gcs_to_samba.py index 23df501f4ccd7..269854cc394a4 100644 --- a/tests/system/providers/samba/example_gcs_to_samba.py +++ b/tests/system/providers/samba/example_gcs_to_samba.py @@ -50,7 +50,7 @@ DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", tags=["example", "gcs", "smb"], ) as dag: create_bucket = GCSCreateBucketOperator( diff --git a/tests/system/providers/singularity/example_singularity.py b/tests/system/providers/singularity/example_singularity.py index d802fbb31e820..d3312152a931a 100644 --- a/tests/system/providers/singularity/example_singularity.py +++ b/tests/system/providers/singularity/example_singularity.py @@ -32,7 +32,7 @@ default_args={"retries": 1}, schedule=timedelta(minutes=10), start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: t1 = BashOperator(task_id="print_date", bash_command="date") diff --git a/tests/system/providers/snowflake/example_copy_into_snowflake.py b/tests/system/providers/snowflake/example_copy_into_snowflake.py index 1cc3902d1b0ee..d4d6ef7117825 100644 --- a/tests/system/providers/snowflake/example_copy_into_snowflake.py +++ b/tests/system/providers/snowflake/example_copy_into_snowflake.py @@ -44,7 +44,7 @@ default_args={"snowflake_conn_id": SNOWFLAKE_CONN_ID}, tags=["example"], schedule="@once", - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_s3_copy_into_snowflake] copy_into_table = CopyFromExternalStageToSnowflakeOperator( diff --git a/tests/system/providers/snowflake/example_snowflake.py b/tests/system/providers/snowflake/example_snowflake.py index 526eb662159b3..538b326874fbb 100644 --- a/tests/system/providers/snowflake/example_snowflake.py +++ b/tests/system/providers/snowflake/example_snowflake.py @@ -46,7 +46,7 @@ default_args={"snowflake_conn_id": SNOWFLAKE_CONN_ID}, tags=["example"], schedule="@once", - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_snowflake] snowflake_op_sql_str = SnowflakeOperator(task_id="snowflake_op_sql_str", sql=CREATE_TABLE_SQL_STRING) diff --git a/tests/system/providers/sqlite/example_sqlite.py b/tests/system/providers/sqlite/example_sqlite.py index 0fb11a5b15dc2..2be7475ef91bd 100644 --- a/tests/system/providers/sqlite/example_sqlite.py +++ b/tests/system/providers/sqlite/example_sqlite.py @@ -39,7 +39,7 @@ schedule="@daily", start_date=datetime(2021, 1, 1), tags=["example"], - catchup=False, + catchup="disable", ) as dag: # [START howto_operator_sqlite] diff --git a/tests/system/providers/tabular/example_tabular.py b/tests/system/providers/tabular/example_tabular.py index 01c9a8b5ff40d..3333cd21217c0 100644 --- a/tests/system/providers/tabular/example_tabular.py +++ b/tests/system/providers/tabular/example_tabular.py @@ -39,7 +39,7 @@ }, start_date=datetime(2021, 1, 1), schedule=timedelta(1), - catchup=False, + catchup="disable", ) as dag: # This also works for the SparkSubmit operator BashOperator( diff --git a/tests/system/providers/zendesk/example_zendesk_custom_get.py b/tests/system/providers/zendesk/example_zendesk_custom_get.py index 701a109fd1159..79c1c3404447d 100644 --- a/tests/system/providers/zendesk/example_zendesk_custom_get.py +++ b/tests/system/providers/zendesk/example_zendesk_custom_get.py @@ -40,7 +40,7 @@ def fetch_organizations() -> list[dict]: dag_id=DAG_ID, schedule=None, start_date=datetime(2021, 1, 1), - catchup=False, + catchup="disable", ) as dag: fetch_organizations() diff --git a/tests/timetables/test_continuous_timetable.py b/tests/timetables/test_continuous_timetable.py index dc78d1c338769..c0cc2f674448b 100644 --- a/tests/timetables/test_continuous_timetable.py +++ b/tests/timetables/test_continuous_timetable.py @@ -45,7 +45,7 @@ def timetable(): def test_no_runs_without_start_date(timetable): next_info = timetable.next_dagrun_info( last_automated_data_interval=None, - restriction=TimeRestriction(earliest=None, latest=None, catchup=False), + restriction=TimeRestriction(earliest=None, latest=None, catchup=Catchup.DISABLE), ) assert next_info is None diff --git a/tests/timetables/test_trigger_timetable.py b/tests/timetables/test_trigger_timetable.py index c07a146d719e8..5cacf67ac894c 100644 --- a/tests/timetables/test_trigger_timetable.py +++ b/tests/timetables/test_trigger_timetable.py @@ -69,7 +69,7 @@ def test_daily_cron_trigger_no_catchup_first_starts_at_next_schedule( last_automated_data_interval: DataInterval | None, next_start_time: pendulum.DateTime, ) -> None: - """If ``catchup=False`` and start_date is a day before""" + """If ``catchup="disable"`` and start_date is a day before""" timetable = CronTriggerTimetable("30 16 * * *", timezone=TIMEZONE) next_info = timetable.next_dagrun_info( last_automated_data_interval=last_automated_data_interval, From 510b10e819ffe5b897337c2aa1463376d694850b Mon Sep 17 00:00:00 2001 From: hussein-awala Date: Sat, 25 Nov 2023 21:48:11 +0200 Subject: [PATCH 4/6] Make dag details enpoint b/c --- airflow/api_connexion/openapi/v1.yaml | 4 +++- airflow/api_connexion/schemas/dag_schema.py | 7 ++++++- airflow/models/dag.py | 2 ++ airflow/serialization/schema.json | 7 ++++++- airflow/www/static/js/types/api-generated.ts | 2 +- 5 files changed, 18 insertions(+), 4 deletions(-) diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 6ccaf013b9e53..27b6bf12d8090 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -3572,7 +3572,9 @@ components: timezone: $ref: "#/components/schemas/Timezone" catchup: - type: string + oneOf: + - type: string + - type: boolean readOnly: true orientation: type: string diff --git a/airflow/api_connexion/schemas/dag_schema.py b/airflow/api_connexion/schemas/dag_schema.py index c771aca73d4ca..314e0414b9afb 100644 --- a/airflow/api_connexion/schemas/dag_schema.py +++ b/airflow/api_connexion/schemas/dag_schema.py @@ -95,7 +95,7 @@ class DAGDetailSchema(DAGSchema): owners = fields.Method("get_owners", dump_only=True) timezone = TimezoneField() - catchup = fields.Boolean() + catchup = fields.Method("get_catchup", dump_only=True) orientation = fields.String() concurrency = fields.Method("get_concurrency") # TODO: Remove in Airflow 3.0 max_active_tasks = fields.Integer() @@ -148,6 +148,11 @@ def get_params(obj: DAG): params = obj.params return {k: v.dump() for k, v in params.items()} + @staticmethod + def get_catchup(obj: DAG): + """Get catchup value defined in a DAG.""" + return obj._backport_catchup + class DAGCollection(NamedTuple): """List of DAGs with metadata.""" diff --git a/airflow/models/dag.py b/airflow/models/dag.py index f8118086f02d7..a3dbb154d4c02 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -632,6 +632,8 @@ def __init__( f"{ORIENTATION_PRESETS}, but get {orientation}" ) self.catchup: Catchup = Catchup(catchup) + # For backward compatibility, we need to store the original value and return it in the API + self._backport_catchup = catchup self.partial: bool = False self.on_success_callback = on_success_callback diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 9e0d876968b65..1ca27f1d02479 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -130,7 +130,12 @@ "value": { "$ref": "#/definitions/dict" } } }, - "catchup": { "type": "string" }, + "catchup": { + "oneOf": [ + { "type": "boolean" }, + { "type": "string" } + ] + }, "is_subdag": { "type": "boolean" }, "fileloc": { "type" : "string"}, "_processor_dags_folder": { diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index 9f54581d8565f..26a1dcdbafcf7 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -1471,7 +1471,7 @@ export interface components { */ DAGDetail: components["schemas"]["DAG"] & { timezone?: components["schemas"]["Timezone"]; - catchup?: string; + catchup?: string | boolean; orientation?: string; concurrency?: number; /** From 39425093637a889acc447e50436dbc5a150d7931 Mon Sep 17 00:00:00 2001 From: hussein-awala Date: Sat, 25 Nov 2023 22:03:27 +0200 Subject: [PATCH 5/6] Update catchup doc --- docs/apache-airflow/core-concepts/dag-run.rst | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/apache-airflow/core-concepts/dag-run.rst b/docs/apache-airflow/core-concepts/dag-run.rst index 160a836b6f4ef..645050d41bc98 100644 --- a/docs/apache-airflow/core-concepts/dag-run.rst +++ b/docs/apache-airflow/core-concepts/dag-run.rst @@ -127,7 +127,7 @@ 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.), -then you will want to turn catchup off. This can be done by setting ``catchup="disable"`` in DAG or ``catchup_by_default=False`` +then you will want to turn catchup off. This can be done by setting ``catchup="disable"`` in DAG or ``catchup_by_default="disable"`` in the configuration file. When turned off, the scheduler creates a DAG run only for the latest interval. .. code-block:: python @@ -167,11 +167,14 @@ In such a case, the single DAG Run created will cover data between 2016-01-01 06 differences between a cron and a delta based schedule, take a look at the :ref:`timetables comparison ` -If the ``dag.catchup`` value had been ``True`` instead, the scheduler would have created a DAG Run +If the ``dag.catchup`` value had been ``"enable"`` instead, the scheduler would have created a DAG Run for each completed interval between 2015-12-01 and 2016-01-02 (but not yet one for 2016-01-02, as that interval hasn't completed) and the scheduler will execute them sequentially. -Catchup is also triggered when you turn off a DAG for a specified period and then re-enable it. +Catchup is also triggered when you turn off a DAG for a specified period and then re-enable it, or the dag run was +missed for some reason (for example there was a pressure on the scheduler, or the scheduler was down). +If you want to keep the catchup on for this case, without triggering all the dag runs between the dag start date and +the logical date of your first dag run, you can set ``dag.catchup`` to ``"ignore_first"``. This behavior is great for atomic datasets that can easily be split into periods. Turning catchup off is great if your DAG performs catchup internally. From 74f04025f0c8eb548484ed68568bba29cb7f28f9 Mon Sep 17 00:00:00 2001 From: hussein-awala Date: Sun, 26 Nov 2023 12:07:31 +0200 Subject: [PATCH 6/6] Fix serialization schema for dag --- airflow/serialization/schema.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 1ca27f1d02479..a0c30a5d836e4 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -130,7 +130,8 @@ "value": { "$ref": "#/definitions/dict" } } }, - "catchup": { + "catchup": { "type": "string" }, + "_backport_catchup": { "oneOf": [ { "type": "boolean" }, { "type": "string" }