Skip to content
Closed
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
7 changes: 7 additions & 0 deletions airflow/api_connexion/openapi/v1.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -3977,6 +3977,8 @@ components:
readOnly: true
weight_rule:
$ref: "#/components/schemas/WeightRule"
priority_weight_strategy:
$ref: "#/components/schemas/PriorityWeightStrategy"
ui_color:
$ref: "#/components/schemas/Color"
ui_fgcolor:
Expand Down Expand Up @@ -5049,11 +5051,16 @@ components:
WeightRule:
description: Weight rule.
type: string
nullable: true
enum:
- downstream
- upstream
- absolute

PriorityWeightStrategy:
description: Priority weight strategy.
type: string

HealthStatus:
description: Health status
type: string
Expand Down
11 changes: 11 additions & 0 deletions airflow/api_connexion/schemas/task_schema.py
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ class TaskSchema(Schema):
retry_exponential_backoff = fields.Boolean(dump_only=True)
priority_weight = fields.Number(dump_only=True)
weight_rule = WeightRuleField(dump_only=True)
priority_weight_strategy = fields.Method("_get_priority_weight_strategy", dump_only=True)
ui_color = ColorField(dump_only=True)
ui_fgcolor = ColorField(dump_only=True)
template_fields = fields.List(fields.String(), dump_only=True)
Expand Down Expand Up @@ -84,6 +85,16 @@ def _get_params(obj):
def _get_is_mapped(obj):
return isinstance(obj, MappedOperator)

@staticmethod
def _get_priority_weight_strategy(obj):
from airflow.serialization.serialized_objects import _encode_priority_weight_strategy

return (
obj.priority_weight_strategy
if isinstance(obj.priority_weight_strategy, str)
else _encode_priority_weight_strategy(obj.priority_weight_strategy)
)


class TaskCollection(NamedTuple):
"""List of Tasks with metadata."""
Expand Down
11 changes: 11 additions & 0 deletions airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,17 @@ core:
description: |
The weighting method used for the effective total priority weight of the task
version_added: 2.2.0
version_deprecated: 2.8.0
deprecation_reason: |
This option is deprecated and will be removed in Airflow 3.0.
Please use ``default_task_priority_weight_strategy`` instead.
type: string
example: ~
default: ~
default_task_priority_weight_strategy:
description: |
The strategy used for the effective total priority weight of the task
version_added: 2.8.0
type: string
example: ~
default: "downstream"
Expand Down
60 changes: 60 additions & 0 deletions airflow/example_dags/example_priority_weight_strategy.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
#
# 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.
"""Example DAG demonstrating the usage of a custom PriorityWeightStrategy class."""

from __future__ import annotations

from typing import TYPE_CHECKING

import pendulum

from airflow.models.dag import DAG
from airflow.operators.python import PythonOperator

if TYPE_CHECKING:
from airflow.models import TaskInstance


def success_on_third_attempt(ti: TaskInstance, **context):
if ti.try_number < 3:
raise Exception("Not yet")


with DAG(
dag_id="example_priority_weight_strategy",
start_date=pendulum.datetime(2021, 1, 1, tz="UTC"),
catchup=False,
schedule="@daily",
tags=["example"],
default_args={
"retries": 3,
"retry_delay": pendulum.duration(seconds=10),
},
) as dag:
fixed_weight_task = PythonOperator(
task_id="fixed_weight_task",
python_callable=success_on_third_attempt,
priority_weight_strategy="downstream",
)

