apache-airflow-providers-standard 0.0.1__py3-none-any.whl
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
Potentially problematic release.
This version of apache-airflow-providers-standard might be problematic. Click here for more details.
- airflow/providers/standard/LICENSE +253 -0
- airflow/providers/standard/__init__.py +39 -0
- airflow/providers/standard/get_provider_info.py +95 -0
- airflow/providers/standard/hooks/__init__.py +16 -0
- airflow/providers/standard/hooks/filesystem.py +89 -0
- airflow/providers/standard/hooks/package_index.py +95 -0
- airflow/providers/standard/hooks/subprocess.py +119 -0
- airflow/providers/standard/operators/__init__.py +16 -0
- airflow/providers/standard/operators/bash.py +304 -0
- airflow/providers/standard/operators/datetime.py +110 -0
- airflow/providers/standard/operators/generic_transfer.py +133 -0
- airflow/providers/standard/operators/python.py +1172 -0
- airflow/providers/standard/operators/weekday.py +120 -0
- airflow/providers/standard/sensors/__init__.py +16 -0
- airflow/providers/standard/sensors/bash.py +114 -0
- airflow/providers/standard/sensors/date_time.py +152 -0
- airflow/providers/standard/sensors/python.py +80 -0
- airflow/providers/standard/sensors/time.py +130 -0
- airflow/providers/standard/sensors/time_delta.py +134 -0
- airflow/providers/standard/sensors/weekday.py +104 -0
- airflow/providers/standard/utils/__init__.py +16 -0
- airflow/providers/standard/utils/python_virtualenv.py +209 -0
- airflow/providers/standard/utils/python_virtualenv_script.jinja2 +101 -0
- airflow/providers/standard/utils/version_references.py +26 -0
- apache_airflow_providers_standard-0.0.1.dist-info/METADATA +112 -0
- apache_airflow_providers_standard-0.0.1.dist-info/RECORD +28 -0
- apache_airflow_providers_standard-0.0.1.dist-info/WHEEL +4 -0
- apache_airflow_providers_standard-0.0.1.dist-info/entry_points.txt +3 -0
|
@@ -0,0 +1,120 @@
|
|
|
1
|
+
#
|
|
2
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
3
|
+
# or more contributor license agreements. See the NOTICE file
|
|
4
|
+
# distributed with this work for additional information
|
|
5
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
6
|
+
# to you under the Apache License, Version 2.0 (the
|
|
7
|
+
# "License"); you may not use this file except in compliance
|
|
8
|
+
# with the License. You may obtain a copy of the License at
|
|
9
|
+
#
|
|
10
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
11
|
+
#
|
|
12
|
+
# Unless required by applicable law or agreed to in writing,
|
|
13
|
+
# software distributed under the License is distributed on an
|
|
14
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
15
|
+
# KIND, either express or implied. See the License for the
|
|
16
|
+
# specific language governing permissions and limitations
|
|
17
|
+
# under the License.
|
|
18
|
+
from __future__ import annotations
|
|
19
|
+
|
|
20
|
+
from typing import TYPE_CHECKING, Iterable
|
|
21
|
+
|
|
22
|
+
from airflow.operators.branch import BaseBranchOperator
|
|
23
|
+
from airflow.utils import timezone
|
|
24
|
+
from airflow.utils.weekday import WeekDay
|
|
25
|
+
|
|
26
|
+
if TYPE_CHECKING:
|
|
27
|
+
from airflow.utils.context import Context
|
|
28
|
+
|
|
29
|
+
|
|
30
|
+
class BranchDayOfWeekOperator(BaseBranchOperator):
|
|
31
|
+
"""
|
|
32
|
+
Branches into one of two lists of tasks depending on the current day.
|
|
33
|
+
|
|
34
|
+
For more information on how to use this operator, take a look at the guide:
|
|
35
|
+
:ref:`howto/operator:BranchDayOfWeekOperator`
|
|
36
|
+
|
|
37
|
+
**Example** (with single day):
|
|
38
|
+
|
|
39
|
+
.. code-block:: python
|
|
40
|
+
|
|
41
|
+
from airflow.operators.empty import EmptyOperator
|
|
42
|
+
from airflow.operators.weekday import BranchDayOfWeekOperator
|
|
43
|
+
|
|
44
|
+
monday = EmptyOperator(task_id="monday")
|
|
45
|
+
other_day = EmptyOperator(task_id="other_day")
|
|
46
|
+
|
|
47
|
+
monday_check = BranchDayOfWeekOperator(
|
|
48
|
+
task_id="monday_check",
|
|
49
|
+
week_day="Monday",
|
|
50
|
+
use_task_logical_date=True,
|
|
51
|
+
follow_task_ids_if_true="monday",
|
|
52
|
+
follow_task_ids_if_false="other_day",
|
|
53
|
+
)
|
|
54
|
+
monday_check >> [monday, other_day]
|
|
55
|
+
|
|
56
|
+
**Example** (with :class:`~airflow.utils.weekday.WeekDay` enum):
|
|
57
|
+
|
|
58
|
+
.. code-block:: python
|
|
59
|
+
|
|
60
|
+
# import WeekDay Enum
|
|
61
|
+
from airflow.utils.weekday import WeekDay
|
|
62
|
+
from airflow.operators.empty import EmptyOperator
|
|
63
|
+
from airflow.operators.weekday import BranchDayOfWeekOperator
|
|
64
|
+
|
|
65
|
+
workday = EmptyOperator(task_id="workday")
|
|
66
|
+
weekend = EmptyOperator(task_id="weekend")
|
|
67
|
+
weekend_check = BranchDayOfWeekOperator(
|
|
68
|
+
task_id="weekend_check",
|
|
69
|
+
week_day={WeekDay.SATURDAY, WeekDay.SUNDAY},
|
|
70
|
+
use_task_logical_date=True,
|
|
71
|
+
follow_task_ids_if_true="weekend",
|
|
72
|
+
follow_task_ids_if_false="workday",
|
|
73
|
+
)
|
|
74
|
+
# add downstream dependencies as you would do with any branch operator
|
|
75
|
+
weekend_check >> [workday, weekend]
|
|
76
|
+
|
|
77
|
+
:param follow_task_ids_if_true: task_id, task_group_id, or a list of task_ids and/or task_group_ids
|
|
78
|
+
to follow if criteria met.
|
|
79
|
+
:param follow_task_ids_if_false: task_id, task_group_id, or a list of task_ids and/or task_group_ids
|
|
80
|
+
to follow if criteria not met.
|
|
81
|
+
:param week_day: Day of the week to check (full name). Optionally, a set
|
|
82
|
+
of days can also be provided using a set. Example values:
|
|
83
|
+
|
|
84
|
+
* ``"MONDAY"``,
|
|
85
|
+
* ``{"Saturday", "Sunday"}``
|
|
86
|
+
* ``{WeekDay.TUESDAY}``
|
|
87
|
+
* ``{WeekDay.SATURDAY, WeekDay.SUNDAY}``
|
|
88
|
+
|
|
89
|
+
To use `WeekDay` enum, import it from `airflow.utils.weekday`
|
|
90
|
+
|
|
91
|
+
:param use_task_logical_date: If ``True``, uses task's logical date to compare
|
|
92
|
+
with is_today. Execution Date is Useful for backfilling.
|
|
93
|
+
If ``False``, uses system's day of the week.
|
|
94
|
+
"""
|
|
95
|
+
|
|
96
|
+
def __init__(
|
|
97
|
+
self,
|
|
98
|
+
*,
|
|
99
|
+
follow_task_ids_if_true: str | Iterable[str],
|
|
100
|
+
follow_task_ids_if_false: str | Iterable[str],
|
|
101
|
+
week_day: str | Iterable[str] | WeekDay | Iterable[WeekDay],
|
|
102
|
+
use_task_logical_date: bool = False,
|
|
103
|
+
**kwargs,
|
|
104
|
+
) -> None:
|
|
105
|
+
super().__init__(**kwargs)
|
|
106
|
+
self.follow_task_ids_if_true = follow_task_ids_if_true
|
|
107
|
+
self.follow_task_ids_if_false = follow_task_ids_if_false
|
|
108
|
+
self.week_day = week_day
|
|
109
|
+
self.use_task_logical_date = use_task_logical_date
|
|
110
|
+
self._week_day_num = WeekDay.validate_week_day(week_day)
|
|
111
|
+
|
|
112
|
+
def choose_branch(self, context: Context) -> str | Iterable[str]:
|
|
113
|
+
if self.use_task_logical_date:
|
|
114
|
+
now = context["logical_date"]
|
|
115
|
+
else:
|
|
116
|
+
now = timezone.make_naive(timezone.utcnow(), self.dag.timezone)
|
|
117
|
+
|
|
118
|
+
if now.isoweekday() in self._week_day_num:
|
|
119
|
+
return self.follow_task_ids_if_true
|
|
120
|
+
return self.follow_task_ids_if_false
|
|
@@ -0,0 +1,16 @@
|
|
|
1
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
2
|
+
# or more contributor license agreements. See the NOTICE file
|
|
3
|
+
# distributed with this work for additional information
|
|
4
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
5
|
+
# to you under the Apache License, Version 2.0 (the
|
|
6
|
+
# "License"); you may not use this file except in compliance
|
|
7
|
+
# with the License. You may obtain a copy of the License at
|
|
8
|
+
#
|
|
9
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
10
|
+
#
|
|
11
|
+
# Unless required by applicable law or agreed to in writing,
|
|
12
|
+
# software distributed under the License is distributed on an
|
|
13
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
14
|
+
# KIND, either express or implied. See the License for the
|
|
15
|
+
# specific language governing permissions and limitations
|
|
16
|
+
# under the License.
|
|
@@ -0,0 +1,114 @@
|
|
|
1
|
+
#
|
|
2
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
3
|
+
# or more contributor license agreements. See the NOTICE file
|
|
4
|
+
# distributed with this work for additional information
|
|
5
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
6
|
+
# to you under the Apache License, Version 2.0 (the
|
|
7
|
+
# "License"); you may not use this file except in compliance
|
|
8
|
+
# with the License. You may obtain a copy of the License at
|
|
9
|
+
#
|
|
10
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
11
|
+
#
|
|
12
|
+
# Unless required by applicable law or agreed to in writing,
|
|
13
|
+
# software distributed under the License is distributed on an
|
|
14
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
15
|
+
# KIND, either express or implied. See the License for the
|
|
16
|
+
# specific language governing permissions and limitations
|
|
17
|
+
# under the License.
|
|
18
|
+
from __future__ import annotations
|
|
19
|
+
|
|
20
|
+
import os
|
|
21
|
+
from subprocess import PIPE, STDOUT, Popen
|
|
22
|
+
from tempfile import NamedTemporaryFile, TemporaryDirectory, gettempdir
|
|
23
|
+
from typing import TYPE_CHECKING, Sequence
|
|
24
|
+
|
|
25
|
+
from airflow.exceptions import AirflowFailException
|
|
26
|
+
from airflow.sensors.base import BaseSensorOperator
|
|
27
|
+
|
|
28
|
+
if TYPE_CHECKING:
|
|
29
|
+
from airflow.utils.context import Context
|
|
30
|
+
|
|
31
|
+
|
|
32
|
+
class BashSensor(BaseSensorOperator):
|
|
33
|
+
"""
|
|
34
|
+
Executes a bash command/script.
|
|
35
|
+
|
|
36
|
+
Return True if and only if the return code is 0.
|
|
37
|
+
|
|
38
|
+
:param bash_command: The command, set of commands or reference to a
|
|
39
|
+
bash script (must be '.sh') to be executed.
|
|
40
|
+
|
|
41
|
+
:param env: If env is not None, it must be a mapping that defines the
|
|
42
|
+
environment variables for the new process; these are used instead
|
|
43
|
+
of inheriting the current process environment, which is the default
|
|
44
|
+
behavior. (templated)
|
|
45
|
+
:param output_encoding: output encoding of bash command.
|
|
46
|
+
:param retry_exit_code: If task exits with this code, treat the sensor
|
|
47
|
+
as not-yet-complete and retry the check later according to the
|
|
48
|
+
usual retry/timeout settings. Any other non-zero return code will
|
|
49
|
+
be treated as an error, and cause the sensor to fail. If set to
|
|
50
|
+
``None`` (the default), any non-zero exit code will cause a retry
|
|
51
|
+
and the task will never raise an error except on time-out.
|
|
52
|
+
|
|
53
|
+
.. seealso::
|
|
54
|
+
For more information on how to use this sensor,take a look at the guide:
|
|
55
|
+
:ref:`howto/operator:BashSensor`
|
|
56
|
+
"""
|
|
57
|
+
|
|
58
|
+
template_fields: Sequence[str] = ("bash_command", "env")
|
|
59
|
+
|
|
60
|
+
def __init__(
|
|
61
|
+
self, *, bash_command, env=None, output_encoding="utf-8", retry_exit_code: int | None = None, **kwargs
|
|
62
|
+
):
|
|
63
|
+
super().__init__(**kwargs)
|
|
64
|
+
self.bash_command = bash_command
|
|
65
|
+
self.env = env
|
|
66
|
+
self.output_encoding = output_encoding
|
|
67
|
+
self.retry_exit_code = retry_exit_code
|
|
68
|
+
|
|
69
|
+
def poke(self, context: Context):
|
|
70
|
+
"""Execute the bash command in a temporary directory."""
|
|
71
|
+
bash_command = self.bash_command
|
|
72
|
+
self.log.info("Tmp dir root location: %s", gettempdir())
|
|
73
|
+
with TemporaryDirectory(prefix="airflowtmp") as tmp_dir, NamedTemporaryFile(
|
|
74
|
+
dir=tmp_dir, prefix=self.task_id
|
|
75
|
+
) as f:
|
|
76
|
+
f.write(bytes(bash_command, "utf_8"))
|
|
77
|
+
f.flush()
|
|
78
|
+
fname = f.name
|
|
79
|
+
script_location = tmp_dir + "/" + fname
|
|
80
|
+
self.log.info("Temporary script location: %s", script_location)
|
|
81
|
+
self.log.info("Running command: %s", bash_command)
|
|
82
|
+
|
|
83
|
+
with Popen(
|
|
84
|
+
["bash", fname],
|
|
85
|
+
stdout=PIPE,
|
|
86
|
+
stderr=STDOUT,
|
|
87
|
+
close_fds=True,
|
|
88
|
+
cwd=tmp_dir,
|
|
89
|
+
env=self.env,
|
|
90
|
+
preexec_fn=os.setsid,
|
|
91
|
+
) as resp:
|
|
92
|
+
if resp.stdout:
|
|
93
|
+
self.log.info("Output:")
|
|
94
|
+
for line in iter(resp.stdout.readline, b""):
|
|
95
|
+
self.log.info(line.decode(self.output_encoding).strip())
|
|
96
|
+
resp.wait()
|
|
97
|
+
self.log.info("Command exited with return code %s", resp.returncode)
|
|
98
|
+
|
|
99
|
+
# zero code means success, the sensor can go green
|
|
100
|
+
if resp.returncode == 0:
|
|
101
|
+
return True
|
|
102
|
+
|
|
103
|
+
# we have a retry exit code, sensor retries if return code matches, otherwise error
|
|
104
|
+
elif self.retry_exit_code is not None:
|
|
105
|
+
if resp.returncode == self.retry_exit_code:
|
|
106
|
+
self.log.info("Return code matches retry code, will retry later")
|
|
107
|
+
return False
|
|
108
|
+
else:
|
|
109
|
+
raise AirflowFailException(f"Command exited with return code {resp.returncode}")
|
|
110
|
+
|
|
111
|
+
# backwards compatibility: sensor retries no matter the error code
|
|
112
|
+
else:
|
|
113
|
+
self.log.info("Non-zero return code and no retry code set, will retry later")
|
|
114
|
+
return False
|
|
@@ -0,0 +1,152 @@
|
|
|
1
|
+
#
|
|
2
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
3
|
+
# or more contributor license agreements. See the NOTICE file
|
|
4
|
+
# distributed with this work for additional information
|
|
5
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
6
|
+
# to you under the Apache License, Version 2.0 (the
|
|
7
|
+
# "License"); you may not use this file except in compliance
|
|
8
|
+
# with the License. You may obtain a copy of the License at
|
|
9
|
+
#
|
|
10
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
11
|
+
#
|
|
12
|
+
# Unless required by applicable law or agreed to in writing,
|
|
13
|
+
# software distributed under the License is distributed on an
|
|
14
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
15
|
+
# KIND, either express or implied. See the License for the
|
|
16
|
+
# specific language governing permissions and limitations
|
|
17
|
+
# under the License.
|
|
18
|
+
from __future__ import annotations
|
|
19
|
+
|
|
20
|
+
import datetime
|
|
21
|
+
from dataclasses import dataclass
|
|
22
|
+
from typing import TYPE_CHECKING, Any, NoReturn, Sequence
|
|
23
|
+
|
|
24
|
+
from airflow.providers.standard.utils.version_references import AIRFLOW_V_3_0_PLUS
|
|
25
|
+
from airflow.sensors.base import BaseSensorOperator
|
|
26
|
+
|
|
27
|
+
try:
|
|
28
|
+
from airflow.triggers.base import StartTriggerArgs
|
|
29
|
+
except ImportError:
|
|
30
|
+
# TODO: Remove this when min airflow version is 2.10.0 for standard provider
|
|
31
|
+
@dataclass
|
|
32
|
+
class StartTriggerArgs: # type: ignore[no-redef]
|
|
33
|
+
"""Arguments required for start task execution from triggerer."""
|
|
34
|
+
|
|
35
|
+
trigger_cls: str
|
|
36
|
+
next_method: str
|
|
37
|
+
trigger_kwargs: dict[str, Any] | None = None
|
|
38
|
+
next_kwargs: dict[str, Any] | None = None
|
|
39
|
+
timeout: datetime.timedelta | None = None
|
|
40
|
+
|
|
41
|
+
|
|
42
|
+
from airflow.triggers.temporal import DateTimeTrigger
|
|
43
|
+
from airflow.utils import timezone
|
|
44
|
+
|
|
45
|
+
if TYPE_CHECKING:
|
|
46
|
+
from airflow.utils.context import Context
|
|
47
|
+
|
|
48
|
+
|
|
49
|
+
class DateTimeSensor(BaseSensorOperator):
|
|
50
|
+
"""
|
|
51
|
+
Waits until the specified datetime.
|
|
52
|
+
|
|
53
|
+
A major advantage of this sensor is idempotence for the ``target_time``.
|
|
54
|
+
It handles some cases for which ``TimeSensor`` and ``TimeDeltaSensor`` are not suited.
|
|
55
|
+
|
|
56
|
+
**Example** 1 :
|
|
57
|
+
If a task needs to wait for 11am on each ``execution_date``. Using
|
|
58
|
+
``TimeSensor`` or ``TimeDeltaSensor``, all backfill tasks started at
|
|
59
|
+
1am have to wait for 10 hours. This is unnecessary, e.g. a backfill
|
|
60
|
+
task with ``{{ ds }} = '1970-01-01'`` does not need to wait because
|
|
61
|
+
``1970-01-01T11:00:00`` has already passed.
|
|
62
|
+
|
|
63
|
+
**Example** 2 :
|
|
64
|
+
If a DAG is scheduled to run at 23:00 daily, but one of the tasks is
|
|
65
|
+
required to run at 01:00 next day, using ``TimeSensor`` will return
|
|
66
|
+
``True`` immediately because 23:00 > 01:00. Instead, we can do this:
|
|
67
|
+
|
|
68
|
+
.. code-block:: python
|
|
69
|
+
|
|
70
|
+
DateTimeSensor(
|
|
71
|
+
task_id="wait_for_0100",
|
|
72
|
+
target_time="{{ next_execution_date.tomorrow().replace(hour=1) }}",
|
|
73
|
+
)
|
|
74
|
+
|
|
75
|
+
:param target_time: datetime after which the job succeeds. (templated)
|
|
76
|
+
"""
|
|
77
|
+
|
|
78
|
+
template_fields: Sequence[str] = ("target_time",)
|
|
79
|
+
|
|
80
|
+
def __init__(self, *, target_time: str | datetime.datetime, **kwargs) -> None:
|
|
81
|
+
super().__init__(**kwargs)
|
|
82
|
+
|
|
83
|
+
# self.target_time can't be a datetime object as it is a template_field
|
|
84
|
+
if isinstance(target_time, datetime.datetime):
|
|
85
|
+
self.target_time = target_time.isoformat()
|
|
86
|
+
elif isinstance(target_time, str):
|
|
87
|
+
self.target_time = target_time
|
|
88
|
+
else:
|
|
89
|
+
raise TypeError(
|
|
90
|
+
f"Expected str or datetime.datetime type for target_time. Got {type(target_time)}"
|
|
91
|
+
)
|
|
92
|
+
|
|
93
|
+
def poke(self, context: Context) -> bool:
|
|
94
|
+
self.log.info("Checking if the time (%s) has come", self.target_time)
|
|
95
|
+
return timezone.utcnow() > timezone.parse(self.target_time)
|
|
96
|
+
|
|
97
|
+
|
|
98
|
+
class DateTimeSensorAsync(DateTimeSensor):
|
|
99
|
+
"""
|
|
100
|
+
Wait until the specified datetime occurs.
|
|
101
|
+
|
|
102
|
+
Deferring itself to avoid taking up a worker slot while it is waiting.
|
|
103
|
+
It is a drop-in replacement for DateTimeSensor.
|
|
104
|
+
|
|
105
|
+
:param target_time: datetime after which the job succeeds. (templated)
|
|
106
|
+
:param start_from_trigger: Start the task directly from the triggerer without going into the worker.
|
|
107
|
+
:param trigger_kwargs: The keyword arguments passed to the trigger when start_from_trigger is set to True
|
|
108
|
+
during dynamic task mapping. This argument is not used in standard usage.
|
|
109
|
+
:param end_from_trigger: End the task directly from the triggerer without going into the worker.
|
|
110
|
+
"""
|
|
111
|
+
|
|
112
|
+
start_trigger_args = StartTriggerArgs(
|
|
113
|
+
trigger_cls="airflow.triggers.temporal.DateTimeTrigger",
|
|
114
|
+
trigger_kwargs={"moment": "", "end_from_trigger": False},
|
|
115
|
+
next_method="execute_complete",
|
|
116
|
+
next_kwargs=None,
|
|
117
|
+
timeout=None,
|
|
118
|
+
)
|
|
119
|
+
start_from_trigger = False
|
|
120
|
+
|
|
121
|
+
def __init__(
|
|
122
|
+
self,
|
|
123
|
+
*,
|
|
124
|
+
start_from_trigger: bool = False,
|
|
125
|
+
end_from_trigger: bool = False,
|
|
126
|
+
trigger_kwargs: dict[str, Any] | None = None,
|
|
127
|
+
**kwargs,
|
|
128
|
+
) -> None:
|
|
129
|
+
super().__init__(**kwargs)
|
|
130
|
+
self.end_from_trigger = end_from_trigger
|
|
131
|
+
|
|
132
|
+
self.start_from_trigger = start_from_trigger
|
|
133
|
+
if self.start_from_trigger:
|
|
134
|
+
self.start_trigger_args.trigger_kwargs = dict(
|
|
135
|
+
moment=timezone.parse(self.target_time),
|
|
136
|
+
end_from_trigger=self.end_from_trigger,
|
|
137
|
+
)
|
|
138
|
+
|
|
139
|
+
def execute(self, context: Context) -> NoReturn:
|
|
140
|
+
self.defer(
|
|
141
|
+
method_name="execute_complete",
|
|
142
|
+
trigger=DateTimeTrigger(
|
|
143
|
+
moment=timezone.parse(self.target_time),
|
|
144
|
+
end_from_trigger=self.end_from_trigger,
|
|
145
|
+
)
|
|
146
|
+
if AIRFLOW_V_3_0_PLUS
|
|
147
|
+
else DateTimeTrigger(moment=timezone.parse(self.target_time)),
|
|
148
|
+
)
|
|
149
|
+
|
|
150
|
+
def execute_complete(self, context: Context, event: Any = None) -> None:
|
|
151
|
+
"""Handle the event when the trigger fires and return immediately."""
|
|
152
|
+
return None
|
|
@@ -0,0 +1,80 @@
|
|
|
1
|
+
#
|
|
2
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
3
|
+
# or more contributor license agreements. See the NOTICE file
|
|
4
|
+
# distributed with this work for additional information
|
|
5
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
6
|
+
# to you under the Apache License, Version 2.0 (the
|
|
7
|
+
# "License"); you may not use this file except in compliance
|
|
8
|
+
# with the License. You may obtain a copy of the License at
|
|
9
|
+
#
|
|
10
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
11
|
+
#
|
|
12
|
+
# Unless required by applicable law or agreed to in writing,
|
|
13
|
+
# software distributed under the License is distributed on an
|
|
14
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
15
|
+
# KIND, either express or implied. See the License for the
|
|
16
|
+
# specific language governing permissions and limitations
|
|
17
|
+
# under the License.
|
|
18
|
+
from __future__ import annotations
|
|
19
|
+
|
|
20
|
+
from typing import TYPE_CHECKING, Any, Callable, Mapping, Sequence
|
|
21
|
+
|
|
22
|
+
from airflow.sensors.base import BaseSensorOperator, PokeReturnValue
|
|
23
|
+
from airflow.utils.context import context_merge
|
|
24
|
+
from airflow.utils.operator_helpers import determine_kwargs
|
|
25
|
+
|
|
26
|
+
if TYPE_CHECKING:
|
|
27
|
+
from airflow.utils.context import Context
|
|
28
|
+
|
|
29
|
+
|
|
30
|
+
class PythonSensor(BaseSensorOperator):
|
|
31
|
+
"""
|
|
32
|
+
Waits for a Python callable to return True.
|
|
33
|
+
|
|
34
|
+
User could put input argument in templates_dict
|
|
35
|
+
e.g ``templates_dict = {'start_ds': 1970}``
|
|
36
|
+
and access the argument by calling ``kwargs['templates_dict']['start_ds']``
|
|
37
|
+
in the callable
|
|
38
|
+
|
|
39
|
+
:param python_callable: A reference to an object that is callable
|
|
40
|
+
:param op_kwargs: a dictionary of keyword arguments that will get unpacked
|
|
41
|
+
in your function
|
|
42
|
+
:param op_args: a list of positional arguments that will get unpacked when
|
|
43
|
+
calling your callable
|
|
44
|
+
:param templates_dict: a dictionary where the values are templates that
|
|
45
|
+
will get templated by the Airflow engine sometime between
|
|
46
|
+
``__init__`` and ``execute`` takes place and are made available
|
|
47
|
+
in your callable's context after the template has been applied.
|
|
48
|
+
|
|
49
|
+
.. seealso::
|
|
50
|
+
For more information on how to use this sensor, take a look at the guide:
|
|
51
|
+
:ref:`howto/operator:PythonSensor`
|
|
52
|
+
"""
|
|
53
|
+
|
|
54
|
+
template_fields: Sequence[str] = ("templates_dict", "op_args", "op_kwargs")
|
|
55
|
+
|
|
56
|
+
def __init__(
|
|
57
|
+
self,
|
|
58
|
+
*,
|
|
59
|
+
python_callable: Callable,
|
|
60
|
+
op_args: list | None = None,
|
|
61
|
+
op_kwargs: Mapping[str, Any] | None = None,
|
|
62
|
+
templates_dict: dict | None = None,
|
|
63
|
+
**kwargs,
|
|
64
|
+
):
|
|
65
|
+
super().__init__(**kwargs)
|
|
66
|
+
self.python_callable = python_callable
|
|
67
|
+
self.op_args = op_args or []
|
|
68
|
+
self.op_kwargs = op_kwargs or {}
|
|
69
|
+
self.templates_dict = templates_dict
|
|
70
|
+
|
|
71
|
+
def poke(self, context: Context) -> PokeReturnValue | bool:
|
|
72
|
+
context_merge(context, self.op_kwargs, templates_dict=self.templates_dict)
|
|
73
|
+
self.op_kwargs = determine_kwargs(self.python_callable, self.op_args, context)
|
|
74
|
+
|
|
75
|
+
self.log.info("Poking callable: %s", str(self.python_callable))
|
|
76
|
+
return_value = self.python_callable(*self.op_args, **self.op_kwargs)
|
|
77
|
+
if isinstance(return_value, PokeReturnValue):
|
|
78
|
+
return return_value
|
|
79
|
+
else:
|
|
80
|
+
return PokeReturnValue(bool(return_value))
|
|
@@ -0,0 +1,130 @@
|
|
|
1
|
+
#
|
|
2
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
3
|
+
# or more contributor license agreements. See the NOTICE file
|
|
4
|
+
# distributed with this work for additional information
|
|
5
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
6
|
+
# to you under the Apache License, Version 2.0 (the
|
|
7
|
+
# "License"); you may not use this file except in compliance
|
|
8
|
+
# with the License. You may obtain a copy of the License at
|
|
9
|
+
#
|
|
10
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
11
|
+
#
|
|
12
|
+
# Unless required by applicable law or agreed to in writing,
|
|
13
|
+
# software distributed under the License is distributed on an
|
|
14
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
15
|
+
# KIND, either express or implied. See the License for the
|
|
16
|
+
# specific language governing permissions and limitations
|
|
17
|
+
# under the License.
|
|
18
|
+
from __future__ import annotations
|
|
19
|
+
|
|
20
|
+
import datetime
|
|
21
|
+
from dataclasses import dataclass
|
|
22
|
+
from typing import TYPE_CHECKING, Any, NoReturn
|
|
23
|
+
|
|
24
|
+
from airflow.providers.standard.utils.version_references import AIRFLOW_V_2_10_PLUS
|
|
25
|
+
from airflow.sensors.base import BaseSensorOperator
|
|
26
|
+
|
|
27
|
+
try:
|
|
28
|
+
from airflow.triggers.base import StartTriggerArgs
|
|
29
|
+
except ImportError:
|
|
30
|
+
# TODO: Remove this when min airflow version is 2.10.0 for standard provider
|
|
31
|
+
@dataclass
|
|
32
|
+
class StartTriggerArgs: # type: ignore[no-redef]
|
|
33
|
+
"""Arguments required for start task execution from triggerer."""
|
|
34
|
+
|
|
35
|
+
trigger_cls: str
|
|
36
|
+
next_method: str
|
|
37
|
+
trigger_kwargs: dict[str, Any] | None = None
|
|
38
|
+
next_kwargs: dict[str, Any] | None = None
|
|
39
|
+
timeout: datetime.timedelta | None = None
|
|
40
|
+
|
|
41
|
+
|
|
42
|
+
from airflow.triggers.temporal import DateTimeTrigger
|
|
43
|
+
from airflow.utils import timezone
|
|
44
|
+
|
|
45
|
+
if TYPE_CHECKING:
|
|
46
|
+
from airflow.utils.context import Context
|
|
47
|
+
|
|
48
|
+
|
|
49
|
+
class TimeSensor(BaseSensorOperator):
|
|
50
|
+
"""
|
|
51
|
+
Waits until the specified time of the day.
|
|
52
|
+
|
|
53
|
+
:param target_time: time after which the job succeeds
|
|
54
|
+
|
|
55
|
+
.. seealso::
|
|
56
|
+
For more information on how to use this sensor, take a look at the guide:
|
|
57
|
+
:ref:`howto/operator:TimeSensor`
|
|
58
|
+
|
|
59
|
+
"""
|
|
60
|
+
|
|
61
|
+
def __init__(self, *, target_time: datetime.time, **kwargs) -> None:
|
|
62
|
+
super().__init__(**kwargs)
|
|
63
|
+
self.target_time = target_time
|
|
64
|
+
|
|
65
|
+
def poke(self, context: Context) -> bool:
|
|
66
|
+
self.log.info("Checking if the time (%s) has come", self.target_time)
|
|
67
|
+
return timezone.make_naive(timezone.utcnow(), self.dag.timezone).time() > self.target_time
|
|
68
|
+
|
|
69
|
+
|
|
70
|
+
class TimeSensorAsync(BaseSensorOperator):
|
|
71
|
+
"""
|
|
72
|
+
Waits until the specified time of the day.
|
|
73
|
+
|
|
74
|
+
This frees up a worker slot while it is waiting.
|
|
75
|
+
|
|
76
|
+
:param target_time: time after which the job succeeds
|
|
77
|
+
:param start_from_trigger: Start the task directly from the triggerer without going into the worker.
|
|
78
|
+
:param end_from_trigger: End the task directly from the triggerer without going into the worker.
|
|
79
|
+
:param trigger_kwargs: The keyword arguments passed to the trigger when start_from_trigger is set to True
|
|
80
|
+
during dynamic task mapping. This argument is not used in standard usage.
|
|
81
|
+
|
|
82
|
+
.. seealso::
|
|
83
|
+
For more information on how to use this sensor, take a look at the guide:
|
|
84
|
+
:ref:`howto/operator:TimeSensorAsync`
|
|
85
|
+
"""
|
|
86
|
+
|
|
87
|
+
start_trigger_args = StartTriggerArgs(
|
|
88
|
+
trigger_cls="airflow.triggers.temporal.DateTimeTrigger",
|
|
89
|
+
trigger_kwargs={"moment": "", "end_from_trigger": False},
|
|
90
|
+
next_method="execute_complete",
|
|
91
|
+
next_kwargs=None,
|
|
92
|
+
timeout=None,
|
|
93
|
+
)
|
|
94
|
+
start_from_trigger = False
|
|
95
|
+
|
|
96
|
+
def __init__(
|
|
97
|
+
self,
|
|
98
|
+
*,
|
|
99
|
+
target_time: datetime.time,
|
|
100
|
+
start_from_trigger: bool = False,
|
|
101
|
+
trigger_kwargs: dict[str, Any] | None = None,
|
|
102
|
+
end_from_trigger: bool = False,
|
|
103
|
+
**kwargs,
|
|
104
|
+
) -> None:
|
|
105
|
+
super().__init__(**kwargs)
|
|
106
|
+
self.start_from_trigger = start_from_trigger
|
|
107
|
+
self.end_from_trigger = end_from_trigger
|
|
108
|
+
self.target_time = target_time
|
|
109
|
+
|
|
110
|
+
aware_time = timezone.coerce_datetime(
|
|
111
|
+
datetime.datetime.combine(datetime.datetime.today(), self.target_time, self.dag.timezone)
|
|
112
|
+
)
|
|
113
|
+
|
|
114
|
+
self.target_datetime = timezone.convert_to_utc(aware_time)
|
|
115
|
+
if self.start_from_trigger:
|
|
116
|
+
self.start_trigger_args.trigger_kwargs = dict(
|
|
117
|
+
moment=self.target_datetime, end_from_trigger=self.end_from_trigger
|
|
118
|
+
)
|
|
119
|
+
|
|
120
|
+
def execute(self, context: Context) -> NoReturn:
|
|
121
|
+
self.defer(
|
|
122
|
+
trigger=DateTimeTrigger(moment=self.target_datetime, end_from_trigger=self.end_from_trigger)
|
|
123
|
+
if AIRFLOW_V_2_10_PLUS
|
|
124
|
+
else DateTimeTrigger(moment=self.target_datetime),
|
|
125
|
+
method_name="execute_complete",
|
|
126
|
+
)
|
|
127
|
+
|
|
128
|
+
def execute_complete(self, context: Context, event: Any = None) -> None:
|
|
129
|
+
"""Handle the event when the trigger fires and return immediately."""
|
|
130
|
+
return None
|