Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

In DAG dependency detector, use class type instead of class name, 2nd attempt #21706

Merged
merged 4 commits into from
Apr 26, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 4 additions & 2 deletions airflow/serialization/serialized_objects.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,9 @@
from airflow.models.param import Param, ParamsDict
from airflow.models.taskmixin import DAGNode
from airflow.models.xcom_arg import XComArg
from airflow.operators.trigger_dagrun import TriggerDagRunOperator
from airflow.providers_manager import ProvidersManager
from airflow.sensors.external_task import ExternalTaskSensor
from airflow.serialization.enums import DagAttributeTypes as DAT, Encoding
from airflow.serialization.helpers import serialize_template_field
from airflow.serialization.json_schema import Validator, load_dag_schema
Expand Down Expand Up @@ -528,14 +530,14 @@ class DependencyDetector:
@staticmethod
def detect_task_dependencies(task: Operator) -> Optional['DagDependency']:
"""Detects dependencies caused by tasks"""
if task.task_type == "TriggerDagRunOperator":
if isinstance(task, TriggerDagRunOperator):
return DagDependency(
source=task.dag_id,
target=getattr(task, "trigger_dag_id"),
dependency_type="trigger",
dependency_id=task.task_id,
)
elif task.task_type == "ExternalTaskSensor":
elif isinstance(task, ExternalTaskSensor):
return DagDependency(
source=getattr(task, "external_dag_id"),
target=task.dag_id,
Expand Down
61 changes: 61 additions & 0 deletions tests/serialization/test_dag_serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -1295,6 +1295,67 @@ class DummyTask(BaseOperator):
'<TIDep(Trigger Rule)>',
]

def test_derived_dag_deps_sensor(self):
"""
Tests DAG dependency detection for sensors, including derived classes
"""
from airflow.operators.empty import EmptyOperator
from airflow.sensors.external_task import ExternalTaskSensor

class DerivedSensor(ExternalTaskSensor):
pass

execution_date = datetime(2020, 1, 1)
for class_ in [ExternalTaskSensor, DerivedSensor]:
with DAG(dag_id="test_derived_dag_deps_sensor", start_date=execution_date) as dag:
task1 = class_(
task_id="task1",
external_dag_id="external_dag_id",
mode="reschedule",
)
task2 = EmptyOperator(task_id="task2")
task1 >> task2

dag = SerializedDAG.to_dict(dag)
assert dag['dag']['dag_dependencies'] == [
{
'source': 'external_dag_id',
'target': 'test_derived_dag_deps_sensor',
'dependency_type': 'sensor',
'dependency_id': 'task1',
}
]

def test_derived_dag_deps_operator(self):
"""
Tests DAG dependency detection for operators, including derived classes
"""
from airflow.operators.empty import EmptyOperator
from airflow.operators.trigger_dagrun import TriggerDagRunOperator

class DerivedOperator(TriggerDagRunOperator):
pass

execution_date = datetime(2020, 1, 1)
for class_ in [TriggerDagRunOperator, DerivedOperator]:
with DAG(dag_id="test_derived_dag_deps_trigger", start_date=execution_date) as dag:
task1 = EmptyOperator(task_id="task1")
task2 = class_(
task_id="task2",
trigger_dag_id="trigger_dag_id",
)
task1 >> task2

dag = SerializedDAG.to_dict(dag)
assert dag['dag']['dag_dependencies'] == [
{
'source': 'test_derived_dag_deps_trigger',
'target': 'trigger_dag_id',
'dependency_type': 'trigger',
'dependency_id': 'task2',
}
]

def test_task_group_sorted(self):
"""
Tests serialize_task_group, make sure the list is in order
Expand Down