Skip to content

Commit

Permalink
Issue deprecation warning for plugins registering ti_deps
Browse files Browse the repository at this point in the history
This is removed in Airflow3 via #45713
  • Loading branch information
ashb committed Jan 17, 2025
1 parent 04d0381 commit 4429f06
Show file tree
Hide file tree
Showing 3 changed files with 27 additions and 2 deletions.
12 changes: 12 additions & 0 deletions airflow/plugins_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import os
import sys
import types
import warnings
from pathlib import Path
from typing import TYPE_CHECKING, Any, Iterable

Expand Down Expand Up @@ -431,6 +432,17 @@ def initialize_ti_deps_plugins():
registered_ti_dep_classes = {}

for plugin in plugins:
if not plugin.ti_deps:
continue

from airflow.exceptions import RemovedInAirflow3Warning

warnings.warn(
"Using custom `ti_deps` on operators has been removed in Airflow 3.0",
RemovedInAirflow3Warning,
stacklevel=1,
)

registered_ti_dep_classes.update(
{qualname(ti_dep.__class__): ti_dep.__class__ for ti_dep in plugin.ti_deps}
)
Expand Down
12 changes: 12 additions & 0 deletions tests/plugins/test_plugins_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@

import pytest

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.hooks.base import BaseHook
from airflow.listeners.listener import get_listener_manager
from airflow.plugins_manager import AirflowPlugin
Expand Down Expand Up @@ -270,6 +271,17 @@ class AirflowAdminMenuLinksPlugin(AirflowPlugin):
),
]

def test_deprecate_ti_deps(self):
class DeprecatedTIDeps(AirflowPlugin):
name = "ti_deps"

ti_deps = [mock.MagicMock()]

with mock_plugin_manager(plugins=[DeprecatedTIDeps()]), pytest.warns(RemovedInAirflow3Warning):
from airflow import plugins_manager

plugins_manager.initialize_ti_deps_plugins()

def test_should_not_warning_about_fab_plugins(self, caplog):
class AirflowAdminViewsPlugin(AirflowPlugin):
name = "test_admin_views_plugin"
Expand Down
5 changes: 3 additions & 2 deletions tests/serialization/test_dag_serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -1612,7 +1612,7 @@ class DummyTask(BaseOperator):
) as dag:
DummyTask(task_id="task1")

with pytest.raises(SerializationError):
with pytest.raises(SerializationError), pytest.warns(RemovedInAirflow3Warning):
SerializedBaseOperator.serialize_operator(dag.task_dict["task1"])

def test_error_on_unregistered_ti_dep_deserialization(self):
Expand Down Expand Up @@ -1644,7 +1644,8 @@ class DummyTask(BaseOperator):
with DAG(dag_id="test_serialize_custom_ti_deps", schedule=None, start_date=execution_date) as dag:
DummyTask(task_id="task1")

serialize_op = SerializedBaseOperator.serialize_operator(dag.task_dict["task1"])
with pytest.warns(RemovedInAirflow3Warning):
serialize_op = SerializedBaseOperator.serialize_operator(dag.task_dict["task1"])

assert serialize_op["deps"] == [
"airflow.ti_deps.deps.mapped_task_upstream_dep.MappedTaskUpstreamDep",
Expand Down

0 comments on commit 4429f06

Please sign in to comment.