python.py 49 KB

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