diff --git a/airflow/example_dags/example_params_ui_tutorial.py b/airflow/example_dags/example_params_ui_tutorial.py index ef9fd568ea83a..df93157c4d5aa 100644 --- a/airflow/example_dags/example_params_ui_tutorial.py +++ b/airflow/example_dags/example_params_ui_tutorial.py @@ -80,7 +80,7 @@ ), # Dates and Times are also supported "date_time": Param( - f"{datetime.date.today()} {datetime.time(hour=12, minute=17, second=00)}", + f"{datetime.date.today()}T{datetime.time(hour=12, minute=17, second=00)}+00:00", type="string", format="date-time", title="Date-Time Picker", diff --git a/airflow/models/param.py b/airflow/models/param.py index 2f93f7dd88341..03f6ff414c147 100644 --- a/airflow/models/param.py +++ b/airflow/models/param.py @@ -18,12 +18,16 @@ import contextlib import copy +import datetime import json import logging import warnings from typing import TYPE_CHECKING, Any, ClassVar, ItemsView, Iterable, MutableMapping, ValuesView +from pendulum.parsing import parse_iso8601 + from airflow.exceptions import AirflowException, ParamValidationError, RemovedInAirflow3Warning +from airflow.utils import timezone from airflow.utils.context import Context from airflow.utils.mixins import ResolveMixin from airflow.utils.types import NOTSET, ArgNotSet @@ -72,6 +76,27 @@ def _warn_if_not_json(value): RemovedInAirflow3Warning, ) + @staticmethod + def _warn_if_not_rfc3339_dt(value): + """Fallback to iso8601 datetime validation if rfc3339 failed.""" + try: + iso8601_value = parse_iso8601(value) + except Exception: + return None + if not isinstance(iso8601_value, datetime.datetime): + return None + warnings.warn( + f"The use of non-RFC3339 datetime: {value!r} is deprecated " + "and will be removed in a future release", + RemovedInAirflow3Warning, + ) + if timezone.is_naive(iso8601_value): + warnings.warn( + "The use naive datetime is deprecated and will be removed in a future release", + RemovedInAirflow3Warning, + ) + return value + def resolve(self, value: Any = NOTSET, suppress_exception: bool = False) -> Any: """ Runs the validations and returns the Param's final value. @@ -98,6 +123,11 @@ def resolve(self, value: Any = NOTSET, suppress_exception: bool = False) -> Any: try: jsonschema.validate(final_val, self.schema, format_checker=FormatChecker()) except ValidationError as err: + if err.schema.get("format") == "date-time": + rfc3339_value = self._warn_if_not_rfc3339_dt(final_val) + if rfc3339_value: + self.value = rfc3339_value + return rfc3339_value if suppress_exception: return None raise ParamValidationError(err) from None diff --git a/airflow/www/static/js/main.js b/airflow/www/static/js/main.js index c12ca1f880699..5c73682fc7d9b 100644 --- a/airflow/www/static/js/main.js +++ b/airflow/www/static/js/main.js @@ -252,7 +252,7 @@ $(document).ready(() => { }); $.fn.datetimepicker.defaults.sideBySide = true; - $('.datetimepicker').datetimepicker({ format: 'YYYY-MM-DD HH:mm:ssZ' }); + $('.datetimepicker').datetimepicker({ format: 'YYYY-MM-DDTHH:mm:ssZ' }); $('.datepicker').datetimepicker({ format: 'YYYY-MM-DD' }); $('.timepicker').datetimepicker({ format: 'HH:mm:ss' }); diff --git a/newsfragments/29395.significant.rst b/newsfragments/29395.significant.rst new file mode 100644 index 0000000000000..14f094397dc71 --- /dev/null +++ b/newsfragments/29395.significant.rst @@ -0,0 +1,8 @@ +The date-time fields passed as API parameters or Params should be RFC3339-compliant. + +In case of API calls, it was possible that "+" passed as part of the date-time fields were not URL-encoded, and +such date-time fields could pass validation. Such date-time parameters should now be URL-encoded (as ``%2B``). + +In case of parameters, we still allow IS8601-compliant date-time (so for example it is possible that +' ' was used instead of ``T`` separating date from time and no timezone was specified) but we raise +deprecation warning. diff --git a/setup.cfg b/setup.cfg index 4a499dccac2c3..fb1ae6c04baea 100644 --- a/setup.cfg +++ b/setup.cfg @@ -103,7 +103,7 @@ install_requires = importlib_resources>=5.2;python_version<"3.9" itsdangerous>=2.0 jinja2>=3.0.0 - jsonschema>=3.2.0 + jsonschema>=4.0.0 lazy-object-proxy linkify-it-py>=2.0.0 lockfile>=0.12.2 @@ -123,6 +123,7 @@ install_requires = python-dateutil>=2.3 python-nvd3>=0.15.0 python-slugify>=5.0 + rfc3339_validator>=0.1.4 rich>=12.4.4 setproctitle>=1.1.8 # We use some deprecated features of sqlalchemy 2.0 and we should replace them before we can upgrade diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index f6012c39480a7..a7f7fe7bcae8b 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import urllib from datetime import timedelta from unittest import mock @@ -512,15 +513,15 @@ class TestGetDagRunsPaginationFilters(TestDagRunEndpoint): "url, expected_dag_run_ids", [ ( - "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_gte=2020-06-18T18:00:00+00:00", + "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_gte=2020-06-18T18%3A00%3A00%2B00%3A00", ["TEST_START_EXEC_DAY_18", "TEST_START_EXEC_DAY_19"], ), ( - "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_lte=2020-06-11T18:00:00+00:00", + "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_lte=2020-06-11T18%3A00%3A00%2B00%3A00", ["TEST_START_EXEC_DAY_10", "TEST_START_EXEC_DAY_11"], ), ( - "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_lte= 2020-06-15T18:00:00+00:00" + "api/v1/dags/TEST_DAG_ID/dagRuns?start_date_lte=2020-06-15T18%3A00%3A00%2B00%3A00" "&start_date_gte=2020-06-12T18:00:00Z", [ "TEST_START_EXEC_DAY_12", @@ -530,7 +531,7 @@ class TestGetDagRunsPaginationFilters(TestDagRunEndpoint): ], ), ( - "api/v1/dags/TEST_DAG_ID/dagRuns?execution_date_lte=2020-06-13T18:00:00+00:00", + "api/v1/dags/TEST_DAG_ID/dagRuns?execution_date_lte=2020-06-13T18%3A00%3A00%2B00%3A00", [ "TEST_START_EXEC_DAY_10", "TEST_START_EXEC_DAY_11", @@ -539,7 +540,7 @@ class TestGetDagRunsPaginationFilters(TestDagRunEndpoint): ], ), ( - "api/v1/dags/TEST_DAG_ID/dagRuns?execution_date_gte=2020-06-16T18:00:00+00:00", + "api/v1/dags/TEST_DAG_ID/dagRuns?execution_date_gte=2020-06-16T18%3A00%3A00%2B00%3A00", [ "TEST_START_EXEC_DAY_16", "TEST_START_EXEC_DAY_17", @@ -595,12 +596,12 @@ class TestGetDagRunsEndDateFilters(TestDagRunEndpoint): [ ( f"api/v1/dags/TEST_DAG_ID/dagRuns?end_date_gte=" - f"{(timezone.utcnow() + timedelta(days=1)).isoformat()}", + f"{urllib.parse.quote((timezone.utcnow() + timedelta(days=1)).isoformat())}", [], ), ( f"api/v1/dags/TEST_DAG_ID/dagRuns?end_date_lte=" - f"{(timezone.utcnow() + timedelta(days=1)).isoformat()}", + f"{urllib.parse.quote((timezone.utcnow() + timedelta(days=1)).isoformat())}", ["TEST_DAG_RUN_ID_1", "TEST_DAG_RUN_ID_2"], ), ], @@ -786,7 +787,7 @@ def test_should_return_accessible_with_tilde_as_dag_id_and_dag_level_permissions ), ({"dag_ids": ["TEST_DAG_ID"], "page_limit": 0}, "0 is less than the minimum of 1 - 'page_limit'"), ({"dag_ids": "TEST_DAG_ID"}, "'TEST_DAG_ID' is not of type 'array' - 'dag_ids'"), - ({"start_date_gte": "2020-06-12T18"}, "{'start_date_gte': ['Not a valid datetime.']}"), + ({"start_date_gte": "2020-06-12T18"}, "'2020-06-12T18' is not a 'date-time' - 'start_date_gte'"), ], ) def test_payload_validation(self, payload, error): @@ -795,7 +796,7 @@ def test_payload_validation(self, payload, error): "api/v1/dags/~/dagRuns/list", json=payload, environ_overrides={"REMOTE_USER": "test"} ) assert response.status_code == 400 - assert error == response.json.get("detail") + assert response.json.get("detail") == error def test_should_raises_401_unauthenticated(self): self._create_test_dag_run() @@ -970,24 +971,30 @@ def _create_dag_runs(self): @pytest.mark.parametrize( "payload, expected_response", [ - ({"execution_date_gte": "2020-11-09T16:25:56.939143"}, "Naive datetime is disallowed"), + ( + {"execution_date_gte": "2020-11-09T16:25:56.939143"}, + "'2020-11-09T16:25:56.939143' is not a 'date-time' - 'execution_date_gte'", + ), ( {"start_date_gte": "2020-06-18T16:25:56.939143"}, - "Naive datetime is disallowed", + "'2020-06-18T16:25:56.939143' is not a 'date-time' - 'start_date_gte'", ), ( {"start_date_lte": "2020-06-18T18:00:00.564434"}, - "Naive datetime is disallowed", + "'2020-06-18T18:00:00.564434' is not a 'date-time' - 'start_date_lte'", ), ( {"start_date_lte": "2020-06-15T18:00:00.653434", "start_date_gte": "2020-06-12T18:00.343534"}, - "Naive datetime is disallowed", + "'2020-06-12T18:00.343534' is not a 'date-time' - 'start_date_gte'", ), ( {"execution_date_lte": "2020-06-13T18:00:00.353454"}, - "Naive datetime is disallowed", + "'2020-06-13T18:00:00.353454' is not a 'date-time' - 'execution_date_lte'", + ), + ( + {"execution_date_gte": "2020-06-16T18:00:00.676443"}, + "'2020-06-16T18:00:00.676443' is not a 'date-time' - 'execution_date_gte'", ), - ({"execution_date_gte": "2020-06-16T18:00:00.676443"}, "Naive datetime is disallowed"), ], ) def test_naive_date_filters_raises_400(self, payload, expected_response): @@ -1146,10 +1153,22 @@ def test_should_response_400_for_conflicting_execution_date_logical_date(self): @pytest.mark.parametrize( "data, expected", [ - ({"execution_date": "2020-11-10T08:25:56.939143"}, "Naive datetime is disallowed"), - ({"execution_date": "2020-11-10T08:25:56P"}, "{'logical_date': ['Not a valid datetime.']}"), - ({"logical_date": "2020-11-10T08:25:56.939143"}, "Naive datetime is disallowed"), - ({"logical_date": "2020-11-10T08:25:56P"}, "{'logical_date': ['Not a valid datetime.']}"), + ( + {"execution_date": "2020-11-10T08:25:56.939143"}, + "'2020-11-10T08:25:56.939143' is not a 'date-time' - 'execution_date'", + ), + ( + {"execution_date": "2020-11-10T08:25:56P"}, + "'2020-11-10T08:25:56P' is not a 'date-time' - 'execution_date'", + ), + ( + {"logical_date": "2020-11-10T08:25:56.939143"}, + "'2020-11-10T08:25:56.939143' is not a 'date-time' - 'logical_date'", + ), + ( + {"logical_date": "2020-11-10T08:25:56P"}, + "'2020-11-10T08:25:56P' is not a 'date-time' - 'logical_date'", + ), ], ) def test_should_response_400_for_naive_datetime_and_bad_datetime(self, data, expected): diff --git a/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py b/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py index cfb141e09b018..4903a88e56d3b 100644 --- a/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py +++ b/tests/api_connexion/endpoints/test_mapped_task_instance_endpoint.py @@ -18,6 +18,7 @@ import datetime as dt import os +import urllib import pytest @@ -38,6 +39,8 @@ DEFAULT_DATETIME_1 = datetime(2020, 1, 1) DEFAULT_DATETIME_STR_1 = "2020-01-01T00:00:00+00:00" DEFAULT_DATETIME_STR_2 = "2020-01-02T00:00:00+00:00" +QUOTED_DEFAULT_DATETIME_STR_1 = urllib.parse.quote(DEFAULT_DATETIME_STR_1) +QUOTED_DEFAULT_DATETIME_STR_2 = urllib.parse.quote(DEFAULT_DATETIME_STR_2) @pytest.fixture(scope="module") @@ -368,7 +371,7 @@ def test_mapped_task_instances_invalid_order(self, one_task_with_many_mapped_tis def test_mapped_task_instances_with_date(self, one_task_with_mapped_tis, session): response = self.client.get( "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - f"?start_date_gte={DEFAULT_DATETIME_STR_1}", + f"?start_date_gte={QUOTED_DEFAULT_DATETIME_STR_1}", environ_overrides={"REMOTE_USER": "test"}, ) assert response.status_code == 200 @@ -377,7 +380,7 @@ def test_mapped_task_instances_with_date(self, one_task_with_mapped_tis, session response = self.client.get( "/api/v1/dags/mapped_tis/dagRuns/run_mapped_tis/taskInstances/task_2/listMapped" - f"?start_date_gte={DEFAULT_DATETIME_STR_2}", + f"?start_date_gte={QUOTED_DEFAULT_DATETIME_STR_2}", environ_overrides={"REMOTE_USER": "test"}, ) assert response.status_code == 200 diff --git a/tests/api_connexion/endpoints/test_task_instance_endpoint.py b/tests/api_connexion/endpoints/test_task_instance_endpoint.py index b9d5e5a4e97d1..51b0e43db5979 100644 --- a/tests/api_connexion/endpoints/test_task_instance_endpoint.py +++ b/tests/api_connexion/endpoints/test_task_instance_endpoint.py @@ -17,6 +17,7 @@ from __future__ import annotations import datetime as dt +import urllib from unittest import mock import pendulum @@ -39,6 +40,9 @@ DEFAULT_DATETIME_STR_1 = "2020-01-01T00:00:00+00:00" DEFAULT_DATETIME_STR_2 = "2020-01-02T00:00:00+00:00" +QUOTED_DEFAULT_DATETIME_STR_1 = urllib.parse.quote(DEFAULT_DATETIME_STR_1) +QUOTED_DEFAULT_DATETIME_STR_2 = urllib.parse.quote(DEFAULT_DATETIME_STR_2) + @pytest.fixture(scope="module") def configured_app(minimal_app_for_api): @@ -480,7 +484,7 @@ class TestGetTaskInstances(TestTaskInstanceEndpoint): False, ( "/api/v1/dags/example_python_operator/dagRuns/~/" - f"taskInstances?execution_date_lte={DEFAULT_DATETIME_STR_1}" + f"taskInstances?execution_date_lte={QUOTED_DEFAULT_DATETIME_STR_1}" ), 1, id="test execution date filter", @@ -494,7 +498,8 @@ class TestGetTaskInstances(TestTaskInstanceEndpoint): True, ( "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances" - f"?start_date_gte={DEFAULT_DATETIME_STR_1}&start_date_lte={DEFAULT_DATETIME_STR_2}" + f"?start_date_gte={QUOTED_DEFAULT_DATETIME_STR_1}&" + f"start_date_lte={QUOTED_DEFAULT_DATETIME_STR_2}" ), 2, id="test start date filter", @@ -508,7 +513,8 @@ class TestGetTaskInstances(TestTaskInstanceEndpoint): True, ( "/api/v1/dags/example_python_operator/dagRuns/~/taskInstances?" - f"end_date_gte={DEFAULT_DATETIME_STR_1}&end_date_lte={DEFAULT_DATETIME_STR_2}" + f"end_date_gte={QUOTED_DEFAULT_DATETIME_STR_1}&" + f"end_date_lte={QUOTED_DEFAULT_DATETIME_STR_2}" ), 2, id="test end date filter", @@ -870,12 +876,12 @@ def test_should_raise_403_forbidden(self): @pytest.mark.parametrize( "payload, expected", [ - ({"end_date_lte": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"end_date_gte": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"start_date_lte": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"start_date_gte": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"execution_date_gte": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), - ({"execution_date_lte": "2020-11-10T12:42:39.442973"}, "Naive datetime is disallowed"), + ({"end_date_lte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), + ({"end_date_gte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), + ({"start_date_lte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), + ({"start_date_gte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), + ({"execution_date_gte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), + ({"execution_date_lte": "2020-11-10T12:42:39.442973"}, "is not a 'date-time'"), ], ) @provide_session @@ -887,7 +893,7 @@ def test_should_raise_400_for_naive_and_bad_datetime(self, payload, expected, se json=payload, ) assert response.status_code == 400 - assert response.json["detail"] == expected + assert expected in response.json["detail"] class TestPostClearTaskInstances(TestTaskInstanceEndpoint): diff --git a/tests/models/test_param.py b/tests/models/test_param.py index 7b7b2c7bf9079..bb5e9b2c44b15 100644 --- a/tests/models/test_param.py +++ b/tests/models/test_param.py @@ -21,7 +21,7 @@ import pytest from airflow.decorators import task -from airflow.exceptions import ParamValidationError +from airflow.exceptions import ParamValidationError, RemovedInAirflow3Warning from airflow.models.param import Param, ParamsDict from airflow.utils import timezone from airflow.utils.types import DagRunType @@ -68,6 +68,74 @@ def test_string_param(self): with pytest.raises(ParamValidationError, match="No value passed and Param has no default value"): p.resolve() + @pytest.mark.parametrize( + "dt", + [ + pytest.param("2022-01-02T03:04:05.678901Z", id="microseconds-zed-timezone"), + pytest.param("2022-01-02T03:04:05.678Z", id="milliseconds-zed-timezone"), + pytest.param("2022-01-02T03:04:05+00:00", id="seconds-00-00-timezone"), + pytest.param("2022-01-02T03:04:05+04:00", id="seconds-custom-timezone"), + ], + ) + def test_string_rfc3339_datetime_format(self, dt): + """Test valid rfc3339 datetime.""" + assert Param(dt, type="string", format="date-time").resolve() == dt + + @pytest.mark.parametrize( + "dt", + [ + pytest.param("2022-01-02 03:04:05.678901Z", id="space-sep"), + pytest.param("2022-01-02T03:04:05.678901", id="tz-naive"), + pytest.param("2022-01-02T03Z", id="datetime-with-day-only"), + pytest.param("20161001T143028+0530", id="not-formatted-date-time"), + ], + ) + def test_string_iso8601_datetime_invalid_rfc3339_format(self, dt): + """Test valid iso8601 datetime but not valid rfc3339 datetime conversion.""" + with pytest.warns(RemovedInAirflow3Warning): + assert Param(dt, type="string", format="date-time").resolve() == dt + + @pytest.mark.parametrize( + "dt", + [ + pytest.param("2022-01-02", id="date"), + pytest.param("03:04:05", id="time"), + pytest.param("Thu, 04 Mar 2021 05:06:07 GMT", id="rfc2822-datetime"), + ], + ) + def test_string_datetime_invalid_format(self, dt): + """Test invalid iso8601 and rfc3339 datetime format.""" + with pytest.raises(ParamValidationError, match="is not a 'date-time'"): + Param(dt, type="string", format="date-time").resolve() + + def test_string_time_format(self): + """Test string time format.""" + assert Param("03:04:05", type="string", format="time").resolve() == "03:04:05" + + error_pattern = "is not a 'time'" + with pytest.raises(ParamValidationError, match=error_pattern): + Param("03:04:05.06", type="string", format="time").resolve() + + with pytest.raises(ParamValidationError, match=error_pattern): + Param("03:04", type="string", format="time").resolve() + + with pytest.raises(ParamValidationError, match=error_pattern): + Param("24:00:00", type="string", format="time").resolve() + + def test_string_date_format(self): + """Test string date format.""" + assert Param("2021-01-01", type="string", format="date").resolve() == "2021-01-01" + + error_pattern = "is not a 'date'" + with pytest.raises(ParamValidationError, match=error_pattern): + Param("01/01/2021", type="string", format="date").resolve() + + with pytest.raises(ParamValidationError, match=error_pattern): + Param("20120503", type="string", format="date").resolve() + + with pytest.raises(ParamValidationError, match=error_pattern): + Param("21 May 1975", type="string", format="date").resolve() + def test_int_param(self): p = Param(5) assert p.resolve() == 5