decreasing_weight_task = PythonOperator(
task_id="decreasing_weight_task",
python_callable=success_on_third_attempt,
# TODO: Uncomment this line to use the decreasing priority weight strategy.
# priority_weight_strategy=("decreasing_priority_weight_strategy.DecreasingPriorityStrategy"),
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
# 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.
from __future__ import annotations

from typing import TYPE_CHECKING

from airflow.plugins_manager import AirflowPlugin
from airflow.task.priority_strategy import PriorityWeightStrategy

if TYPE_CHECKING:
from airflow.models import TaskInstance


class DecreasingPriorityStrategy(PriorityWeightStrategy):
"""A priority weight strategy that decreases the priority weight with each attempt."""

def get_weight(self, ti: TaskInstance):
return max(3 - ti._try_number + 1, 1)


class DecreasingPriorityWeightStrategyPlugin(AirflowPlugin):
name = "decreasing_priority_weight_strategy_plugin"
priority_weight_strategies = [DecreasingPriorityStrategy]
2 changes: 1 addition & 1 deletion airflow/executors/base_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ def queue_task_instance(
self.queue_command(
task_instance,
command_list_to_run,
priority=task_instance.task.priority_weight_total,
priority=task_instance.priority_weight,
queue=task_instance.task.queue,
)

Expand Down
2 changes: 1 addition & 1 deletion airflow/executors/debug_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ def queue_task_instance(
self.queue_command(
task_instance,
[str(task_instance)], # Just for better logging, it's not used anywhere
priority=task_instance.task.priority_weight_total,
priority=task_instance.priority_weight,
queue=task_instance.task.queue,
)
# Save params for TaskInstance._run_raw_task
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
#
# 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.

"""add priority_weight_strategy to task_instance

Revision ID: 624ecf3b6a5e
Revises: 1fd565369930
Create Date: 2023-10-29 02:01:34.774596

"""

import sqlalchemy as sa
from alembic import op

# revision identifiers, used by Alembic.
revision = "624ecf3b6a5e"
down_revision = "ab34f260b71c"
branch_labels = None
depends_on = None
airflow_version = "2.9.0"


def upgrade():
"""Apply add priority_weight_strategy to task_instance"""
with op.batch_alter_table("task_instance") as batch_op:
batch_op.add_column(sa.Column("_priority_weight_strategy", sa.JSON()))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand we need to store some context information about the selcted priority weight strategy - but do we really need to add this to the DB? TaskInstance is the most largest table in the DB scheme and potentially contains millions of rows. Do we really want to store the same values in mostly millions of cases? Or can we leave it NULL and store only a value if we have this special rule and data needs stored?

I like the approach of this PR in general but fear this will create a lot of overhead in DB. Especially as it is a JSON field.



def downgrade():
"""Unapply add priority_weight_strategy to task_instance"""
with op.batch_alter_table("task_instance") as batch_op:
batch_op.drop_column("_priority_weight_strategy")
27 changes: 24 additions & 3 deletions airflow/models/abstractoperator.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import datetime
import inspect
import warnings
from functools import cached_property
from typing import TYPE_CHECKING, Any, Callable, ClassVar, Collection, Iterable, Iterator, Sequence

Expand Down Expand Up @@ -53,6 +54,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 All @@ -69,8 +71,14 @@
)
MAX_RETRY_DELAY: int = conf.getint("core", "max_task_retry_delay", fallback=24 * 60 * 60)

DEFAULT_WEIGHT_RULE: WeightRule = WeightRule(
conf.get("core", "default_task_weight_rule", fallback=WeightRule.DOWNSTREAM)
DEFAULT_WEIGHT_RULE: WeightRule | None = (
WeightRule(conf.get("core", "default_task_weight_rule", fallback=None))
if conf.get("core", "default_task_weight_rule", fallback=None)
else None
)

DEFAULT_PRIORITY_WEIGHT_STRATEGY: str = conf.get(
"core", "default_task_priority_weight_strategy", fallback=WeightRule.DOWNSTREAM
)
DEFAULT_TRIGGER_RULE: TriggerRule = TriggerRule.ALL_SUCCESS
DEFAULT_TASK_EXECUTION_TIMEOUT: datetime.timedelta | None = conf.gettimedelta(
Expand All @@ -97,7 +105,8 @@ class AbstractOperator(Templater, DAGNode):

operator_class: type[BaseOperator] | dict[str, Any]

weight_rule: str
weight_rule: str | None
priority_weight_strategy: str | PriorityWeightStrategy
priority_weight: int

# Defines the operator level extra links.
Expand Down Expand Up @@ -197,6 +206,12 @@ def on_failure_fail_dagrun(self, value):
)
self._on_failure_fail_dagrun = value

@property
def parsed_priority_weight_strategy(self) -> PriorityWeightStrategy:
from airflow.task.priority_strategy import validate_and_load_priority_weight_strategy

return validate_and_load_priority_weight_strategy(self.priority_weight_strategy)

def as_setup(self):
self.is_setup = True
return self
Expand Down Expand Up @@ -397,6 +412,12 @@ def priority_weight_total(self) -> int:
- WeightRule.DOWNSTREAM - adds priority weight of all downstream tasks
- WeightRule.UPSTREAM - adds priority weight of all upstream tasks
"""
warnings.warn(
"Accessing `priority_weight_total` from AbstractOperator instance is deprecated."
" Please use `priority_weight` from task instance instead.",
DeprecationWarning,
stacklevel=2,
)
if self.weight_rule == WeightRule.ABSOLUTE:
return self.priority_weight
elif self.weight_rule == WeightRule.DOWNSTREAM:
Expand Down
Loading