-
Notifications
You must be signed in to change notification settings - Fork 14.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add
@task.short_circuit
TaskFlow decorator (#25752)
- Loading branch information
Showing
7 changed files
with
256 additions
and
18 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
# 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 typing import Callable, Optional, Sequence | ||
|
||
from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory | ||
from airflow.operators.python import ShortCircuitOperator | ||
|
||
|
||
class _ShortCircuitDecoratedOperator(DecoratedOperator, ShortCircuitOperator): | ||
""" | ||
Wraps a Python callable and captures args/kwargs when called for execution. | ||
:param python_callable: A reference to an object that is callable | ||
:param op_kwargs: a dictionary of keyword arguments that will get unpacked | ||
in your function (templated) | ||
:param op_args: a list of positional arguments that will get unpacked when | ||
calling your callable (templated) | ||
:param multiple_outputs: If set to True, the decorated function's return value will be unrolled to | ||
multiple XCom values. Dict will unroll to XCom values with its keys as XCom keys. Defaults to False. | ||
""" | ||
|
||
template_fields: Sequence[str] = ('op_args', 'op_kwargs') | ||
template_fields_renderers = {"op_args": "py", "op_kwargs": "py"} | ||
|
||
# since we won't mutate the arguments, we should just do the shallow copy | ||
# there are some cases we can't deepcopy the objects (e.g protobuf). | ||
shallow_copy_attrs: Sequence[str] = ('python_callable',) | ||
|
||
custom_operator_name: str = '@task.short_circuit' | ||
|
||
def __init__(self, *, python_callable, op_args, op_kwargs, **kwargs) -> None: | ||
kwargs_to_upstream = { | ||
"python_callable": python_callable, | ||
"op_args": op_args, | ||
"op_kwargs": op_kwargs, | ||
} | ||
super().__init__( | ||
kwargs_to_upstream=kwargs_to_upstream, | ||
python_callable=python_callable, | ||
op_args=op_args, | ||
op_kwargs=op_kwargs, | ||
**kwargs, | ||
) | ||
|
||
|
||
def short_circuit_task( | ||
python_callable: Optional[Callable] = None, | ||
multiple_outputs: Optional[bool] = None, | ||
**kwargs, | ||
) -> TaskDecorator: | ||
"""Wraps a function into an ShortCircuitOperator. | ||
Accepts kwargs for operator kwarg. Can be reused in a single DAG. | ||
This function is only used only used during type checking or auto-completion. | ||
:param python_callable: Function to decorate | ||
:param multiple_outputs: If set to True, the decorated function's return value will be unrolled to | ||
multiple XCom values. Dict will unroll to XCom values with its keys as XCom keys. Defaults to False. | ||
:meta private: | ||
""" | ||
return task_decorator_factory( | ||
python_callable=python_callable, | ||
multiple_outputs=multiple_outputs, | ||
decorated_operator_class=_ShortCircuitDecoratedOperator, | ||
**kwargs, | ||
) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,59 @@ | ||
# 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 the `@task.short_circuit()` TaskFlow decorator.""" | ||
import pendulum | ||
|
||
from airflow.decorators import dag, task | ||
from airflow.models.baseoperator import chain | ||
from airflow.operators.empty import EmptyOperator | ||
from airflow.utils.trigger_rule import TriggerRule | ||
|
||
|
||
@dag(start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=['example']) | ||
def example_short_circuit_decorator(): | ||
# [START howto_operator_short_circuit] | ||
@task.short_circuit() | ||
def check_condition(condition): | ||
return condition | ||
|
||
ds_true = [EmptyOperator(task_id='true_' + str(i)) for i in [1, 2]] | ||
ds_false = [EmptyOperator(task_id='false_' + str(i)) for i in [1, 2]] | ||
|
||
condition_is_true = check_condition.override(task_id="condition_is_true")(condition=True) | ||
condition_is_false = check_condition.override(task_id="condition_is_false")(condition=False) | ||
|
||
chain(condition_is_true, *ds_true) | ||
chain(condition_is_false, *ds_false) | ||
# [END howto_operator_short_circuit] | ||
|
||
# [START howto_operator_short_circuit_trigger_rules] | ||
[task_1, task_2, task_3, task_4, task_5, task_6] = [ | ||
EmptyOperator(task_id=f"task_{i}") for i in range(1, 7) | ||
] | ||
|
||
task_7 = EmptyOperator(task_id="task_7", trigger_rule=TriggerRule.ALL_DONE) | ||
|
||
short_circuit = check_condition.override(task_id="short_circuit", ignore_downstream_trigger_rules=False)( | ||
condition=False | ||
) | ||
|
||
chain(task_1, [task_2, short_circuit], [task_3, task_4], [task_5, task_6], task_7) | ||
# [END howto_operator_short_circuit_trigger_rules] | ||
|
||
|
||
example_dag = example_short_circuit_decorator() |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
# | ||
# 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 pendulum import datetime | ||
|
||
from airflow.decorators import task | ||
from airflow.utils.state import State | ||
from airflow.utils.trigger_rule import TriggerRule | ||
|
||
DEFAULT_DATE = datetime(2022, 8, 17) | ||
|
||
|
||
def test_short_circuit_decorator(dag_maker): | ||
with dag_maker(): | ||
|
||
@task | ||
def empty(): | ||
... | ||
|
||
@task.short_circuit() | ||
def short_circuit(condition): | ||
return condition | ||
|
||
short_circuit_false = short_circuit.override(task_id="short_circuit_false")(condition=False) | ||
task_1 = empty.override(task_id="task_1")() | ||
short_circuit_false >> task_1 | ||
|
||
short_circuit_true = short_circuit.override(task_id="short_circuit_true")(condition=True) | ||
task_2 = empty.override(task_id="task_2")() | ||
short_circuit_true >> task_2 | ||
|
||
short_circuit_respect_trigger_rules = short_circuit.override( | ||
task_id="short_circuit_respect_trigger_rules", ignore_downstream_trigger_rules=False | ||
)(condition=False) | ||
task_3 = empty.override(task_id="task_3")() | ||
task_4 = empty.override(task_id="task_4")() | ||
task_5 = empty.override(task_id="task_5", trigger_rule=TriggerRule.ALL_DONE)() | ||
short_circuit_respect_trigger_rules >> [task_3, task_4] >> task_5 | ||
|
||
dr = dag_maker.create_dagrun() | ||
|
||
for t in dag_maker.dag.tasks: | ||
t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) | ||
|
||
task_state_mapping = { | ||
"short_circuit_false": State.SUCCESS, | ||
"task_1": State.SKIPPED, | ||
"short_circuit_true": State.SUCCESS, | ||
"task_2": State.SUCCESS, | ||
"short_circuit_respect_trigger_rules": State.SUCCESS, | ||
"task_3": State.SKIPPED, | ||
"task_4": State.SKIPPED, | ||
"task_5": State.SUCCESS, | ||
} | ||
|
||
tis = dr.get_task_instances() | ||
for ti in tis: | ||
assert ti.state == task_state_mapping[ti.task_id] |