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,1172 @@
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 inspect
21
+ import json
22
+ import logging
23
+ import os
24
+ import shutil
25
+ import subprocess
26
+ import sys
27
+ import textwrap
28
+ import types
29
+ import warnings
30
+ from abc import ABCMeta, abstractmethod
31
+ from collections.abc import Container
32
+ from functools import cache
33
+ from pathlib import Path
34
+ from tempfile import TemporaryDirectory
35
+ from typing import TYPE_CHECKING, Any, Callable, Collection, Iterable, Mapping, NamedTuple, Sequence, cast
36
+
37
+ import lazy_object_proxy
38
+
39
+ from airflow.exceptions import (
40
+ AirflowConfigException,
41
+ AirflowException,
42
+ AirflowSkipException,
43
+ DeserializingResultError,
44
+ RemovedInAirflow3Warning,
45
+ )
46
+ from airflow.models.baseoperator import BaseOperator
47
+ from airflow.models.skipmixin import SkipMixin
48
+ from airflow.models.taskinstance import _CURRENT_CONTEXT
49
+ from airflow.models.variable import Variable
50
+ from airflow.operators.branch import BranchMixIn
51
+ from airflow.providers.standard.utils.python_virtualenv import prepare_virtualenv, write_python_script
52
+ from airflow.providers.standard.utils.version_references import AIRFLOW_V_2_10_PLUS, AIRFLOW_V_3_0_PLUS
53
+ from airflow.settings import _ENABLE_AIP_44
54
+ from airflow.typing_compat import Literal
55
+ from airflow.utils import hashlib_wrapper
56
+ from airflow.utils.context import context_copy_partial, context_merge
57
+ from airflow.utils.file import get_unique_dag_module_name
58
+ from airflow.utils.operator_helpers import KeywordParameters
59
+ from airflow.utils.process_utils import execute_in_subprocess, execute_in_subprocess_with_kwargs
60
+ from airflow.utils.session import create_session
61
+
62
+ log = logging.getLogger(__name__)
63
+
64
+ if TYPE_CHECKING:
65
+ from pendulum.datetime import DateTime
66
+
67
+ from airflow.serialization.enums import Encoding
68
+ from airflow.utils.context import Context
69
+
70
+
71
+ @cache
72
+ def _parse_version_info(text: str) -> tuple[int, int, int, str, int]:
73
+ """Parse python version info from a text."""
74
+ parts = text.strip().split(".")
75
+ if len(parts) != 5:
76
+ msg = f"Invalid Python version info, expected 5 components separated by '.', but got {text!r}."
77
+ raise ValueError(msg)
78
+ try:
79
+ return int(parts[0]), int(parts[1]), int(parts[2]), parts[3], int(parts[4])
80
+ except ValueError:
81
+ msg = f"Unable to convert parts {parts} parsed from {text!r} to (int, int, int, str, int)."
82
+ raise ValueError(msg) from None
83
+
84
+
85
+ class _PythonVersionInfo(NamedTuple):
86
+ """Provide the same interface as ``sys.version_info``."""
87
+
88
+ major: int
89
+ minor: int
90
+ micro: int
91
+ releaselevel: str
92
+ serial: int
93
+
94
+ @classmethod
95
+ def from_executable(cls, executable: str) -> _PythonVersionInfo:
96
+ """Parse python version info from an executable."""
97
+ cmd = [executable, "-c", 'import sys; print(".".join(map(str, sys.version_info)))']
98
+ try:
99
+ result = subprocess.check_output(cmd, text=True)
100
+ except Exception as e:
101
+ raise ValueError(f"Error while executing command {cmd}: {e}")
102
+ return cls(*_parse_version_info(result.strip()))
103
+
104
+
105
+ class PythonOperator(BaseOperator):
106
+ """
107
+ Executes a Python callable.
108
+
109
+ .. seealso::
110
+ For more information on how to use this operator, take a look at the guide:
111
+ :ref:`howto/operator:PythonOperator`
112
+
113
+ When running your callable, Airflow will pass a set of keyword arguments that can be used in your
114
+ function. This set of kwargs correspond exactly to what you can use in your jinja templates.
115
+ For this to work, you need to define ``**kwargs`` in your function header, or you can add directly the
116
+ keyword arguments you would like to get - for example with the below code your callable will get
117
+ the values of ``ti`` and ``next_ds`` context variables.
118
+
119
+ With explicit arguments:
120
+
121
+ .. code-block:: python
122
+
123
+ def my_python_callable(ti, next_ds):
124
+ pass
125
+
126
+ With kwargs:
127
+
128
+ .. code-block:: python
129
+
130
+ def my_python_callable(**kwargs):
131
+ ti = kwargs["ti"]
132
+ next_ds = kwargs["next_ds"]
133
+
134
+
135
+ :param python_callable: A reference to an object that is callable
136
+ :param op_args: a list of positional arguments that will get unpacked when
137
+ calling your callable
138
+ :param op_kwargs: a dictionary of keyword arguments that will get unpacked
139
+ in your function
140
+ :param templates_dict: a dictionary where the values are templates that
141
+ will get templated by the Airflow engine sometime between
142
+ ``__init__`` and ``execute`` takes place and are made available
143
+ in your callable's context after the template has been applied. (templated)
144
+ :param templates_exts: a list of file extensions to resolve while
145
+ processing templated fields, for examples ``['.sql', '.hql']``
146
+ :param show_return_value_in_logs: a bool value whether to show return_value
147
+ logs. Defaults to True, which allows return value log output.
148
+ It can be set to False to prevent log output of return value when you return huge data
149
+ such as transmission a large amount of XCom to TaskAPI.
150
+ """
151
+
152
+ template_fields: Sequence[str] = ("templates_dict", "op_args", "op_kwargs")
153
+ template_fields_renderers = {"templates_dict": "json", "op_args": "py", "op_kwargs": "py"}
154
+ BLUE = "#ffefeb"
155
+ ui_color = BLUE
156
+
157
+ # since we won't mutate the arguments, we should just do the shallow copy
158
+ # there are some cases we can't deepcopy the objects(e.g protobuf).
159
+ shallow_copy_attrs: Sequence[str] = ("python_callable", "op_kwargs")
160
+
161
+ def __init__(
162
+ self,
163
+ *,
164
+ python_callable: Callable,
165
+ op_args: Collection[Any] | None = None,
166
+ op_kwargs: Mapping[str, Any] | None = None,
167
+ templates_dict: dict[str, Any] | None = None,
168
+ templates_exts: Sequence[str] | None = None,
169
+ show_return_value_in_logs: bool = True,
170
+ **kwargs,
171
+ ) -> None:
172
+ super().__init__(**kwargs)
173
+ if not callable(python_callable):
174
+ raise AirflowException("`python_callable` param must be callable")
175
+ self.python_callable = python_callable
176
+ self.op_args = op_args or ()
177
+ self.op_kwargs = op_kwargs or {}
178
+ self.templates_dict = templates_dict
179
+ if templates_exts:
180
+ self.template_ext = templates_exts
181
+ self.show_return_value_in_logs = show_return_value_in_logs
182
+
183
+ def execute(self, context: Context) -> Any:
184
+ context_merge(context, self.op_kwargs, templates_dict=self.templates_dict)
185
+ self.op_kwargs = self.determine_kwargs(context)
186
+
187
+ if AIRFLOW_V_3_0_PLUS:
188
+ from airflow.utils.context import context_get_outlet_events
189
+
190
+ self._asset_events = context_get_outlet_events(context)
191
+ elif AIRFLOW_V_2_10_PLUS:
192
+ from airflow.utils.context import context_get_outlet_events
193
+
194
+ self._dataset_events = context_get_outlet_events(context)
195
+
196
+ return_value = self.execute_callable()
197
+ if self.show_return_value_in_logs:
198
+ self.log.info("Done. Returned value was: %s", return_value)
199
+ else:
200
+ self.log.info("Done. Returned value not shown")
201
+
202
+ return return_value
203
+
204
+ def determine_kwargs(self, context: Mapping[str, Any]) -> Mapping[str, Any]:
205
+ return KeywordParameters.determine(self.python_callable, self.op_args, context).unpacking()
206
+
207
+ def execute_callable(self) -> Any:
208
+ """
209
+ Call the python callable with the given arguments.
210
+
211
+ :return: the return value of the call.
212
+ """
213
+ try:
214
+ from airflow.utils.operator_helpers import ExecutionCallableRunner
215
+
216
+ asset_events = self._asset_events if AIRFLOW_V_3_0_PLUS else self._dataset_events
217
+
218
+ runner = ExecutionCallableRunner(self.python_callable, asset_events, logger=self.log)
219
+ except ImportError:
220
+ # Handle Pre Airflow 3.10 case where ExecutionCallableRunner was not available
221
+ return self.python_callable(*self.op_args, **self.op_kwargs)
222
+ return runner.run(*self.op_args, **self.op_kwargs)
223
+
224
+
225
+ class BranchPythonOperator(PythonOperator, BranchMixIn):
226
+ """
227
+ A workflow can "branch" or follow a path after the execution of this task.
228
+
229
+ It derives the PythonOperator and expects a Python function that returns
230
+ a single task_id, a single task_group_id, or a list of task_ids and/or
231
+ task_group_ids to follow. The task_id(s) and/or task_group_id(s) returned
232
+ should point to a task or task group directly downstream from {self}. All
233
+ other "branches" or directly downstream tasks are marked with a state of
234
+ ``skipped`` so that these paths can't move forward. The ``skipped`` states
235
+ are propagated downstream to allow for the DAG state to fill up and
236
+ the DAG run's state to be inferred.
237
+ """
238
+
239
+ def execute(self, context: Context) -> Any:
240
+ return self.do_branch(context, super().execute(context))
241
+
242
+
243
+ class ShortCircuitOperator(PythonOperator, SkipMixin):
244
+ """
245
+ Allows a pipeline to continue based on the result of a ``python_callable``.
246
+
247
+ The ShortCircuitOperator is derived from the PythonOperator and evaluates the result of a
248
+ ``python_callable``. If the returned result is False or a falsy value, the pipeline will be
249
+ short-circuited. Downstream tasks will be marked with a state of "skipped" based on the short-circuiting
250
+ mode configured. If the returned result is True or a truthy value, downstream tasks proceed as normal and
251
+ an ``XCom`` of the returned result is pushed.
252
+
253
+ The short-circuiting can be configured to either respect or ignore the ``trigger_rule`` set for
254
+ downstream tasks. If ``ignore_downstream_trigger_rules`` is set to True, the default setting, all
255
+ downstream tasks are skipped without considering the ``trigger_rule`` defined for tasks. However, if this
256
+ parameter is set to False, the direct downstream tasks are skipped but the specified ``trigger_rule`` for
257
+ other subsequent downstream tasks are respected. In this mode, the operator assumes the direct downstream
258
+ tasks were purposely meant to be skipped but perhaps not other subsequent tasks.
259
+
260
+ .. seealso::
261
+ For more information on how to use this operator, take a look at the guide:
262
+ :ref:`howto/operator:ShortCircuitOperator`
263
+
264
+ :param ignore_downstream_trigger_rules: If set to True, all downstream tasks from this operator task will
265
+ be skipped. This is the default behavior. If set to False, the direct, downstream task(s) will be
266
+ skipped but the ``trigger_rule`` defined for all other downstream tasks will be respected.
267
+ """
268
+
269
+ def __init__(self, *, ignore_downstream_trigger_rules: bool = True, **kwargs) -> None:
270
+ super().__init__(**kwargs)
271
+ self.ignore_downstream_trigger_rules = ignore_downstream_trigger_rules
272
+
273
+ def execute(self, context: Context) -> Any:
274
+ condition = super().execute(context)
275
+ self.log.info("Condition result is %s", condition)
276
+
277
+ if condition:
278
+ self.log.info("Proceeding with downstream tasks...")
279
+ return condition
280
+
281
+ if not self.downstream_task_ids:
282
+ self.log.info("No downstream tasks; nothing to do.")
283
+ return condition
284
+
285
+ dag_run = context["dag_run"]
286
+
287
+ def get_tasks_to_skip():
288
+ if self.ignore_downstream_trigger_rules is True:
289
+ tasks = context["task"].get_flat_relatives(upstream=False)
290
+ else:
291
+ tasks = context["task"].get_direct_relatives(upstream=False)
292
+ for t in tasks:
293
+ if not t.is_teardown:
294
+ yield t
295
+
296
+ to_skip = get_tasks_to_skip()
297
+
298
+ # this let's us avoid an intermediate list unless debug logging
299
+ if self.log.getEffectiveLevel() <= logging.DEBUG:
300
+ self.log.debug("Downstream task IDs %s", to_skip := list(get_tasks_to_skip()))
301
+
302
+ self.log.info("Skipping downstream tasks")
303
+ if AIRFLOW_V_3_0_PLUS:
304
+ self.skip(
305
+ dag_run=dag_run,
306
+ tasks=to_skip,
307
+ map_index=context["ti"].map_index,
308
+ )
309
+ else:
310
+ self.skip(
311
+ dag_run=dag_run,
312
+ tasks=to_skip,
313
+ execution_date=cast("DateTime", dag_run.execution_date), # type: ignore[call-arg]
314
+ map_index=context["ti"].map_index,
315
+ )
316
+
317
+ self.log.info("Done.")
318
+ # returns the result of the super execute method as it is instead of returning None
319
+ return condition
320
+
321
+
322
+ def _load_pickle():
323
+ import pickle
324
+
325
+ return pickle
326
+
327
+
328
+ def _load_dill():
329
+ try:
330
+ import dill
331
+ except ModuleNotFoundError:
332
+ log.error("Unable to import `dill` module. Please please make sure that it installed.")
333
+ raise
334
+ return dill
335
+
336
+
337
+ def _load_cloudpickle():
338
+ try:
339
+ import cloudpickle
340
+ except ModuleNotFoundError:
341
+ log.error(
342
+ "Unable to import `cloudpickle` module. "
343
+ "Please install it with: pip install 'apache-airflow[cloudpickle]'"
344
+ )
345
+ raise
346
+ return cloudpickle
347
+
348
+
349
+ _SerializerTypeDef = Literal["pickle", "cloudpickle", "dill"]
350
+ _SERIALIZERS: dict[_SerializerTypeDef, Any] = {
351
+ "pickle": lazy_object_proxy.Proxy(_load_pickle),
352
+ "dill": lazy_object_proxy.Proxy(_load_dill),
353
+ "cloudpickle": lazy_object_proxy.Proxy(_load_cloudpickle),
354
+ }
355
+
356
+
357
+ class _BasePythonVirtualenvOperator(PythonOperator, metaclass=ABCMeta):
358
+ BASE_SERIALIZABLE_CONTEXT_KEYS = {
359
+ "ds",
360
+ "ds_nodash",
361
+ "expanded_ti_count",
362
+ "inlets",
363
+ "next_ds",
364
+ "next_ds_nodash",
365
+ "outlets",
366
+ "prev_ds",
367
+ "prev_ds_nodash",
368
+ "run_id",
369
+ "task_instance_key_str",
370
+ "test_mode",
371
+ "tomorrow_ds",
372
+ "tomorrow_ds_nodash",
373
+ "ts",
374
+ "ts_nodash",
375
+ "ts_nodash_with_tz",
376
+ "yesterday_ds",
377
+ "yesterday_ds_nodash",
378
+ }
379
+ PENDULUM_SERIALIZABLE_CONTEXT_KEYS = {
380
+ "data_interval_end",
381
+ "data_interval_start",
382
+ "execution_date",
383
+ "logical_date",
384
+ "next_execution_date",
385
+ "prev_data_interval_end_success",
386
+ "prev_data_interval_start_success",
387
+ "prev_execution_date",
388
+ "prev_execution_date_success",
389
+ "prev_start_date_success",
390
+ "prev_end_date_success",
391
+ }
392
+
393
+ AIRFLOW_SERIALIZABLE_CONTEXT_KEYS = {
394
+ "macros",
395
+ "conf",
396
+ "dag",
397
+ "dag_run",
398
+ "task",
399
+ "params",
400
+ "triggering_asset_events" if AIRFLOW_V_3_0_PLUS else "triggering_dataset_events",
401
+ }
402
+
403
+ def __init__(
404
+ self,
405
+ *,
406
+ python_callable: Callable,
407
+ serializer: _SerializerTypeDef | None = None,
408
+ op_args: Collection[Any] | None = None,
409
+ op_kwargs: Mapping[str, Any] | None = None,
410
+ string_args: Iterable[str] | None = None,
411
+ templates_dict: dict | None = None,
412
+ templates_exts: list[str] | None = None,
413
+ expect_airflow: bool = True,
414
+ skip_on_exit_code: int | Container[int] | None = None,
415
+ env_vars: dict[str, str] | None = None,
416
+ inherit_env: bool = True,
417
+ use_dill: bool = False,
418
+ use_airflow_context: bool = False,
419
+ **kwargs,
420
+ ):
421
+ if (
422
+ not isinstance(python_callable, types.FunctionType)
423
+ or isinstance(python_callable, types.LambdaType)
424
+ and python_callable.__name__ == "<lambda>"
425
+ ):
426
+ raise ValueError(f"{type(self).__name__} only supports functions for python_callable arg")
427
+ if inspect.isgeneratorfunction(python_callable):
428
+ raise ValueError(f"{type(self).__name__} does not support using 'yield' in python_callable")
429
+ super().__init__(
430
+ python_callable=python_callable,
431
+ op_args=op_args,
432
+ op_kwargs=op_kwargs,
433
+ templates_dict=templates_dict,
434
+ templates_exts=templates_exts,
435
+ **kwargs,
436
+ )
437
+ self.string_args = string_args or []
438
+
439
+ if use_dill:
440
+ warnings.warn(
441
+ "`use_dill` is deprecated and will be removed in a future version. "
442
+ "Please provide serializer='dill' instead.",
443
+ RemovedInAirflow3Warning,
444
+ stacklevel=3,
445
+ )
446
+ if serializer:
447
+ raise AirflowException(
448
+ "Both 'use_dill' and 'serializer' parameters are set. Please set only one of them"
449
+ )
450
+ serializer = "dill"
451
+ serializer = serializer or "pickle"
452
+ if serializer not in _SERIALIZERS:
453
+ msg = (
454
+ f"Unsupported serializer {serializer!r}. "
455
+ f"Expected one of {', '.join(map(repr, _SERIALIZERS))}"
456
+ )
457
+ raise AirflowException(msg)
458
+
459
+ self.pickling_library = _SERIALIZERS[serializer]
460
+ self.serializer: _SerializerTypeDef = serializer
461
+
462
+ self.expect_airflow = expect_airflow
463
+ self.skip_on_exit_code = (
464
+ skip_on_exit_code
465
+ if isinstance(skip_on_exit_code, Container)
466
+ else [skip_on_exit_code]
467
+ if skip_on_exit_code is not None
468
+ else []
469
+ )
470
+ self.env_vars = env_vars
471
+ self.inherit_env = inherit_env
472
+ self.use_airflow_context = use_airflow_context
473
+
474
+ @abstractmethod
475
+ def _iter_serializable_context_keys(self):
476
+ pass
477
+
478
+ def execute(self, context: Context) -> Any:
479
+ serializable_keys = set(self._iter_serializable_context_keys())
480
+ serializable_context = context_copy_partial(context, serializable_keys)
481
+ return super().execute(context=serializable_context)
482
+
483
+ def get_python_source(self):
484
+ """Return the source of self.python_callable."""
485
+ return textwrap.dedent(inspect.getsource(self.python_callable))
486
+
487
+ def _write_args(self, file: Path):
488
+ if self.op_args or self.op_kwargs:
489
+ self.log.info("Use %r as serializer.", self.serializer)
490
+ file.write_bytes(self.pickling_library.dumps({"args": self.op_args, "kwargs": self.op_kwargs}))
491
+
492
+ def _write_string_args(self, file: Path):
493
+ file.write_text("\n".join(map(str, self.string_args)))
494
+
495
+ def _read_result(self, path: Path):
496
+ if path.stat().st_size == 0:
497
+ return None
498
+ try:
499
+ return self.pickling_library.loads(path.read_bytes())
500
+ except ValueError as value_error:
501
+ raise DeserializingResultError() from value_error
502
+
503
+ def __deepcopy__(self, memo):
504
+ # module objects can't be copied _at all__
505
+ memo[id(self.pickling_library)] = self.pickling_library
506
+ return super().__deepcopy__(memo)
507
+
508
+ def _execute_python_callable_in_subprocess(self, python_path: Path):
509
+ with TemporaryDirectory(prefix="venv-call") as tmp:
510
+ tmp_dir = Path(tmp)
511
+ op_kwargs: dict[str, Any] = dict(self.op_kwargs)
512
+ if self.templates_dict:
513
+ op_kwargs["templates_dict"] = self.templates_dict
514
+ input_path = tmp_dir / "script.in"
515
+ output_path = tmp_dir / "script.out"
516
+ string_args_path = tmp_dir / "string_args.txt"
517
+ script_path = tmp_dir / "script.py"
518
+ termination_log_path = tmp_dir / "termination.log"
519
+ airflow_context_path = tmp_dir / "airflow_context.json"
520
+
521
+ self._write_args(input_path)
522
+ self._write_string_args(string_args_path)
523
+
524
+ if self.use_airflow_context and not _ENABLE_AIP_44:
525
+ error_msg = "`get_current_context()` needs to be used with AIP-44 enabled."
526
+ raise AirflowException(error_msg)
527
+
528
+ jinja_context = {
529
+ "op_args": self.op_args,
530
+ "op_kwargs": op_kwargs,
531
+ "expect_airflow": self.expect_airflow,
532
+ "pickling_library": self.serializer,
533
+ "python_callable": self.python_callable.__name__,
534
+ "python_callable_source": self.get_python_source(),
535
+ "use_airflow_context": self.use_airflow_context,
536
+ }
537
+
538
+ if inspect.getfile(self.python_callable) == self.dag.fileloc:
539
+ jinja_context["modified_dag_module_name"] = get_unique_dag_module_name(self.dag.fileloc)
540
+
541
+ write_python_script(
542
+ jinja_context=jinja_context,
543
+ filename=os.fspath(script_path),
544
+ render_template_as_native_obj=self.dag.render_template_as_native_obj,
545
+ )
546
+ if self.use_airflow_context:
547
+ from airflow.serialization.serialized_objects import BaseSerialization
548
+
549
+ context = get_current_context()
550
+ with create_session() as session:
551
+ # FIXME: DetachedInstanceError
552
+ dag_run, task_instance = context["dag_run"], context["task_instance"]
553
+ session.add_all([dag_run, task_instance])
554
+ serializable_context: dict[Encoding, Any] = BaseSerialization.serialize(
555
+ context, use_pydantic_models=True
556
+ )
557
+ with airflow_context_path.open("w+") as file:
558
+ json.dump(serializable_context, file)
559
+
560
+ env_vars = dict(os.environ) if self.inherit_env else {}
561
+ if self.env_vars:
562
+ env_vars.update(self.env_vars)
563
+
564
+ try:
565
+ cmd: list[str] = [
566
+ os.fspath(python_path),
567
+ os.fspath(script_path),
568
+ os.fspath(input_path),
569
+ os.fspath(output_path),
570
+ os.fspath(string_args_path),
571
+ os.fspath(termination_log_path),
572
+ os.fspath(airflow_context_path),
573
+ ]
574
+ if AIRFLOW_V_2_10_PLUS:
575
+ execute_in_subprocess(
576
+ cmd=cmd,
577
+ env=env_vars,
578
+ )
579
+ else:
580
+ execute_in_subprocess_with_kwargs(
581
+ cmd=cmd,
582
+ env=env_vars,
583
+ )
584
+ except subprocess.CalledProcessError as e:
585
+ if e.returncode in self.skip_on_exit_code:
586
+ raise AirflowSkipException(f"Process exited with code {e.returncode}. Skipping.")
587
+ elif termination_log_path.exists() and termination_log_path.stat().st_size > 0:
588
+ error_msg = f"Process returned non-zero exit status {e.returncode}.\n"
589
+ with open(termination_log_path) as file:
590
+ error_msg += file.read()
591
+ raise AirflowException(error_msg) from None
592
+ else:
593
+ raise
594
+
595
+ if 0 in self.skip_on_exit_code:
596
+ raise AirflowSkipException("Process exited with code 0. Skipping.")
597
+
598
+ return self._read_result(output_path)
599
+
600
+ def determine_kwargs(self, context: Mapping[str, Any]) -> Mapping[str, Any]:
601
+ return KeywordParameters.determine(self.python_callable, self.op_args, context).serializing()
602
+
603
+
604
+ class PythonVirtualenvOperator(_BasePythonVirtualenvOperator):
605
+ """
606
+ Run a function in a virtualenv that is created and destroyed automatically.
607
+
608
+ The function (has certain caveats) must be defined using def, and not be
609
+ part of a class. All imports must happen inside the function
610
+ and no variables outside the scope may be referenced. A global scope
611
+ variable named virtualenv_string_args will be available (populated by
612
+ string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
613
+ can use a return value.
614
+ Note that if your virtualenv runs in a different Python major version than Airflow,
615
+ you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
616
+ Airflow through plugins. You can use string_args though.
617
+
618
+ .. seealso::
619
+ For more information on how to use this operator, take a look at the guide:
620
+ :ref:`howto/operator:PythonVirtualenvOperator`
621
+
622
+ :param python_callable: A python function with no references to outside variables,
623
+ defined with def, which will be run in a virtual environment.
624
+ :param requirements: Either a list of requirement strings, or a (templated)
625
+ "requirements file" as specified by pip.
626
+ :param python_version: The Python version to run the virtual environment with. Note that
627
+ both 2 and 2.7 are acceptable forms.
628
+ :param serializer: Which serializer use to serialize the args and result. It can be one of the following:
629
+
630
+ - ``"pickle"``: (default) Use pickle for serialization. Included in the Python Standard Library.
631
+ - ``"cloudpickle"``: Use cloudpickle for serialize more complex types,
632
+ this requires to include cloudpickle in your requirements.
633
+ - ``"dill"``: Use dill for serialize more complex types,
634
+ this requires to include dill in your requirements.
635
+ :param system_site_packages: Whether to include
636
+ system_site_packages in your virtual environment.
637
+ See virtualenv documentation for more information.
638
+ :param pip_install_options: a list of pip install options when installing requirements
639
+ See 'pip install -h' for available options
640
+ :param op_args: A list of positional arguments to pass to python_callable.
641
+ :param op_kwargs: A dict of keyword arguments to pass to python_callable.
642
+ :param string_args: Strings that are present in the global var virtualenv_string_args,
643
+ available to python_callable at runtime as a list[str]. Note that args are split
644
+ by newline.
645
+ :param templates_dict: a dictionary where the values are templates that
646
+ will get templated by the Airflow engine sometime between
647
+ ``__init__`` and ``execute`` takes place and are made available
648
+ in your callable's context after the template has been applied
649
+ :param templates_exts: a list of file extensions to resolve while
650
+ processing templated fields, for examples ``['.sql', '.hql']``
651
+ :param expect_airflow: expect Airflow to be installed in the target environment. If true, the operator
652
+ will raise warning if Airflow is not installed, and it will attempt to load Airflow
653
+ macros when starting.
654
+ :param skip_on_exit_code: If python_callable exits with this exit code, leave the task
655
+ in ``skipped`` state (default: None). If set to ``None``, any non-zero
656
+ exit code will be treated as a failure.
657
+ :param index_urls: an optional list of index urls to load Python packages from.
658
+ If not provided the system pip conf will be used to source packages from.
659
+ :param venv_cache_path: Optional path to the virtual environment parent folder in which the
660
+ virtual environment will be cached, creates a sub-folder venv-{hash} whereas hash will be replaced
661
+ with a checksum of requirements. If not provided the virtual environment will be created and deleted
662
+ in a temp folder for every execution.
663
+ :param env_vars: A dictionary containing additional environment variables to set for the virtual
664
+ environment when it is executed.
665
+ :param inherit_env: Whether to inherit the current environment variables when executing the virtual
666
+ environment. If set to ``True``, the virtual environment will inherit the environment variables
667
+ of the parent process (``os.environ``). If set to ``False``, the virtual environment will be
668
+ executed with a clean environment.
669
+ :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize
670
+ the args and result (pickle is default). This allows more complex types
671
+ but requires you to include dill in your requirements.
672
+ :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable.
673
+ """
674
+
675
+ template_fields: Sequence[str] = tuple(
676
+ {"requirements", "index_urls", "venv_cache_path"}.union(PythonOperator.template_fields)
677
+ )
678
+ template_ext: Sequence[str] = (".txt",)
679
+
680
+ def __init__(
681
+ self,
682
+ *,
683
+ python_callable: Callable,
684
+ requirements: None | Iterable[str] | str = None,
685
+ python_version: str | None = None,
686
+ serializer: _SerializerTypeDef | None = None,
687
+ system_site_packages: bool = True,
688
+ pip_install_options: list[str] | None = None,
689
+ op_args: Collection[Any] | None = None,
690
+ op_kwargs: Mapping[str, Any] | None = None,
691
+ string_args: Iterable[str] | None = None,
692
+ templates_dict: dict | None = None,
693
+ templates_exts: list[str] | None = None,
694
+ expect_airflow: bool = True,
695
+ skip_on_exit_code: int | Container[int] | None = None,
696
+ index_urls: None | Collection[str] | str = None,
697
+ venv_cache_path: None | os.PathLike[str] = None,
698
+ env_vars: dict[str, str] | None = None,
699
+ inherit_env: bool = True,
700
+ use_dill: bool = False,
701
+ use_airflow_context: bool = False,
702
+ **kwargs,
703
+ ):
704
+ if (
705
+ python_version
706
+ and str(python_version)[0] != str(sys.version_info.major)
707
+ and (op_args or op_kwargs)
708
+ ):
709
+ raise AirflowException(
710
+ "Passing op_args or op_kwargs is not supported across different Python "
711
+ "major versions for PythonVirtualenvOperator. Please use string_args."
712
+ f"Sys version: {sys.version_info}. Virtual environment version: {python_version}"
713
+ )
714
+ if python_version is not None and not isinstance(python_version, str):
715
+ raise AirflowException(
716
+ "Passing non-string types (e.g. int or float) as python_version not supported"
717
+ )
718
+ if use_airflow_context and not AIRFLOW_V_3_0_PLUS:
719
+ raise AirflowException(
720
+ "The `use_airflow_context=True` is only supported in Airflow 3.0.0 and later."
721
+ )
722
+ if use_airflow_context and (not expect_airflow and not system_site_packages):
723
+ raise AirflowException(
724
+ "The `use_airflow_context` parameter is set to True, but "
725
+ "expect_airflow and system_site_packages are set to False."
726
+ )
727
+ if not requirements:
728
+ self.requirements: list[str] = []
729
+ elif isinstance(requirements, str):
730
+ self.requirements = [requirements]
731
+ else:
732
+ self.requirements = list(requirements)
733
+ self.python_version = python_version
734
+ self.system_site_packages = system_site_packages
735
+ self.pip_install_options = pip_install_options
736
+ if isinstance(index_urls, str):
737
+ self.index_urls: list[str] | None = [index_urls]
738
+ elif isinstance(index_urls, Collection):
739
+ self.index_urls = list(index_urls)
740
+ else:
741
+ self.index_urls = None
742
+ self.venv_cache_path = venv_cache_path
743
+ super().__init__(
744
+ python_callable=python_callable,
745
+ serializer=serializer,
746
+ op_args=op_args,
747
+ op_kwargs=op_kwargs,
748
+ string_args=string_args,
749
+ templates_dict=templates_dict,
750
+ templates_exts=templates_exts,
751
+ expect_airflow=expect_airflow,
752
+ skip_on_exit_code=skip_on_exit_code,
753
+ env_vars=env_vars,
754
+ inherit_env=inherit_env,
755
+ use_dill=use_dill,
756
+ use_airflow_context=use_airflow_context,
757
+ **kwargs,
758
+ )
759
+
760
+ def _requirements_list(self, exclude_cloudpickle: bool = False) -> list[str]:
761
+ """Prepare a list of requirements that need to be installed for the virtual environment."""
762
+ requirements = [str(dependency) for dependency in self.requirements]
763
+ if not self.system_site_packages:
764
+ if (
765
+ self.serializer == "cloudpickle"
766
+ and not exclude_cloudpickle
767
+ and "cloudpickle" not in requirements
768
+ ):
769
+ requirements.append("cloudpickle")
770
+ elif self.serializer == "dill" and "dill" not in requirements:
771
+ requirements.append("dill")
772
+ requirements.sort() # Ensure a hash is stable
773
+ return requirements
774
+
775
+ def _prepare_venv(self, venv_path: Path) -> None:
776
+ """Prepare the requirements and installs the virtual environment."""
777
+ requirements_file = venv_path / "requirements.txt"
778
+ requirements_file.write_text("\n".join(self._requirements_list()))
779
+ prepare_virtualenv(
780
+ venv_directory=str(venv_path),
781
+ python_bin=f"python{self.python_version}" if self.python_version else "python",
782
+ system_site_packages=self.system_site_packages,
783
+ requirements_file_path=str(requirements_file),
784
+ pip_install_options=self.pip_install_options,
785
+ index_urls=self.index_urls,
786
+ )
787
+
788
+ def _calculate_cache_hash(self, exclude_cloudpickle: bool = False) -> tuple[str, str]:
789
+ """
790
+ Generate the hash of the cache folder to use.
791
+
792
+ The following factors are used as input for the hash:
793
+ - (sorted) list of requirements
794
+ - pip install options
795
+ - flag of system site packages
796
+ - python version
797
+ - Variable to override the hash with a cache key
798
+ - Index URLs
799
+
800
+ Returns a hash and the data dict which is the base for the hash as text.
801
+ """
802
+ hash_dict = {
803
+ "requirements_list": self._requirements_list(exclude_cloudpickle=exclude_cloudpickle),
804
+ "pip_install_options": self.pip_install_options,
805
+ "index_urls": self.index_urls,
806
+ "cache_key": str(Variable.get("PythonVirtualenvOperator.cache_key", "")),
807
+ "python_version": self.python_version,
808
+ "system_site_packages": self.system_site_packages,
809
+ }
810
+ hash_text = json.dumps(hash_dict, sort_keys=True)
811
+ hash_object = hashlib_wrapper.md5(hash_text.encode())
812
+ requirements_hash = hash_object.hexdigest()
813
+ return requirements_hash[:8], hash_text
814
+
815
+ def _ensure_venv_cache_exists(self, venv_cache_path: Path) -> Path:
816
+ """Ensure a valid virtual environment is set up and will create inplace."""
817
+ cache_hash, hash_data = self._calculate_cache_hash()
818
+ venv_path = venv_cache_path / f"venv-{cache_hash}"
819
+ self.log.info("Python virtual environment will be cached in %s", venv_path)
820
+ venv_path.parent.mkdir(parents=True, exist_ok=True)
821
+ with open(f"{venv_path}.lock", "w") as f:
822
+ # Ensure that cache is not build by parallel workers
823
+ import fcntl
824
+
825
+ fcntl.flock(f, fcntl.LOCK_EX)
826
+
827
+ hash_marker = venv_path / "install_complete_marker.json"
828
+ try:
829
+ if venv_path.exists():
830
+ if hash_marker.exists():
831
+ previous_hash_data = hash_marker.read_text(encoding="utf8")
832
+ if previous_hash_data == hash_data:
833
+ self.log.info("Re-using cached Python virtual environment in %s", venv_path)
834
+ return venv_path
835
+
836
+ _, hash_data_before_upgrade = self._calculate_cache_hash(exclude_cloudpickle=True)
837
+ if previous_hash_data == hash_data_before_upgrade:
838
+ self.log.warning(
839
+ "Found a previous virtual environment in with outdated dependencies %s, "
840
+ "deleting and re-creating.",
841
+ venv_path,
842
+ )
843
+ else:
844
+ self.log.error(
845
+ "Unicorn alert: Found a previous virtual environment in %s "
846
+ "with the same hash but different parameters. Previous setup: '%s' / "
847
+ "Requested venv setup: '%s'. Please report a bug to airflow!",
848
+ venv_path,
849
+ previous_hash_data,
850
+ hash_data,
851
+ )
852
+ else:
853
+ self.log.warning(
854
+ "Found a previous (probably partial installed) virtual environment in %s, "
855
+ "deleting and re-creating.",
856
+ venv_path,
857
+ )
858
+
859
+ shutil.rmtree(venv_path)
860
+
861
+ venv_path.mkdir(parents=True)
862
+ self._prepare_venv(venv_path)
863
+ hash_marker.write_text(hash_data, encoding="utf8")
864
+ except Exception as e:
865
+ shutil.rmtree(venv_path)
866
+ raise AirflowException(f"Unable to create new virtual environment in {venv_path}") from e
867
+ self.log.info("New Python virtual environment created in %s", venv_path)
868
+ return venv_path
869
+
870
+ def execute_callable(self):
871
+ if self.venv_cache_path:
872
+ venv_path = self._ensure_venv_cache_exists(Path(self.venv_cache_path))
873
+ python_path = venv_path / "bin" / "python"
874
+ return self._execute_python_callable_in_subprocess(python_path)
875
+
876
+ with TemporaryDirectory(prefix="venv") as tmp_dir:
877
+ tmp_path = Path(tmp_dir)
878
+ self._prepare_venv(tmp_path)
879
+ python_path = tmp_path / "bin" / "python"
880
+ result = self._execute_python_callable_in_subprocess(python_path)
881
+ return result
882
+
883
+ def _iter_serializable_context_keys(self):
884
+ yield from self.BASE_SERIALIZABLE_CONTEXT_KEYS
885
+ if self.system_site_packages or "apache-airflow" in self.requirements:
886
+ yield from self.AIRFLOW_SERIALIZABLE_CONTEXT_KEYS
887
+ yield from self.PENDULUM_SERIALIZABLE_CONTEXT_KEYS
888
+ elif "pendulum" in self.requirements:
889
+ yield from self.PENDULUM_SERIALIZABLE_CONTEXT_KEYS
890
+
891
+
892
+ class BranchPythonVirtualenvOperator(PythonVirtualenvOperator, BranchMixIn):
893
+ """
894
+ A workflow can "branch" or follow a path after the execution of this task in a virtual environment.
895
+
896
+ It derives the PythonVirtualenvOperator and expects a Python function that returns
897
+ a single task_id, a single task_group_id, or a list of task_ids and/or
898
+ task_group_ids to follow. The task_id(s) and/or task_group_id(s) returned
899
+ should point to a task or task group directly downstream from {self}. All
900
+ other "branches" or directly downstream tasks are marked with a state of
901
+ ``skipped`` so that these paths can't move forward. The ``skipped`` states
902
+ are propagated downstream to allow for the DAG state to fill up and
903
+ the DAG run's state to be inferred.
904
+
905
+ .. seealso::
906
+ For more information on how to use this operator, take a look at the guide:
907
+ :ref:`howto/operator:BranchPythonVirtualenvOperator`
908
+ """
909
+
910
+ def execute(self, context: Context) -> Any:
911
+ return self.do_branch(context, super().execute(context))
912
+
913
+
914
+ class ExternalPythonOperator(_BasePythonVirtualenvOperator):
915
+ """
916
+ Run a function in a virtualenv that is not re-created.
917
+
918
+ Reused as is without the overhead of creating the virtual environment (with certain caveats).
919
+
920
+ The function must be defined using def, and not be
921
+ part of a class. All imports must happen inside the function
922
+ and no variables outside the scope may be referenced. A global scope
923
+ variable named virtualenv_string_args will be available (populated by
924
+ string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
925
+ can use a return value.
926
+ Note that if your virtual environment runs in a different Python major version than Airflow,
927
+ you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
928
+ Airflow through plugins. You can use string_args though.
929
+
930
+ If Airflow is installed in the external environment in different version that the version
931
+ used by the operator, the operator will fail.,
932
+
933
+ .. seealso::
934
+ For more information on how to use this operator, take a look at the guide:
935
+ :ref:`howto/operator:ExternalPythonOperator`
936
+
937
+ :param python: Full path string (file-system specific) that points to a Python binary inside
938
+ a virtual environment that should be used (in ``VENV/bin`` folder). Should be absolute path
939
+ (so usually start with "/" or "X:/" depending on the filesystem/os used).
940
+ :param python_callable: A python function with no references to outside variables,
941
+ defined with def, which will be run in a virtual environment.
942
+ :param serializer: Which serializer use to serialize the args and result. It can be one of the following:
943
+
944
+ - ``"pickle"``: (default) Use pickle for serialization. Included in the Python Standard Library.
945
+ - ``"cloudpickle"``: Use cloudpickle for serialize more complex types,
946
+ this requires to include cloudpickle in your requirements.
947
+ - ``"dill"``: Use dill for serialize more complex types,
948
+ this requires to include dill in your requirements.
949
+ :param op_args: A list of positional arguments to pass to python_callable.
950
+ :param op_kwargs: A dict of keyword arguments to pass to python_callable.
951
+ :param string_args: Strings that are present in the global var virtualenv_string_args,
952
+ available to python_callable at runtime as a list[str]. Note that args are split
953
+ by newline.
954
+ :param templates_dict: a dictionary where the values are templates that
955
+ will get templated by the Airflow engine sometime between
956
+ ``__init__`` and ``execute`` takes place and are made available
957
+ in your callable's context after the template has been applied
958
+ :param templates_exts: a list of file extensions to resolve while
959
+ processing templated fields, for examples ``['.sql', '.hql']``
960
+ :param expect_airflow: expect Airflow to be installed in the target environment. If true, the operator
961
+ will raise warning if Airflow is not installed, and it will attempt to load Airflow
962
+ macros when starting.
963
+ :param skip_on_exit_code: If python_callable exits with this exit code, leave the task
964
+ in ``skipped`` state (default: None). If set to ``None``, any non-zero
965
+ exit code will be treated as a failure.
966
+ :param env_vars: A dictionary containing additional environment variables to set for the virtual
967
+ environment when it is executed.
968
+ :param inherit_env: Whether to inherit the current environment variables when executing the virtual
969
+ environment. If set to ``True``, the virtual environment will inherit the environment variables
970
+ of the parent process (``os.environ``). If set to ``False``, the virtual environment will be
971
+ executed with a clean environment.
972
+ :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize
973
+ the args and result (pickle is default). This allows more complex types
974
+ but requires you to include dill in your requirements.
975
+ :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable.
976
+ """
977
+
978
+ template_fields: Sequence[str] = tuple({"python"}.union(PythonOperator.template_fields))
979
+
980
+ def __init__(
981
+ self,
982
+ *,
983
+ python: str,
984
+ python_callable: Callable,
985
+ serializer: _SerializerTypeDef | None = None,
986
+ op_args: Collection[Any] | None = None,
987
+ op_kwargs: Mapping[str, Any] | None = None,
988
+ string_args: Iterable[str] | None = None,
989
+ templates_dict: dict | None = None,
990
+ templates_exts: list[str] | None = None,
991
+ expect_airflow: bool = True,
992
+ expect_pendulum: bool = False,
993
+ skip_on_exit_code: int | Container[int] | None = None,
994
+ env_vars: dict[str, str] | None = None,
995
+ inherit_env: bool = True,
996
+ use_dill: bool = False,
997
+ use_airflow_context: bool = False,
998
+ **kwargs,
999
+ ):
1000
+ if not python:
1001
+ raise ValueError("Python Path must be defined in ExternalPythonOperator")
1002
+ if use_airflow_context and not AIRFLOW_V_3_0_PLUS:
1003
+ raise AirflowException(
1004
+ "The `use_airflow_context=True` is only supported in Airflow 3.0.0 and later."
1005
+ )
1006
+ if use_airflow_context and not expect_airflow:
1007
+ raise AirflowException(
1008
+ "The `use_airflow_context` parameter is set to True, but expect_airflow is set to False."
1009
+ )
1010
+ self.python = python
1011
+ self.expect_pendulum = expect_pendulum
1012
+ super().__init__(
1013
+ python_callable=python_callable,
1014
+ serializer=serializer,
1015
+ op_args=op_args,
1016
+ op_kwargs=op_kwargs,
1017
+ string_args=string_args,
1018
+ templates_dict=templates_dict,
1019
+ templates_exts=templates_exts,
1020
+ expect_airflow=expect_airflow,
1021
+ skip_on_exit_code=skip_on_exit_code,
1022
+ env_vars=env_vars,
1023
+ inherit_env=inherit_env,
1024
+ use_dill=use_dill,
1025
+ use_airflow_context=use_airflow_context,
1026
+ **kwargs,
1027
+ )
1028
+
1029
+ def execute_callable(self):
1030
+ python_path = Path(self.python)
1031
+ if not python_path.exists():
1032
+ raise ValueError(f"Python Path '{python_path}' must exists")
1033
+ if not python_path.is_file():
1034
+ raise ValueError(f"Python Path '{python_path}' must be a file")
1035
+ if not python_path.is_absolute():
1036
+ raise ValueError(f"Python Path '{python_path}' must be an absolute path.")
1037
+ python_version = _PythonVersionInfo.from_executable(self.python)
1038
+ if python_version.major != sys.version_info.major and (self.op_args or self.op_kwargs):
1039
+ raise AirflowException(
1040
+ "Passing op_args or op_kwargs is not supported across different Python "
1041
+ "major versions for ExternalPythonOperator. Please use string_args."
1042
+ f"Sys version: {sys.version_info}. "
1043
+ f"Virtual environment version: {python_version}"
1044
+ )
1045
+ return self._execute_python_callable_in_subprocess(python_path)
1046
+
1047
+ def _iter_serializable_context_keys(self):
1048
+ yield from self.BASE_SERIALIZABLE_CONTEXT_KEYS
1049
+ if self._get_airflow_version_from_target_env():
1050
+ yield from self.AIRFLOW_SERIALIZABLE_CONTEXT_KEYS
1051
+ yield from self.PENDULUM_SERIALIZABLE_CONTEXT_KEYS
1052
+ elif self._is_pendulum_installed_in_target_env():
1053
+ yield from self.PENDULUM_SERIALIZABLE_CONTEXT_KEYS
1054
+
1055
+ def _is_pendulum_installed_in_target_env(self) -> bool:
1056
+ try:
1057
+ subprocess.check_call([self.python, "-c", "import pendulum"])
1058
+ return True
1059
+ except Exception as e:
1060
+ if self.expect_pendulum:
1061
+ self.log.warning("When checking for Pendulum installed in virtual environment got %s", e)
1062
+ self.log.warning(
1063
+ "Pendulum is not properly installed in the virtual environment "
1064
+ "Pendulum context keys will not be available. "
1065
+ "Please Install Pendulum or Airflow in your virtual environment to access them."
1066
+ )
1067
+ return False
1068
+
1069
+ @property
1070
+ def _external_airflow_version_script(self):
1071
+ """
1072
+ Return python script which determines the version of the Apache Airflow.
1073
+
1074
+ Import airflow as a module might take a while as a result,
1075
+ obtaining a version would take up to 1 second.
1076
+ On the other hand, `importlib.metadata.version` will retrieve the package version pretty fast
1077
+ something below 100ms; this includes new subprocess overhead.
1078
+
1079
+ Possible side effect: It might be a situation that `importlib.metadata` is not available (Python < 3.8),
1080
+ as well as backport `importlib_metadata` which might indicate that venv doesn't contain an `apache-airflow`
1081
+ or something wrong with the environment.
1082
+ """
1083
+ return textwrap.dedent(
1084
+ """
1085
+ try:
1086
+ from importlib.metadata import version
1087
+ except ImportError:
1088
+ from importlib_metadata import version
1089
+ print(version("apache-airflow"))
1090
+ """
1091
+ )
1092
+
1093
+ def _get_airflow_version_from_target_env(self) -> str | None:
1094
+ from airflow import __version__ as airflow_version
1095
+
1096
+ try:
1097
+ result = subprocess.check_output(
1098
+ [self.python, "-c", self._external_airflow_version_script],
1099
+ text=True,
1100
+ )
1101
+ target_airflow_version = result.strip()
1102
+ if target_airflow_version != airflow_version:
1103
+ raise AirflowConfigException(
1104
+ f"The version of Airflow installed for the {self.python} "
1105
+ f"({target_airflow_version}) is different than the runtime Airflow version: "
1106
+ f"{airflow_version}. Make sure your environment has the same Airflow version "
1107
+ f"installed as the Airflow runtime."
1108
+ )
1109
+ return target_airflow_version
1110
+ except Exception as e:
1111
+ if self.expect_airflow:
1112
+ self.log.warning("When checking for Airflow installed in virtual environment got %s", e)
1113
+ self.log.warning(
1114
+ "This means that Airflow is not properly installed by %s. "
1115
+ "Airflow context keys will not be available. "
1116
+ "Please Install Airflow %s in your environment to access them.",
1117
+ self.python,
1118
+ airflow_version,
1119
+ )
1120
+ return None
1121
+
1122
+
1123
+ class BranchExternalPythonOperator(ExternalPythonOperator, BranchMixIn):
1124
+ """
1125
+ A workflow can "branch" or follow a path after the execution of this task.
1126
+
1127
+ Extends ExternalPythonOperator, so expects to get Python:
1128
+ virtual environment that should be used (in ``VENV/bin`` folder). Should be absolute path,
1129
+ so it can run on separate virtual environment similarly to ExternalPythonOperator.
1130
+
1131
+ .. seealso::
1132
+ For more information on how to use this operator, take a look at the guide:
1133
+ :ref:`howto/operator:BranchExternalPythonOperator`
1134
+ """
1135
+
1136
+ def execute(self, context: Context) -> Any:
1137
+ return self.do_branch(context, super().execute(context))
1138
+
1139
+
1140
+ def get_current_context() -> Context:
1141
+ """
1142
+ Retrieve the execution context dictionary without altering user method's signature.
1143
+
1144
+ This is the simplest method of retrieving the execution context dictionary.
1145
+
1146
+ **Old style:**
1147
+
1148
+ .. code:: python
1149
+
1150
+ def my_task(**context):
1151
+ ti = context["ti"]
1152
+
1153
+ **New style:**
1154
+
1155
+ .. code:: python
1156
+
1157
+ from airflow.providers.standard.operators.python import get_current_context
1158
+
1159
+
1160
+ def my_task():
1161
+ context = get_current_context()
1162
+ ti = context["ti"]
1163
+
1164
+ Current context will only have value if this method was called after an operator
1165
+ was starting to execute.
1166
+ """
1167
+ if not _CURRENT_CONTEXT:
1168
+ raise AirflowException(
1169
+ "Current context was requested but no context was found! "
1170
+ "Are you running within an airflow task?"
1171
+ )
1172
+ return _CURRENT_CONTEXT[-1]