Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
18 changes: 3 additions & 15 deletions airflow/models/abstractoperator.py
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
from airflow.models.mappedoperator import MappedOperator
from airflow.models.operator import Operator
from airflow.models.taskinstance import TaskInstance
from airflow.task.priority_strategy import PriorityWeightStrategy
from airflow.utils.task_group import TaskGroup

DEFAULT_OWNER: str = conf.get_mandatory_value("operators", "default_owner")
Expand Down Expand Up @@ -99,6 +100,7 @@ class AbstractOperator(Templater, DAGNode):

weight_rule: str
priority_weight: int
_weight_strategy: PriorityWeightStrategy

# Defines the operator level extra links.
operator_extra_links: Collection[BaseOperatorLink]
Expand Down Expand Up @@ -397,21 +399,7 @@ def priority_weight_total(self) -> int:
- WeightRule.DOWNSTREAM - adds priority weight of all downstream tasks
- WeightRule.UPSTREAM - adds priority weight of all upstream tasks
"""
if self.weight_rule == WeightRule.ABSOLUTE:
return self.priority_weight
elif self.weight_rule == WeightRule.DOWNSTREAM:
upstream = False
elif self.weight_rule == WeightRule.UPSTREAM:
upstream = True
else:
upstream = False
dag = self.get_dag()
if dag is None:
return self.priority_weight
return self.priority_weight + sum(
dag.task_dict[task_id].priority_weight
for task_id in self.get_flat_relative_ids(upstream=upstream)
)
return self._weight_strategy.get_weight(self)

@cached_property
def operator_extra_link_dict(self) -> dict[str, Any]:
Expand Down
9 changes: 2 additions & 7 deletions airflow/models/baseoperator.py
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@
from airflow.models.taskinstance import TaskInstance, clear_task_instances
from airflow.models.taskmixin import DependencyMixin
from airflow.serialization.enums import DagAttributeTypes
from airflow.task.priority_strategy import get_priority_weight_strategy
from airflow.ti_deps.deps.not_in_retry_period_dep import NotInRetryPeriodDep
from airflow.ti_deps.deps.not_previously_skipped_dep import NotPreviouslySkippedDep
from airflow.ti_deps.deps.prev_dagrun_dep import PrevDagrunDep
Expand All @@ -92,7 +93,6 @@
from airflow.utils.setup_teardown import SetupTeardownContext
from airflow.utils.trigger_rule import TriggerRule
from airflow.utils.types import NOTSET
from airflow.utils.weight_rule import WeightRule
from airflow.utils.xcom import XCOM_RETURN_KEY

if TYPE_CHECKING:
Expand Down Expand Up @@ -899,13 +899,8 @@ def __init__(
f"received '{type(priority_weight)}'."
)
self.priority_weight = priority_weight
if not WeightRule.is_valid(weight_rule):
raise AirflowException(
f"The weight_rule must be one of "
f"{WeightRule.all_weight_rules},'{dag.dag_id if dag else ''}.{task_id}'; "
f"received '{weight_rule}'."
)
self.weight_rule = weight_rule
self._weight_strategy = get_priority_weight_strategy(weight_rule)
self.resources = coerce_resources(resources)
if task_concurrency and not max_active_tis_per_dag:
# TODO: Remove in Airflow 3.0
Expand Down
2 changes: 2 additions & 0 deletions airflow/models/mappedoperator.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
)
from airflow.models.pool import Pool
from airflow.serialization.enums import DagAttributeTypes
from airflow.task.priority_strategy import get_priority_weight_strategy
from airflow.ti_deps.deps.mapped_task_expanded import MappedTaskIsExpanded
from airflow.typing_compat import Literal
from airflow.utils.context import context_update_for_unmapped
Expand Down Expand Up @@ -328,6 +329,7 @@ def __attrs_post_init__(self):
f"SLAs are unsupported with mapped tasks. Please set `sla=None` for task "
f"{self.task_id!r}."
)
self._weight_strategy = get_priority_weight_strategy(str(self.weight_rule))

@classmethod
@cache
Expand Down
90 changes: 90 additions & 0 deletions airflow/task/priority_strategy.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""Priority weight strategies for task scheduling."""
from __future__ import annotations

from abc import ABC, abstractmethod
from typing import TYPE_CHECKING

