Skip to content

Commit

Permalink
Fix validation of date-time field in API and Parameter schemas (#29395)
Browse files Browse the repository at this point in the history
The open-api-schema-validator 0.4.3 made RFC3339 validation of
the date-time fields mandatory and this revealed problems in our
test URLs - the '+' was not url-encoded and it was replaced with
space - thus the dates passed were not valid RFC3339 date-time
specifications.

This however revealed one more problem. The RFC3339-validator package
is automatically installed by the open-api-schema-validator, but when
installed, it also adds validation to date-time fields validated by
the Params of ours - for example naive date-time parameters. We
are limiting the supported parameters now to be iso8601-compliant
for backwards compatibility but we deprecate iso8601 in favor of
RFC3339.

This might introduce breaking changes for users who use non-valid
date-time parameters in the API, however we should consider that as a
bugfix, and accidental support, because the date-time schema should
expect RFC3999-formatted date time.

Co-authored-by: Andrey Anshin <Andrey.Anshin@taragol.is>
  • Loading branch information
potiuk and Taragolis authored Feb 7, 2023
1 parent 0d2555b commit a1faa2c
Show file tree
Hide file tree
Showing 9 changed files with 170 additions and 35 deletions.
2 changes: 1 addition & 1 deletion airflow/example_dags/example_params_ui_tutorial.py
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
30 changes: 30 additions & 0 deletions airflow/models/param.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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.
Expand All @@ -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
Expand Down
2 changes: 1 addition & 1 deletion airflow/www/static/js/main.js
Original file line number Diff line number Diff line change
Expand Up @@ -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' });

Expand Down
8 changes: 8 additions & 0 deletions newsfragments/29395.significant.rst
Original file line number Diff line number Diff line change
@@ -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.
3 changes: 2 additions & 1 deletion setup.cfg
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down
57 changes: 38 additions & 19 deletions tests/api_connexion/endpoints/test_dag_run_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
# under the License.
from __future__ import annotations

import urllib
from datetime import timedelta
from unittest import mock

Expand Down Expand Up @@ -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",
Expand All @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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"],
),
],
Expand Down Expand Up @@ -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):
Expand All @@ -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()
Expand Down Expand Up @@ -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):
Expand Down Expand Up @@ -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):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import datetime as dt
import os
import urllib

import pytest

Expand All @@ -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")
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down
26 changes: 16 additions & 10 deletions tests/api_connexion/endpoints/test_task_instance_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
from __future__ import annotations

import datetime as dt
import urllib
from unittest import mock

import pendulum
Expand All @@ -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):
Expand Down Expand Up @@ -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",
Expand All @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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
Expand All @@ -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):
Expand Down
Loading

0 comments on commit a1faa2c

Please sign in to comment.