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.

Files changed (28) hide show
  1. airflow/providers/standard/LICENSE +253 -0
  2. airflow/providers/standard/__init__.py +39 -0
  3. airflow/providers/standard/get_provider_info.py +95 -0
  4. airflow/providers/standard/hooks/__init__.py +16 -0
  5. airflow/providers/standard/hooks/filesystem.py +89 -0
  6. airflow/providers/standard/hooks/package_index.py +95 -0
  7. airflow/providers/standard/hooks/subprocess.py +119 -0
  8. airflow/providers/standard/operators/__init__.py +16 -0
  9. airflow/providers/standard/operators/bash.py +304 -0
  10. airflow/providers/standard/operators/datetime.py +110 -0
  11. airflow/providers/standard/operators/generic_transfer.py +133 -0
  12. airflow/providers/standard/operators/python.py +1172 -0
  13. airflow/providers/standard/operators/weekday.py +120 -0
  14. airflow/providers/standard/sensors/__init__.py +16 -0
  15. airflow/providers/standard/sensors/bash.py +114 -0
  16. airflow/providers/standard/sensors/date_time.py +152 -0
  17. airflow/providers/standard/sensors/python.py +80 -0
  18. airflow/providers/standard/sensors/time.py +130 -0
  19. airflow/providers/standard/sensors/time_delta.py +134 -0
  20. airflow/providers/standard/sensors/weekday.py +104 -0
  21. airflow/providers/standard/utils/__init__.py +16 -0
  22. airflow/providers/standard/utils/python_virtualenv.py +209 -0
  23. airflow/providers/standard/utils/python_virtualenv_script.jinja2 +101 -0
  24. airflow/providers/standard/utils/version_references.py +26 -0
  25. apache_airflow_providers_standard-0.0.1.dist-info/METADATA +112 -0
  26. apache_airflow_providers_standard-0.0.1.dist-info/RECORD +28 -0
  27. apache_airflow_providers_standard-0.0.1.dist-info/WHEEL +4 -0
  28. apache_airflow_providers_standard-0.0.1.dist-info/entry_points.txt +3 -0
