Skip to content

Commit

Permalink
Disable most irrelevant "core" tests in database isolation mode (#41266)
Browse files Browse the repository at this point in the history
A lot of the "core" tests are not relevant for database isolation
mode and they should be skipped.

Related: #41067
  • Loading branch information
potiuk authored Aug 5, 2024
1 parent a7353d2 commit 7c14f20
Show file tree
Hide file tree
Showing 46 changed files with 61 additions and 40 deletions.
2 changes: 1 addition & 1 deletion tests/core/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
from airflow.utils.types import DagRunType
from tests.test_utils.db import clear_db_dags, clear_db_runs, clear_db_task_fail

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]

DEFAULT_DATE = datetime(2015, 1, 1)

Expand Down
2 changes: 2 additions & 0 deletions tests/core/test_sentry.py
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,7 @@ def sentry_minimum(self):

importlib.reload(sentry)

@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.db_test
def test_add_tagging(self, sentry, task_instance):
"""
Expand All @@ -151,6 +152,7 @@ def test_add_tagging(self, sentry, task_instance):
for key, value in scope._tags.items():
assert value == TEST_SCOPE[key]

@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.db_test
@time_machine.travel(CRUMB_DATE)
def test_add_breadcrumbs(self, sentry, task_instance):
Expand Down
2 changes: 1 addition & 1 deletion tests/core/test_sqlalchemy_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@

SQL_ALCHEMY_CONNECT_ARGS = {"test": 43503, "dict": {"is": 1, "supported": "too"}}

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestSqlAlchemySettings:
Expand Down
2 changes: 2 additions & 0 deletions tests/executors/test_base_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,8 @@
from airflow.utils import timezone
from airflow.utils.state import State, TaskInstanceState

pytestmark = pytest.mark.skip_if_database_isolation_mode


def test_supports_sentry():
assert not BaseExecutor.supports_sentry
Expand Down
2 changes: 2 additions & 0 deletions tests/executors/test_executor_loader.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
from airflow.providers.celery.executors.celery_executor import CeleryExecutor
from tests.test_utils.config import conf_vars

pytestmark = pytest.mark.skip_if_database_isolation_mode

# Plugin Manager creates new modules, which is difficult to mock, so we use test isolation by a unique name.
TEST_PLUGIN_NAME = "unique_plugin_name_to_avoid_collision_i_love_kitties"

Expand Down
2 changes: 1 addition & 1 deletion tests/executors/test_local_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from airflow.executors.local_executor import LocalExecutor
from airflow.utils.state import State

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestLocalExecutor:
Expand Down
2 changes: 1 addition & 1 deletion tests/jobs/test_scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@
from tests.test_utils.mock_operators import CustomOperator
from tests.utils.test_timezone import UTC

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]

ROOT_FOLDER = os.path.realpath(
os.path.join(os.path.dirname(os.path.realpath(__file__)), os.pardir, os.pardir)
Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_base.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
from airflow.models.base import get_id_collation_args
from tests.test_utils.config import conf_vars

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


@pytest.mark.parametrize(
Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_cleartasks.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
from tests.models import DEFAULT_DATE
from tests.test_utils import db

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestClearTasks:
Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@
if TYPE_CHECKING:
from sqlalchemy.orm import Session

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]

TEST_DATE = datetime_tz(2015, 1, 2, 0, 0)

Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_dagcode.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
from airflow.utils.session import create_session
from tests.test_utils.db import clear_db_dag_code

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


def make_example_dags(module):
Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_dagrun.py
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@
from tests.test_utils.config import conf_vars
from tests.test_utils.mock_operators import MockOperator

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


if TYPE_CHECKING:
Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_dagwarning.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
from airflow.models.dagwarning import DagWarning
from tests.test_utils.db import clear_db_dags

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDagWarning:
Expand Down
2 changes: 1 addition & 1 deletion tests/models/test_pool.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
from airflow.utils.state import State
from tests.test_utils.db import clear_db_dags, clear_db_pools, clear_db_runs, set_default_pool_slots

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]

DEFAULT_DATE = timezone.datetime(2016, 1, 1)

Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_dag_ti_slots_available_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
from airflow.models import TaskInstance
from airflow.ti_deps.deps.dag_ti_slots_available_dep import DagTISlotsAvailableDep

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDagTISlotsAvailableDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_dag_unpaused_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
from airflow.models import TaskInstance
from airflow.ti_deps.deps.dag_unpaused_dep import DagUnpausedDep

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDagUnpausedDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_dagrun_exists_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
from airflow.ti_deps.deps.dagrun_exists_dep import DagrunRunningDep
from airflow.utils.state import State

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDagrunRunningDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_dagrun_id_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
from airflow.ti_deps.deps.dagrun_backfill_dep import DagRunNotBackfillDep
from airflow.utils.types import DagRunType

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDagrunRunningDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_mapped_task_upstream_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from airflow.ti_deps.deps.mapped_task_upstream_dep import MappedTaskUpstreamDep
from airflow.utils.state import TaskInstanceState

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]

if TYPE_CHECKING:
from sqlalchemy.orm.session import Session
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_not_in_retry_period_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from airflow.utils.state import State
from airflow.utils.timezone import datetime

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestNotInRetryPeriodDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_not_previously_skipped_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from airflow.utils.state import State
from airflow.utils.types import DagRunType

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


@pytest.fixture(autouse=True)
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_pool_slots_available_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from airflow.utils.state import TaskInstanceState
from tests.test_utils import db

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestPoolSlotsAvailableDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_prev_dagrun_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
from airflow.utils.types import DagRunType
from tests.test_utils.db import clear_db_runs

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestPrevDagrunDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_ready_to_reschedule_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
from airflow.utils.state import State
from tests.test_utils import db

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


DEFAULT_DATE = timezone.datetime(2016, 1, 1)
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_runnable_exec_date_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from airflow.utils.timezone import datetime
from airflow.utils.types import DagRunType

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


@pytest.fixture(autouse=True)
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_task_concurrency.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
from airflow.ti_deps.dep_context import DepContext
from airflow.ti_deps.deps.task_concurrency_dep import TaskConcurrencyDep

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestTaskConcurrencyDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_task_not_running_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
from airflow.ti_deps.deps.task_not_running_dep import TaskNotRunningDep
from airflow.utils.state import State

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestTaskNotRunningDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_trigger_rule_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
from airflow.utils.state import DagRunState, TaskInstanceState
from airflow.utils.trigger_rule import TriggerRule

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]

if TYPE_CHECKING:
from sqlalchemy.orm.session import Session
Expand Down
2 changes: 1 addition & 1 deletion tests/ti_deps/deps/test_valid_state_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
from airflow.ti_deps.deps.valid_state_dep import ValidStateDep
from airflow.utils.state import State

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestValidStateDep:
Expand Down
2 changes: 1 addition & 1 deletion tests/utils/log/test_colored_log.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
from airflow.configuration import conf
from airflow.utils.log.colored_log import CustomTTYColoredFormatter

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


@patch("airflow.utils.log.timezone_aware.TimezoneAware.formatTime")
Expand Down
2 changes: 1 addition & 1 deletion tests/utils/log/test_log_reader.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@
from tests.test_utils.config import conf_vars
from tests.test_utils.db import clear_db_dags, clear_db_runs

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


if TYPE_CHECKING:
Expand Down
7 changes: 7 additions & 0 deletions tests/utils/log/test_task_context_logger.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@

logger = logging.getLogger(__name__)

pytestmark = pytest.mark.skip_if_database_isolation_mode


@pytest.fixture
def mock_handler():
Expand Down Expand Up @@ -66,6 +68,7 @@ def test_task_handler_not_supports_task_context_logging(mock_handler, supported)
assert t.enabled is supported


@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.db_test
@pytest.mark.parametrize("supported", [True, False])
def test_task_context_log_with_correct_arguments(ti, mock_handler, supported):
Expand All @@ -80,6 +83,7 @@ def test_task_context_log_with_correct_arguments(ti, mock_handler, supported):
mock_handler.emit.assert_not_called()


@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.db_test
@mock.patch("airflow.utils.log.task_context_logger._ensure_ti")
@pytest.mark.parametrize("supported", [True, False])
Expand All @@ -97,13 +101,16 @@ def test_task_context_log_with_task_instance_key(mock_ensure_ti, ti, mock_handle
mock_handler.emit.assert_not_called()


@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.db_test
def test_task_context_log_closes_task_handler(ti, mock_handler):
t = TaskContextLogger("blah")
t.info("test message", ti=ti)
mock_handler.close.assert_called_once()


@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.skip_if_database_isolation_mode
@pytest.mark.db_test
def test_task_context_log_also_emits_to_call_site_logger(ti):
logger = logging.getLogger("abc123567")
Expand Down
5 changes: 1 addition & 4 deletions tests/utils/test_cli_util.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@
# Mark entire module as db_test because ``action_cli`` wrapper still could use DB on callbacks:
# - ``cli_action_loggers.on_pre_execution``
# - ``cli_action_loggers.on_post_execution``
pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]
repo_root = Path(airflow.__file__).parent.parent


Expand Down Expand Up @@ -82,7 +82,6 @@ def test_success_function(self):
def test_process_subdir_path_with_placeholder(self):
assert os.path.join(settings.DAGS_FOLDER, "abc") == cli.process_subdir("DAGS_FOLDER/abc")

@pytest.mark.db_test
def test_get_dags(self):
dags = cli.get_dags(None, "example_subdag_operator")
assert len(dags) == 1
Expand All @@ -93,7 +92,6 @@ def test_get_dags(self):
with pytest.raises(AirflowException):
cli.get_dags(None, "foobar", True)

@pytest.mark.db_test
@pytest.mark.parametrize(
["given_command", "expected_masked_command"],
[
Expand Down Expand Up @@ -174,7 +172,6 @@ def test_setup_locations_none_pid_path(self):
pid, _, _, _ = cli.setup_locations(process=process_name)
assert pid == default_pid_path

@pytest.mark.db_test
def test_get_dag_by_pickle(self, session, dag_maker):
from airflow.models.dagpickle import DagPickle

Expand Down
2 changes: 1 addition & 1 deletion tests/utils/test_db.py
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@
)
from airflow.utils.session import NEW_SESSION

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDb:
Expand Down
2 changes: 1 addition & 1 deletion tests/utils/test_db_cleanup.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@
from airflow.utils.session import create_session
from tests.test_utils.db import clear_db_dags, clear_db_datasets, clear_db_runs, drop_tables_with_prefix

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


@pytest.fixture(autouse=True)
Expand Down
2 changes: 1 addition & 1 deletion tests/utils/test_dot_renderer.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@

START_DATE = timezone.utcnow()

pytestmark = pytest.mark.db_test
pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode]


class TestDotRenderer:
Expand Down
2 changes: 2 additions & 0 deletions tests/utils/test_email.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@

send_email_test = mock.MagicMock()

pytestmark = pytest.mark.skip_if_database_isolation_mode


class TestEmail:
def setup_method(self):
Expand Down
Loading

0 comments on commit 7c14f20

Please sign in to comment.