From b606b96e5053c42c75f31fc8f6b48cd6aba72452 Mon Sep 17 00:00:00 2001 From: Andrey Anshin Date: Thu, 4 Apr 2024 20:05:39 +0400 Subject: [PATCH] Respect `@pytest.mark.filterwarnings` in prohibited warnings (#38684) * Respect `@pytest.mark.filterwarnings` in prohibited warnings * Extend list to ignore --- .../common/test_mark_tasks.py | 18 +++++-- tests/conftest.py | 51 +++++++++++-------- tests/deprecations_ignore.yml | 29 +++++++++++ tests/jobs/test_backfill_job.py | 8 ++- .../experimental/test_dag_runs_endpoint.py | 8 ++- tests/www/views/conftest.py | 12 +++-- 6 files changed, 95 insertions(+), 31 deletions(-) diff --git a/tests/api_experimental/common/test_mark_tasks.py b/tests/api_experimental/common/test_mark_tasks.py index 9b28136bba279..0fd42f0e2375f 100644 --- a/tests/api_experimental/common/test_mark_tasks.py +++ b/tests/api_experimental/common/test_mark_tasks.py @@ -18,6 +18,7 @@ from __future__ import annotations import datetime +import warnings from typing import Callable import pytest @@ -33,6 +34,7 @@ set_dag_run_state_to_success, set_state, ) +from airflow.exceptions import RemovedInAirflow3Warning from airflow.models import DagRun from airflow.utils import timezone from airflow.utils.session import create_session, provide_session @@ -50,9 +52,13 @@ def dagbag(): from airflow.models.dagbag import DagBag - # Ensure the DAGs we are looking at from the DB are up-to-date - non_serialized_dagbag = DagBag(read_dags_from_db=False, include_examples=True) - non_serialized_dagbag.sync_to_db() + with warnings.catch_warnings(): + # Some dags use deprecated operators, e.g SubDagOperator + # if it is not imported, then it might have side effects for the other tests + warnings.simplefilter("ignore", category=RemovedInAirflow3Warning) + # Ensure the DAGs we are looking at from the DB are up-to-date + non_serialized_dagbag = DagBag(read_dags_from_db=False, include_examples=True) + non_serialized_dagbag.sync_to_db() return DagBag(read_dags_from_db=True) @@ -478,7 +484,11 @@ class TestMarkDAGRun: @classmethod def setup_class(cls): - dagbag = models.DagBag(include_examples=True, read_dags_from_db=False) + with warnings.catch_warnings(): + # Some dags use deprecated operators, e.g SubDagOperator + # if it is not imported, then it might have side effects for the other tests + warnings.simplefilter("ignore", category=RemovedInAirflow3Warning) + dagbag = models.DagBag(include_examples=True, read_dags_from_db=False) cls.dag1 = dagbag.dags["miscellaneous_test_dag"] cls.dag1.sync_to_db() cls.dag2 = dagbag.dags["example_subdag_operator"] diff --git a/tests/conftest.py b/tests/conftest.py index 52e701e8cec9c..b3efb02fa954d 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import functools import json import os import platform @@ -586,6 +587,34 @@ def skip_if_credential_file_missing(item): pytest.skip(f"The test requires credential file {credential_path}: {item}") +@functools.lru_cache(maxsize=None) +def deprecations_ignore() -> tuple[str, ...]: + with open(Path(__file__).resolve().parent / "deprecations_ignore.yml") as fp: + return tuple(yaml.safe_load(fp)) + + +def setup_error_warnings(item: pytest.Item): + if item.nodeid.startswith(deprecations_ignore()): + return + + # We cannot add everything related to the airflow package it into `filterwarnings` + # in the pyproject.toml sections, because it invokes airflow import before we setup test environment. + # Instead of that, we are dynamically adding as `filterwarnings` marker. + prohibited_warnings = ( + "airflow.exceptions.RemovedInAirflow3Warning", + "airflow.utils.context.AirflowContextDeprecationWarning", + "airflow.exceptions.AirflowProviderDeprecationWarning", + ) + for w in prohibited_warnings: + # Add marker at the beginning of the markers list. In this case, it does not conflict with + # filterwarnings markers, which are set explicitly in the test suite. + item.add_marker(pytest.mark.filterwarnings(f"error::{w}"), append=False) + + +def pytest_itemcollected(item: pytest.Item): + setup_error_warnings(item) + + def pytest_runtest_setup(item): selected_integrations_list = item.config.option.integration selected_systems_list = item.config.option.system @@ -1205,28 +1234,6 @@ def cleanup_providers_manager(): ProvidersManager()._cleanup() -@pytest.fixture(scope="session") -def deprecations_ignore() -> tuple[str, ...]: - with open(Path(__file__).absolute().parent.resolve() / "deprecations_ignore.yml") as fp: - return tuple(yaml.safe_load(fp)) - - -@pytest.fixture(autouse=True) -def check_deprecations(request: pytest.FixtureRequest, deprecations_ignore): - from airflow.exceptions import AirflowProviderDeprecationWarning, RemovedInAirflow3Warning - from airflow.utils.context import AirflowContextDeprecationWarning - - if request.node.nodeid.startswith(deprecations_ignore): - yield - return - - with warnings.catch_warnings(): - warnings.simplefilter("error", AirflowProviderDeprecationWarning) - warnings.simplefilter("error", RemovedInAirflow3Warning) - warnings.simplefilter("error", AirflowContextDeprecationWarning) - yield - - # The code below is a modified version of capture-warning code from # https://github.com/athinkingape/pytest-capture-warnings diff --git a/tests/deprecations_ignore.yml b/tests/deprecations_ignore.yml index 1c61b4953df40..b5ca3a4c891d2 100644 --- a/tests/deprecations_ignore.yml +++ b/tests/deprecations_ignore.yml @@ -337,6 +337,29 @@ # WWW +- tests/www/api/experimental/test_dag_runs_endpoint.py::TestDagRunsEndpoint::test_get_dag_runs_invalid_dag_id +- tests/www/api/experimental/test_dag_runs_endpoint.py::TestDagRunsEndpoint::test_get_dag_runs_no_runs +- tests/www/api/experimental/test_dag_runs_endpoint.py::TestDagRunsEndpoint::test_get_dag_runs_success +- tests/www/api/experimental/test_dag_runs_endpoint.py::TestDagRunsEndpoint::test_get_dag_runs_success_with_capital_state_parameter +- tests/www/api/experimental/test_dag_runs_endpoint.py::TestDagRunsEndpoint::test_get_dag_runs_success_with_state_no_result +- tests/www/api/experimental/test_dag_runs_endpoint.py::TestDagRunsEndpoint::test_get_dag_runs_success_with_state_parameter +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_dag_paused +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_dagrun_status +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_get_dag_code +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_info +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_task_info +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_task_instance_info +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_trigger_dag +- tests/www/api/experimental/test_endpoints.py::TestApiExperimental::test_trigger_dag_for_date +- tests/www/api/experimental/test_endpoints.py::TestLineageApiExperimental::test_lineage_info +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_create_pool +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_create_pool_with_bad_name +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_delete_default_pool +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_delete_pool +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_delete_pool_non_existing +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_get_pool +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_get_pool_non_existing +- tests/www/api/experimental/test_endpoints.py::TestPoolApiExperimental::test_get_pools - tests/www/test_utils.py::test_dag_run_custom_sqla_interface_delete_no_collateral_damage - tests/www/views/test_views_acl.py::test_success - tests/www/views/test_views_cluster_activity.py::test_historical_metrics_data @@ -675,6 +698,12 @@ - tests/providers/databricks/sensors/test_databricks_sql.py::TestDatabricksSqlSensor::test_unsupported_conn_type - tests/providers/docker/operators/test_docker.py::test_hook_usage - tests/providers/elasticsearch/log/test_es_task_handler.py::test_retrieve_retry_on_timeout +- tests/providers/google/common/auth_backend/test_google_openid.py::TestGoogleOpenID::test_invalid_id_token +- tests/providers/google/common/auth_backend/test_google_openid.py::TestGoogleOpenID::test_invalid_iss_in_jwt_token +- tests/providers/google/common/auth_backend/test_google_openid.py::TestGoogleOpenID::test_malformed_headers +- tests/providers/google/common/auth_backend/test_google_openid.py::TestGoogleOpenID::test_missing_id_token +- tests/providers/google/common/auth_backend/test_google_openid.py::TestGoogleOpenID::test_success +- tests/providers/google/common/auth_backend/test_google_openid.py::TestGoogleOpenID::test_user_not_exists - tests/providers/google/cloud/hooks/test_automl.py::TestAutoMLHook::test_batch_predict - tests/providers/google/cloud/hooks/test_automl.py::TestAutoMLHook::test_create_dataset - tests/providers/google/cloud/hooks/test_automl.py::TestAutoMLHook::test_create_model diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index ad80166147c52..7d445a7286b63 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -21,6 +21,7 @@ import json import logging import threading +import warnings from collections import defaultdict from unittest import mock from unittest.mock import patch @@ -35,6 +36,7 @@ BackfillUnfinished, DagConcurrencyLimitReached, NoAvailablePoolSlot, + RemovedInAirflow3Warning, TaskConcurrencyLimitReached, ) from airflow.executors.executor_constants import MOCK_EXECUTOR @@ -75,7 +77,11 @@ @pytest.fixture(scope="module") def dag_bag(): - return DagBag(include_examples=True) + with warnings.catch_warnings(): + # Some dags use deprecated operators, e.g SubDagOperator + # if it is not imported, then it might have side effects for the other tests + warnings.simplefilter("ignore", category=RemovedInAirflow3Warning) + return DagBag(include_examples=True) # Patch the MockExecutor into the dict of known executors in the Loader diff --git a/tests/www/api/experimental/test_dag_runs_endpoint.py b/tests/www/api/experimental/test_dag_runs_endpoint.py index 645f3614e57fb..1e4e964f49fa5 100644 --- a/tests/www/api/experimental/test_dag_runs_endpoint.py +++ b/tests/www/api/experimental/test_dag_runs_endpoint.py @@ -18,10 +18,12 @@ from __future__ import annotations import json +import warnings import pytest from airflow.api.common.trigger_dag import trigger_dag +from airflow.exceptions import RemovedInAirflow3Warning from airflow.models import DagBag, DagRun from airflow.models.serialized_dag import SerializedDagModel from airflow.settings import Session @@ -36,7 +38,11 @@ def _setup_session(self): session.query(DagRun).delete() session.commit() session.close() - dagbag = DagBag(include_examples=True) + with warnings.catch_warnings(): + # Some dags use deprecated operators, e.g SubDagOperator + # if it is not imported, then it might have side effects for the other tests + warnings.simplefilter("ignore", category=RemovedInAirflow3Warning) + dagbag = DagBag(include_examples=True) for dag in dagbag.dags.values(): dag.sync_to_db() SerializedDagModel.write_dag(dag) diff --git a/tests/www/views/conftest.py b/tests/www/views/conftest.py index 821f541ef0c43..b27e50763959e 100644 --- a/tests/www/views/conftest.py +++ b/tests/www/views/conftest.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import warnings from contextlib import contextmanager from typing import Any, Generator, NamedTuple @@ -25,6 +26,7 @@ import pytest from airflow import settings +from airflow.exceptions import RemovedInAirflow3Warning from airflow.models import DagBag from airflow.www.app import create_app from tests.test_utils.api_connexion_utils import delete_user @@ -41,9 +43,13 @@ def session(): @pytest.fixture(autouse=True, scope="module") def examples_dag_bag(session): - DagBag(include_examples=True).sync_to_db() - dag_bag = DagBag(include_examples=True, read_dags_from_db=True) - session.commit() + with warnings.catch_warnings(): + # Some dags use deprecated operators, e.g SubDagOperator + # if it is not imported, then it might have side effects for the other tests + warnings.simplefilter("ignore", category=RemovedInAirflow3Warning) + DagBag(include_examples=True).sync_to_db() + dag_bag = DagBag(include_examples=True, read_dags_from_db=True) + session.commit() return dag_bag