@@ -0,0 +1,119 @@
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.
17
+ from __future__ import annotations
18
+
19
+ import contextlib
20
+ import os
21
+ import signal
22
+ from collections import namedtuple
23
+ from subprocess import PIPE, STDOUT, Popen
24
+ from tempfile import TemporaryDirectory, gettempdir
25
+ from typing import Iterator
26
+
27
+ from airflow.hooks.base import BaseHook
28
+
29
+ SubprocessResult = namedtuple("SubprocessResult", ["exit_code", "output"])
30
+
31
+
32
+ @contextlib.contextmanager
33
+ def working_directory(cwd: str | None = None) -> Iterator[str]:
34
+ """
35
+ Context manager for handling (temporary) working directory.
36
+
37
+ Use the given cwd as working directory, if provided.
38
+ Otherwise, create a temporary directory.
39
+ """
40
+ with contextlib.ExitStack() as stack:
41
+ if cwd is None:
42
+ cwd = stack.enter_context(TemporaryDirectory(prefix="airflowtmp"))
43
+ yield cwd
44
+
45
+
46
+ class SubprocessHook(BaseHook):
47
+ """Hook for running processes with the ``subprocess`` module."""
48
+
49
+ def __init__(self, **kwargs) -> None:
50
+ self.sub_process: Popen[bytes] | None = None
51
+ super().__init__(**kwargs)
52
+
53
+ def run_command(
54
+ self,
55
+ command: list[str],
56
+ env: dict[str, str] | None = None,
57
+ output_encoding: str = "utf-8",
58
+ cwd: str | None = None,
59
+ ) -> SubprocessResult:
60
+ """
61
+ Execute the command.
62
+
63
+ If ``cwd`` is None, execute the command in a temporary directory which will be cleaned afterwards.
64
+ If ``env`` is not supplied, ``os.environ`` is passed
65
+
66
+ :param command: the command to run
67
+ :param env: Optional dict containing environment variables to be made available to the shell
68
+ environment in which ``command`` will be executed. If omitted, ``os.environ`` will be used.
69
+ Note, that in case you have Sentry configured, original variables from the environment
70
+ will also be passed to the subprocess with ``SUBPROCESS_`` prefix. See:
71
+ https://airflow.apache.org/docs/apache-airflow/stable/administration-and-deployment/logging-monitoring/errors.html for details.
72
+ :param output_encoding: encoding to use for decoding stdout
73
+ :param cwd: Working directory to run the command in.
74
+ If None (default), the command is run in a temporary directory.
75
+ :return: :class:`namedtuple` containing ``exit_code`` and ``output``, the last line from stderr
76
+ or stdout
77
+ """
78
+ self.log.info("Tmp dir root location: %s", gettempdir())
79
+ with working_directory(cwd=cwd) as cwd:
80
+
81
+ def pre_exec():
82
+ # Restore default signal disposition and invoke setsid
83
+ for sig in ("SIGPIPE", "SIGXFZ", "SIGXFSZ"):
84
+ if hasattr(signal, sig):
85
+ signal.signal(getattr(signal, sig), signal.SIG_DFL)
86
+ os.setsid()
87
+
88
+ self.log.info("Running command: %s", command)
89
+
90
+ self.sub_process = Popen(
91
+ command,
92
+ stdout=PIPE,
93
+ stderr=STDOUT,
94
+ cwd=cwd,
95
+ env=env if env or env == {} else os.environ,
96
+ preexec_fn=pre_exec,
97
+ )
98
+
99
+ self.log.info("Output:")
100
+ line = ""
101
+ if self.sub_process is None:
102
+ raise RuntimeError("The subprocess should be created here and is None!")
103
+ if self.sub_process.stdout is not None:
104
+ for raw_line in iter(self.sub_process.stdout.readline, b""):
105
+ line = raw_line.decode(output_encoding, errors="backslashreplace").rstrip()
106
+ self.log.info("%s", line)
107
+
108
+ self.sub_process.wait()
109
+
110
+ self.log.info("Command exited with return code %s", self.sub_process.returncode)
111
+ return_code: int = self.sub_process.returncode
112
+
113
+ return SubprocessResult(exit_code=return_code, output=line)
114
+
115
+ def send_sigterm(self):
116
+ """Send SIGTERM signal to ``self.sub_process`` if one exists."""
117
+ self.log.info("Sending SIGTERM signal to process group")
118
+ if self.sub_process and hasattr(self.sub_process, "pid"):
119
+ os.killpg(os.getpgid(self.sub_process.pid), signal.SIGTERM)
@@ -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,304 @@
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
+ import shutil
22
+ import tempfile
23
+ import warnings
24
+ from functools import cached_property
25
+ from typing import TYPE_CHECKING, Any, Callable, Container, Sequence, cast
26
+
27
+ from airflow.exceptions import AirflowException, AirflowSkipException
28
+ from airflow.models.baseoperator import BaseOperator
29
+ from airflow.providers.standard.hooks.subprocess import SubprocessHook, SubprocessResult, working_directory
30
+ from airflow.utils.operator_helpers import context_to_airflow_vars
31
+ from airflow.utils.types import ArgNotSet
32
+
33
+ if TYPE_CHECKING:
34
+ from airflow.models.taskinstance import TaskInstance
35
+ from airflow.utils.context import Context
36
+
37
+
38
+ class BashOperator(BaseOperator):
39
+ r"""
40
+ Execute a Bash script, command or set of commands.
41
+
42
+ .. seealso::
43
+ For more information on how to use this operator, take a look at the guide:
44
+ :ref:`howto/operator:BashOperator`
45
+
46
+ If BaseOperator.do_xcom_push is True, the last line written to stdout
47
+ will also be pushed to an XCom when the bash command completes
48
+
49
+ :param bash_command: The command, set of commands or reference to a
50
+ Bash script (must be '.sh' or '.bash') to be executed. (templated)
51
+ :param env: If env is not None, it must be a dict that defines the
52
+ environment variables for the new process; these are used instead
53
+ of inheriting the current process environment, which is the default
54
+ behavior. (templated)
55
+ :param append_env: If False(default) uses the environment variables passed in env params
56
+ and does not inherit the current process environment. If True, inherits the environment variables
57
+ from current passes and then environment variable passed by the user will either update the existing
58
+ inherited environment variables or the new variables gets appended to it
59
+ :param output_encoding: Output encoding of Bash command
60
+ :param skip_on_exit_code: If task exits with this exit code, leave the task
61
+ in ``skipped`` state (default: 99). If set to ``None``, any non-zero
62
+ exit code will be treated as a failure.
63
+ :param cwd: Working directory to execute the command in (templated).
64
+ If None (default), the command is run in a temporary directory.
65
+ To use current DAG folder as the working directory,
66
+ you might set template ``{{ dag_run.dag.folder }}``.
67
+ When bash_command is a '.sh' or '.bash' file, Airflow must have write
68
+ access to the working directory. The script will be rendered (Jinja
69
+ template) into a new temporary file in this directory.
70
+ :param output_processor: Function to further process the output of the bash script
71
+ (default is lambda output: output).
72
+
73
+ Airflow will evaluate the exit code of the Bash command. In general, a non-zero exit code will result in
74
+ task failure and zero will result in task success.
75
+ Exit code ``99`` (or another set in ``skip_on_exit_code``)
76
+ will throw an :class:`airflow.exceptions.AirflowSkipException`, which will leave the task in ``skipped``
77
+ state. You can have all non-zero exit codes be treated as a failure by setting ``skip_on_exit_code=None``.
78
+
79
+ .. list-table::
80
+ :widths: 25 25
81
+ :header-rows: 1
82
+
83
+ * - Exit code
84
+ - Behavior
85
+ * - 0
86
+ - success
87
+ * - `skip_on_exit_code` (default: 99)
88
+ - raise :class:`airflow.exceptions.AirflowSkipException`
89
+ * - otherwise
90
+ - raise :class:`airflow.exceptions.AirflowException`
91
+
92
+ .. note::
93
+
94
+ Airflow will not recognize a non-zero exit code unless the whole shell exit with a non-zero exit
95
+ code. This can be an issue if the non-zero exit arises from a sub-command. The easiest way of
96
+ addressing this is to prefix the command with ``set -e;``
97
+
98
+ .. code-block:: python
99
+
100
+ bash_command = "set -e; python3 script.py '{{ next_execution_date }}'"
101
+
102
+ .. note::
103
+
104
+ To simply execute a ``.sh`` or ``.bash`` script (without any Jinja template), add a space after the
105
+ script name ``bash_command`` argument -- for example ``bash_command="my_script.sh "``. This
106
+ is because Airflow tries to load this file and process it as a Jinja template when
107
+ it ends with ``.sh`` or ``.bash``.
108
+
109
+ If you have Jinja template in your script, do not put any blank space. And add the script's directory
110
+ in the DAG's ``template_searchpath``. If you specify a ``cwd``, Airflow must have write access to
111
+ this directory. The script will be rendered (Jinja template) into a new temporary file in this directory.
112
+
113
+ .. warning::
114
+
115
+ Care should be taken with "user" input or when using Jinja templates in the
116
+ ``bash_command``, as this bash operator does not perform any escaping or
117
+ sanitization of the command.
118
+
119
+ This applies mostly to using "dag_run" conf, as that can be submitted via
120
+ users in the Web UI. Most of the default template variables are not at
121
+ risk.
122
+
123
+ For example, do **not** do this:
124
+
125
+ .. code-block:: python
126
+
127
+ bash_task = BashOperator(
128
+ task_id="bash_task",
129
+ bash_command='echo "Here is the message: \'{{ dag_run.conf["message"] if dag_run else "" }}\'"',
130
+ )
131
+
132
+ Instead, you should pass this via the ``env`` kwarg and use double-quotes
133
+ inside the bash_command, as below:
134
+
135
+ .. code-block:: python
136
+
137
+ bash_task = BashOperator(
138
+ task_id="bash_task",
139
+ bash_command="echo \"here is the message: '$message'\"",
140
+ env={"message": '{{ dag_run.conf["message"] if dag_run else "" }}'},
141
+ )
142
+
143
+ .. versionadded:: 2.10.0
144
+ The `output_processor` parameter.
145
+
146
+ """
147
+
148
+ template_fields: Sequence[str] = ("bash_command", "env", "cwd")
149
+ template_fields_renderers = {"bash_command": "bash", "env": "json"}
150
+ template_ext: Sequence[str] = (".sh", ".bash")
151
+ ui_color = "#f0ede4"
152
+
153
+ def __init__(
154
+ self,
155
+ *,
156
+ bash_command: str | ArgNotSet,
157
+ env: dict[str, str] | None = None,
158
+ append_env: bool = False,
159
+ output_encoding: str = "utf-8",
160
+ skip_exit_code: int | None = None,
161
+ skip_on_exit_code: int | Container[int] | None = 99,
162
+ cwd: str | None = None,
163
+ output_processor: Callable[[str], Any] = lambda result: result,
164
+ **kwargs,
165
+ ) -> None:
166
+ super().__init__(**kwargs)
167
+ self.bash_command = bash_command
168
+ self.env = env
169
+ self.output_encoding = output_encoding
170
+ if skip_exit_code is not None:
171
+ warnings.warn(
172
+ "skip_exit_code is deprecated. Please use skip_on_exit_code", DeprecationWarning, stacklevel=2
173
+ )
174
+ skip_on_exit_code = skip_exit_code
175
+ self.skip_on_exit_code = (
176
+ skip_on_exit_code
177
+ if isinstance(skip_on_exit_code, Container)
178
+ else [skip_on_exit_code]
179
+ if skip_on_exit_code is not None
180
+ else []
181
+ )
182
+ self.cwd = cwd
183
+ self.append_env = append_env
184
+ self.output_processor = output_processor
185
+
186
+ # When using the @task.bash decorator, the Bash command is not known until the underlying Python
187
+ # callable is executed and therefore set to NOTSET initially. This flag is useful during execution to
188
+ # determine whether the bash_command value needs to re-rendered.
189
+ self._init_bash_command_not_set = isinstance(self.bash_command, ArgNotSet)
190
+
191
+ # Keep a copy of the original bash_command, without the Jinja template rendered.
192
+ # This is later used to determine if the bash_command is a script or an inline string command.
193
+ # We do this later, because the bash_command is not available in __init__ when using @task.bash.
194
+ self._unrendered_bash_command: str | ArgNotSet = bash_command
195
+
196
+ @cached_property
197
+ def subprocess_hook(self):
198
+ """Returns hook for running the bash command."""
199
+ return SubprocessHook()
200
+
201
+ @staticmethod
202
+ def refresh_bash_command(ti: TaskInstance) -> None:
203
+ """
204
+ Rewrite the underlying rendered bash_command value for a task instance in the metadatabase.
205
+
206
+ TaskInstance.get_rendered_template_fields() cannot be used because this will retrieve the
207
+ RenderedTaskInstanceFields from the metadatabase which doesn't have the runtime-evaluated bash_command
208
+ value.
209
+
210
+ :meta private:
211
+ """
212
+ from airflow.models.renderedtifields import RenderedTaskInstanceFields
213
+
214
+ RenderedTaskInstanceFields._update_runtime_evaluated_template_fields(ti)
215
+
216
+ def get_env(self, context) -> dict:
217
+ """Build the set of environment variables to be exposed for the bash command."""
218
+ system_env = os.environ.copy()
219
+ env = self.env
220
+ if env is None:
221
+ env = system_env
222
+ else:
223
+ if self.append_env:
224
+ system_env.update(env)
225
+ env = system_env
226
+
227
+ airflow_context_vars = context_to_airflow_vars(context, in_env_var_format=True)
228
+ self.log.debug(
229
+ "Exporting env vars: %s",
230
+ " ".join(f"{k}={v!r}" for k, v in airflow_context_vars.items()),
231
+ )
232
+ env.update(airflow_context_vars)
233
+ return env
234
+
235
+ def execute(self, context: Context):
236
+ bash_path: str = shutil.which("bash") or "bash"
237
+ if self.cwd is not None:
238
+ if not os.path.exists(self.cwd):
239
+ raise AirflowException(f"Can not find the cwd: {self.cwd}")
240
+ if not os.path.isdir(self.cwd):
241
+ raise AirflowException(f"The cwd {self.cwd} must be a directory")
242
+ env = self.get_env(context)
243
+
244
+ # Because the bash_command value is evaluated at runtime using the @task.bash decorator, the
245
+ # RenderedTaskInstanceField data needs to be rewritten and the bash_command value re-rendered -- the
246
+ # latter because the returned command from the decorated callable could contain a Jinja expression.
247
+ # Both will ensure the correct Bash command is executed and that the Rendered Template view in the UI
248
+ # displays the executed command (otherwise it will display as an ArgNotSet type).
249
+ if self._init_bash_command_not_set:
250
+ is_inline_command = self._is_inline_command(bash_command=cast(str, self.bash_command))
251
+ ti = cast("TaskInstance", context["ti"])
252
+ self.refresh_bash_command(ti)
253
+ else:
254
+ is_inline_command = self._is_inline_command(bash_command=cast(str, self._unrendered_bash_command))
255
+
256
+ if is_inline_command:
257
+ result = self._run_inline_command(bash_path=bash_path, env=env)
258
+ else:
259
+ result = self._run_rendered_script_file(bash_path=bash_path, env=env)
260
+
261
+ if result.exit_code in self.skip_on_exit_code:
262
+ raise AirflowSkipException(f"Bash command returned exit code {result.exit_code}. Skipping.")
263
+ elif result.exit_code != 0:
264
+ raise AirflowException(
265
+ f"Bash command failed. The command returned a non-zero exit code {result.exit_code}."
266
+ )
267
+
268
+ return self.output_processor(result.output)
269
+
270
+ def _run_inline_command(self, bash_path: str, env: dict) -> SubprocessResult:
271
+ """Pass the bash command as string directly in the subprocess."""
272
+ return self.subprocess_hook.run_command(
273
+ command=[bash_path, "-c", self.bash_command],
274
+ env=env,
275
+ output_encoding=self.output_encoding,
276
+ cwd=self.cwd,
277
+ )
278
+
279
+ def _run_rendered_script_file(self, bash_path: str, env: dict) -> SubprocessResult:
280
+ """
281
+ Save the bash command into a file and execute this file.
282
+
283
+ This allows for longer commands, and prevents "Argument list too long error".
284
+ """
285
+ with working_directory(cwd=self.cwd) as cwd:
286
+ with tempfile.NamedTemporaryFile(mode="w", dir=cwd, suffix=".sh") as file:
287
+ file.write(cast(str, self.bash_command))
288
+ file.flush()
289
+
290
+ bash_script = os.path.basename(file.name)
291
+ return self.subprocess_hook.run_command(
292
+ command=[bash_path, bash_script],
293
+ env=env,
294
+ output_encoding=self.output_encoding,
295
+ cwd=cwd,
296
+ )
297
+
298
+ @classmethod
299
+ def _is_inline_command(cls, bash_command: str) -> bool:
300
+ """Return True if the bash command is an inline string. False if it's a bash script file."""
301
+ return not bash_command.endswith(tuple(cls.template_ext))
302
+
303
+ def on_kill(self) -> None:
304
+ self.subprocess_hook.send_sigterm()
@@ -0,0 +1,110 @@
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.
17
+ from __future__ import annotations
18
+
19
+ import datetime
20
+ from typing import TYPE_CHECKING, Iterable
21
+
22
+ from airflow.exceptions import AirflowException
23
+ from airflow.operators.branch import BaseBranchOperator
24
+ from airflow.utils import timezone
25
+
26
+ if TYPE_CHECKING:
27
+ from airflow.utils.context import Context
28
+
29
+
30
+ class BranchDateTimeOperator(BaseBranchOperator):
31
+ """
32
+ Branches into one of two lists of tasks depending on the current datetime.
33
+
34
+ For more information on how to use this operator, take a look at the guide:
35
+ :ref:`howto/operator:BranchDateTimeOperator`.
36
+
37
+ True branch will be returned when ``datetime.datetime.now()`` falls below
38
+ ``target_upper`` and above ``target_lower``.
39
+
40
+ :param follow_task_ids_if_true: task_id, task_group_id, or a list of task_ids and/or task_group_ids
41
+ to follow if ``datetime.datetime.now()`` falls above target_lower and below target_upper.
42
+ :param follow_task_ids_if_false: task_id, task_group_id, or a list of task_ids and/or task_group_ids
43
+ to follow if ``datetime.datetime.now()`` falls below target_lower or above target_upper.
44
+ :param target_lower: target lower bound.
45
+ :param target_upper: target upper bound.
46
+ :param use_task_logical_date: If ``True``, uses task's logical date to compare with targets.
47
+ Execution date is useful for backfilling. If ``False``, uses system's date.
48
+ """
49
+
50
+ def __init__(
51
+ self,
52
+ *,
53
+ follow_task_ids_if_true: str | Iterable[str],
54
+ follow_task_ids_if_false: str | Iterable[str],
55
+ target_lower: datetime.datetime | datetime.time | None,
56
+ target_upper: datetime.datetime | datetime.time | None,
57
+ use_task_logical_date: bool = False,
58
+ **kwargs,
59
+ ) -> None:
60
+ super().__init__(**kwargs)
61
+ if target_lower is None and target_upper is None:
62
+ raise AirflowException(
63
+ "Both target_upper and target_lower are None. At least one "
64
+ "must be defined to be compared to the current datetime"
65
+ )
66
+
67
+ self.target_lower = target_lower
68
+ self.target_upper = target_upper
69
+ self.follow_task_ids_if_true = follow_task_ids_if_true
70
+ self.follow_task_ids_if_false = follow_task_ids_if_false
71
+ self.use_task_logical_date = use_task_logical_date
72
+
73
+ def choose_branch(self, context: Context) -> str | Iterable[str]:
74
+ if self.use_task_logical_date:
75
+ now = context["logical_date"]
76
+ else:
77
+ now = timezone.coerce_datetime(timezone.utcnow())
78
+ lower, upper = target_times_as_dates(now, self.target_lower, self.target_upper)
79
+ lower = timezone.coerce_datetime(lower, self.dag.timezone)
80
+ upper = timezone.coerce_datetime(upper, self.dag.timezone)
81
+
82
+ if upper is not None and upper < now:
83
+ return self.follow_task_ids_if_false
84
+
85
+ if lower is not None and lower > now:
86
+ return self.follow_task_ids_if_false
87
+
88
+ return self.follow_task_ids_if_true
89
+
90
+
91
+ def target_times_as_dates(
92
+ base_date: datetime.datetime,
93
+ lower: datetime.datetime | datetime.time | None,
94
+ upper: datetime.datetime | datetime.time | None,
95
+ ):
96
+ """Ensure upper and lower time targets are datetimes by combining them with base_date."""
97
+ if isinstance(lower, datetime.datetime) and isinstance(upper, datetime.datetime):
98
+ return lower, upper
99
+
100
+ if lower is not None and isinstance(lower, datetime.time):
101
+ lower = datetime.datetime.combine(base_date, lower)
102
+ if upper is not None and isinstance(upper, datetime.time):
103
+ upper = datetime.datetime.combine(base_date, upper)
104
+
105
+ if lower is None or upper is None:
106
+ return lower, upper
107
+
108
+ if upper < lower:
109
+ upper += datetime.timedelta(days=1)
110
+ return lower, upper
@@ -0,0 +1,133 @@
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, Sequence
21
+
22
+ from airflow.hooks.base import BaseHook
23
+ from airflow.models import BaseOperator
24
+
25
+ if TYPE_CHECKING:
26
+ from airflow.utils.context import Context
27
+
28
+
29
+ class GenericTransfer(BaseOperator):
30
+ """
31
+ Moves data from a connection to another.
32
+
33
+ Assuming that they both provide the required methods in their respective hooks.
34
+ The source hook needs to expose a `get_records` method, and the destination a
35
+ `insert_rows` method.
36
+
37
+ This is meant to be used on small-ish datasets that fit in memory.
38
+
39
+ :param sql: SQL query to execute against the source database. (templated)
40
+ :param destination_table: target table. (templated)
41
+ :param source_conn_id: source connection. (templated)
42
+ :param destination_conn_id: destination connection. (templated)
43
+ :param preoperator: sql statement or list of statements to be
44
+ executed prior to loading the data. (templated)
45
+ :param insert_args: extra params for `insert_rows` method.
46
+ """
47
+
48
+ template_fields: Sequence[str] = (
49
+ "source_conn_id",
50
+ "destination_conn_id",
51
+ "sql",
52
+ "destination_table",
53
+ "preoperator",
54
+ "insert_args",
55
+ )
56
+ template_ext: Sequence[str] = (
57
+ ".sql",
58
+ ".hql",
59
+ )
60
+ template_fields_renderers = {"preoperator": "sql"}
61
+ ui_color = "#b0f07c"
62
+
63
+ def __init__(
64
+ self,
65
+ *,
66
+ sql: str,
67
+ destination_table: str,
68
+ source_conn_id: str,
69
+ source_hook_params: dict | None = None,
70
+ destination_conn_id: str,
71
+ destination_hook_params: dict | None = None,
72
+ preoperator: str | list[str] | None = None,
73
+ insert_args: dict | None = None,
74
+ **kwargs,
75
+ ) -> None:
76
+ super().__init__(**kwargs)
77
+ self.sql = sql
78
+ self.destination_table = destination_table
79
+ self.source_conn_id = source_conn_id
80
+ self.source_hook_params = source_hook_params
81
+ self.destination_conn_id = destination_conn_id
82
+ self.destination_hook_params = destination_hook_params
83
+ self.preoperator = preoperator
84
+ self.insert_args = insert_args or {}
85
+
86
+ @classmethod
87
+ def get_hook(cls, conn_id: str, hook_params: dict | None = None) -> BaseHook:
88
+ """
89
+ Return default hook for this connection id.
90
+
91
+ :param conn_id: connection id
92
+ :param hook_params: hook parameters
93
+ :return: default hook for this connection
94
+ """
95
+ connection = BaseHook.get_connection(conn_id)
96
+ return connection.get_hook(hook_params=hook_params)
97
+
98
+ def execute(self, context: Context):
99
+ source_hook = self.get_hook(conn_id=self.source_conn_id, hook_params=self.source_hook_params)
100
+ destination_hook = self.get_hook(
101
+ conn_id=self.destination_conn_id, hook_params=self.destination_hook_params
102
+ )
103
+
104
+ self.log.info("Extracting data from %s", self.source_conn_id)
105
+ self.log.info("Executing: \n %s", self.sql)
106
+ get_records = getattr(source_hook, "get_records", None)
107
+ if not callable(get_records):
108
+ raise RuntimeError(
109
+ f"Hook for connection {self.source_conn_id!r} "
110
+ f"({type(source_hook).__name__}) has no `get_records` method"
111
+ )
112
+ else:
113
+ results = get_records(self.sql)
114
+
115
+ if self.preoperator:
116
+ run = getattr(destination_hook, "run", None)
117
+ if not callable(run):
118
+ raise RuntimeError(
119
+ f"Hook for connection {self.destination_conn_id!r} "
120
+ f"({type(destination_hook).__name__}) has no `run` method"
121
+ )
122
+ self.log.info("Running preoperator")
123
+ self.log.info(self.preoperator)
124
+ run(self.preoperator)
125
+
126
+ insert_rows = getattr(destination_hook, "insert_rows", None)
127
+ if not callable(insert_rows):
128
+ raise RuntimeError(
129
+ f"Hook for connection {self.destination_conn_id!r} "
130
+ f"({type(destination_hook).__name__}) has no `insert_rows` method"
131
+ )
132
+ self.log.info("Inserting rows into %s", self.destination_conn_id)
133
+ insert_rows(table=self.destination_table, rows=results, **self.insert_args)