from airflow.exceptions import AirflowException
from airflow.utils.module_loading import import_string

if TYPE_CHECKING:
from airflow.models.abstractoperator import AbstractOperator


class PriorityWeightStrategy(ABC):
"""Priority weight strategy interface."""

@abstractmethod
def get_weight(self, task: AbstractOperator):
"""Get the priority weight of a task."""
...


class AbsolutePriorityWeightStrategy(PriorityWeightStrategy):
"""Priority weight strategy that uses the task's priority weight directly."""

def get_weight(self, task: AbstractOperator):
return task.priority_weight


class DownstreamPriorityWeightStrategy(PriorityWeightStrategy):
"""Priority weight strategy that uses the sum of the priority weights of all downstream tasks."""

def get_weight(self, task: AbstractOperator):
dag = task.get_dag()
if dag is None:
return task.priority_weight
return task.priority_weight + sum(
dag.task_dict[task_id].priority_weight for task_id in task.get_flat_relative_ids(upstream=False)
)


class UpstreamPriorityWeightStrategy(PriorityWeightStrategy):
"""Priority weight strategy that uses the sum of the priority weights of all upstream tasks."""

def get_weight(self, task: AbstractOperator):
dag = task.get_dag()
if dag is None:
return task.priority_weight
return task.priority_weight + sum(
dag.task_dict[task_id].priority_weight for task_id in task.get_flat_relative_ids(upstream=True)
)


_airflow_priority_weight_strategies = {
"absolute": AbsolutePriorityWeightStrategy(),
"downstream": DownstreamPriorityWeightStrategy(),
"upstream": UpstreamPriorityWeightStrategy(),
}


def get_priority_weight_strategy(strategy_name: str) -> PriorityWeightStrategy:
"""Get a priority weight strategy by name or class path."""
if strategy_name not in _airflow_priority_weight_strategies:
try:
priority_strategy_class = import_string(strategy_name)
if not issubclass(priority_strategy_class, PriorityWeightStrategy):
raise AirflowException(
f"Priority strategy {priority_strategy_class} is not a subclass of PriorityWeightStrategy"
)
_airflow_priority_weight_strategies[strategy_name] = priority_strategy_class()
except ImportError:
raise AirflowException(f"Unknown priority strategy {strategy_name}")
return _airflow_priority_weight_strategies[strategy_name]
17 changes: 17 additions & 0 deletions tests/models/test_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
from datetime import timedelta
from io import StringIO
from pathlib import Path
from typing import TYPE_CHECKING
from unittest import mock
from unittest.mock import patch

Expand Down Expand Up @@ -69,6 +70,7 @@
from airflow.operators.python import PythonOperator
from airflow.operators.subdag import SubDagOperator
from airflow.security import permissions
from airflow.task.priority_strategy import PriorityWeightStrategy
from airflow.templates import NativeEnvironment, SandboxedEnvironment
from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
from airflow.timetables.simple import (
Expand All @@ -93,6 +95,9 @@
from tests.test_utils.mapping import expand_mapped_task
from tests.test_utils.timetables import cron_timetable, delta_timetable

if TYPE_CHECKING:
from airflow.models.abstractoperator import AbstractOperator

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

repo_root = Path(__file__).parents[2]
Expand All @@ -114,6 +119,11 @@ def clear_datasets():
clear_db_datasets()


class TestPriorityWeightStrategy(PriorityWeightStrategy):
def get_weight(self, task: AbstractOperator):
return 99


class TestDag:
def setup_method(self) -> None:
clear_db_runs()
Expand Down Expand Up @@ -428,6 +438,13 @@ def test_dag_task_invalid_weight_rule(self):
with pytest.raises(AirflowException):
EmptyOperator(task_id="should_fail", weight_rule="no rule")

def test_dag_task_custom_weight_strategy(self):
with DAG("dag", start_date=DEFAULT_DATE, default_args={"owner": "owner1"}):
task = EmptyOperator(
task_id="empty_task", weight_rule="tests.models.test_dag.TestPriorityWeightStrategy"
)
assert task.priority_weight_total == 99

def test_get_num_task_instances(self):
test_dag_id = "test_get_num_task_instances_dag"
test_task_id = "task_1"
Expand Down