diff --git a/airflow/cli/commands/dag_command.py b/airflow/cli/commands/dag_command.py index 2a8031cf90208..7f64f9006d68e 100644 --- a/airflow/cli/commands/dag_command.py +++ b/airflow/cli/commands/dag_command.py @@ -133,10 +133,12 @@ def dag_backfill(args, dag: list[DAG] | DAG | None = None) -> None: """Create backfill job or dry run for a DAG or list of DAGs using regex.""" logging.basicConfig(level=settings.LOGGING_LEVEL, format=settings.SIMPLE_LOG_FORMAT) signal.signal(signal.SIGTERM, sigint_handler) - warnings.warn( - "--ignore-first-depends-on-past is deprecated as the value is always set to True", - category=RemovedInAirflow3Warning, - ) + if args.ignore_first_depends_on_past: + warnings.warn( + "--ignore-first-depends-on-past is deprecated as the value is always set to True", + category=RemovedInAirflow3Warning, + ) + args.ignore_first_depends_on_past = True if not args.treat_dag_id_as_regex and args.treat_dag_as_regex: warnings.warn( @@ -145,9 +147,6 @@ def dag_backfill(args, dag: list[DAG] | DAG | None = None) -> None: ) args.treat_dag_id_as_regex = args.treat_dag_as_regex - if args.ignore_first_depends_on_past is False: - args.ignore_first_depends_on_past = True - if not args.start_date and not args.end_date: raise AirflowException("Provide a start_date and/or end_date") diff --git a/tests/cli/commands/test_dag_command.py b/tests/cli/commands/test_dag_command.py index cac538139b287..fb472fbdb0c50 100644 --- a/tests/cli/commands/test_dag_command.py +++ b/tests/cli/commands/test_dag_command.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import argparse import contextlib import json import os @@ -34,7 +35,7 @@ from airflow.cli import cli_parser from airflow.cli.commands import dag_command from airflow.decorators import task -from airflow.exceptions import AirflowException +from airflow.exceptions import AirflowException, RemovedInAirflow3Warning from airflow.models import DagBag, DagModel, DagRun from airflow.models.baseoperator import BaseOperator from airflow.models.dag import _run_inline_trigger @@ -61,6 +62,8 @@ class TestCliDags: + parser: argparse.ArgumentParser + @classmethod def setup_class(cls): cls.dagbag = DagBag(include_examples=True) @@ -328,8 +331,15 @@ def test_show_dag_imgcat(self, mock_render_dag, mock_popen): assert "OUT" in out assert "ERR" in out + @pytest.mark.parametrize( + "cli_arg", + [ + pytest.param("-I", id="short"), + pytest.param("--ignore-first-depends-on-past", id="full"), + ], + ) @mock.patch("airflow.cli.commands.dag_command.DAG.run") - def test_cli_backfill_depends_on_past(self, mock_run): + def test_cli_backfill_deprecated_ignore_first_depends_on_past(self, mock_run, cli_arg: str): """ Test that CLI respects -I argument @@ -345,11 +355,12 @@ def test_cli_backfill_depends_on_past(self, mock_run): "--local", "--start-date", run_date.isoformat(), - "--ignore-first-depends-on-past", + cli_arg, ] dag = self.dagbag.get_dag(dag_id) - dag_command.dag_backfill(self.parser.parse_args(args), dag=dag) + with pytest.warns(RemovedInAirflow3Warning, match="ignore-first-depends-on-past is deprecated"): + dag_command.dag_backfill(self.parser.parse_args(args), dag=dag) mock_run.assert_called_once_with( start_date=run_date, @@ -369,11 +380,16 @@ def test_cli_backfill_depends_on_past(self, mock_run): disable_retry=False, ) + @pytest.mark.parametrize( + "cli_arg", + [ + pytest.param("-B", id="short"), + pytest.param("--run-backwards", id="full"), + ], + ) @mock.patch("airflow.cli.commands.dag_command.DAG.run") - def test_cli_backfill_depends_on_past_backwards(self, mock_run): - """ - Test that CLI respects -B argument and raises on interaction with depends_on_past - """ + def test_cli_backfill_depends_on_past_run_backwards(self, mock_run, cli_arg: str): + """Test that CLI respects -B argument.""" dag_id = "test_depends_on_past" start_date = DEFAULT_DATE + timedelta(days=1) end_date = start_date + timedelta(days=1) @@ -386,8 +402,7 @@ def test_cli_backfill_depends_on_past_backwards(self, mock_run): start_date.isoformat(), "--end-date", end_date.isoformat(), - "--ignore-first-depends-on-past", - "--run-backwards", + cli_arg, ] dag = self.dagbag.get_dag(dag_id) diff --git a/tests/deprecations_ignore.yml b/tests/deprecations_ignore.yml index 6d27f4f5388d6..1c61b4953df40 100644 --- a/tests/deprecations_ignore.yml +++ b/tests/deprecations_ignore.yml @@ -60,11 +60,6 @@ - tests/cli/commands/test_connection_command.py::TestCliImportConnections::test_cli_connections_import_should_load_connections - tests/cli/commands/test_connection_command.py::TestCliImportConnections::test_cli_connections_import_should_not_overwrite_existing_connections - tests/cli/commands/test_connection_command.py::TestCliImportConnections::test_cli_connections_import_should_overwrite_existing_connections -- tests/cli/commands/test_dag_command.py::TestCliDags::test_backfill -- tests/cli/commands/test_dag_command.py::TestCliDags::test_backfill_fails_without_loading_dags -- tests/cli/commands/test_dag_command.py::TestCliDags::test_backfill_with_custom_timetable -- tests/cli/commands/test_dag_command.py::TestCliDags::test_cli_backfill_depends_on_past -- tests/cli/commands/test_dag_command.py::TestCliDags::test_cli_backfill_depends_on_past_backwards - tests/cli/commands/test_kubernetes_command.py::TestGenerateDagYamlCommand::test_generate_dag_yaml - tests/cli/commands/test_task_command.py::TestCliTasks::test_parentdag_downstream_clear - tests/cli/commands/test_task_command.py::TestCliTasks::test_subdag_clear