Skip to content

Commit a206012

Browse files
authored
Add Python Virtualenv Operator Caching (#33355)
1 parent a74ec40 commit a206012

File tree

4 files changed

+157
-31
lines changed

4 files changed

+157
-31
lines changed

airflow/decorators/__init__.pyi

+8-3
Original file line numberDiff line numberDiff line change
@@ -110,6 +110,7 @@ class TaskDecoratorCollection:
110110
pip_install_options: list[str] | None = None,
111111
skip_on_exit_code: int | Container[int] | None = None,
112112
index_urls: None | Collection[str] | str = None,
113+
venv_cache_path: None | str = None,
113114
show_return_value_in_logs: bool = True,
114115
**kwargs,
115116
) -> TaskDecorator:
@@ -119,13 +120,13 @@ class TaskDecoratorCollection:
119120
Dict will unroll to XCom values with keys as XCom keys. Defaults to False.
120121
:param requirements: Either a list of requirement strings, or a (templated)
121122
"requirements file" as specified by pip.
122-
:param python_version: The Python version to run the virtualenv with. Note that
123+
:param python_version: The Python version to run the virtual environment with. Note that
123124
both 2 and 2.7 are acceptable forms.
124125
:param use_dill: Whether to use dill to serialize
125126
the args and result (pickle is default). This allow more complex types
126127
but requires you to include dill in your requirements.
127128
:param system_site_packages: Whether to include
128-
system_site_packages in your virtualenv.
129+
system_site_packages in your virtual environment.
129130
See virtualenv documentation for more information.
130131
:param pip_install_options: a list of pip install options when installing requirements
131132
See 'pip install -h' for available options
@@ -134,6 +135,10 @@ class TaskDecoratorCollection:
134135
exit code will be treated as a failure.
135136
:param index_urls: an optional list of index urls to load Python packages from.
136137
If not provided the system pip conf will be used to source packages from.
138+
:param venv_cache_path: Optional path to the virtual environment parent folder in which the
139+
virtual environment will be cached, creates a sub-folder venv-{hash} whereas hash will be
140+
replaced with a checksum of requirements. If not provided the virtual environment will be
141+
created and deleted in a temp folder for every execution.
137142
:param templates_dict: a dictionary where the values are templates that
138143
will get templated by the Airflow engine sometime between
139144
``__init__`` and ``execute`` takes place and are made available
@@ -160,7 +165,7 @@ class TaskDecoratorCollection:
160165
"""Create a decorator to convert the decorated callable to a virtual environment task.
161166
162167
:param python: Full path string (file-system specific) that points to a Python binary inside
163-
a virtualenv that should be used (in ``VENV/bin`` folder). Should be absolute path
168+
a virtual environment that should be used (in ``VENV/bin`` folder). Should be absolute path
164169
(so usually start with "/" or "X:/" depending on the filesystem/os used).
165170
:param multiple_outputs: If set, function return value will be unrolled to multiple XCom values.
166171
Dict will unroll to XCom values with keys as XCom keys. Defaults to False.

airflow/operators/python.py

+110-21
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,10 @@
1717
# under the License.
1818
from __future__ import annotations
1919

20+
import fcntl
2021
import importlib
2122
import inspect
23+
import json
2224
import logging
2325
import os
2426
import pickle
@@ -46,7 +48,9 @@
4648
from airflow.models.baseoperator import BaseOperator
4749
from airflow.models.skipmixin import SkipMixin
4850
from airflow.models.taskinstance import _CURRENT_CONTEXT
51+
from airflow.models.variable import Variable
4952
from airflow.operators.branch import BranchMixIn
53+
from airflow.utils import hashlib_wrapper
5054
from airflow.utils.context import context_copy_partial, context_merge
5155
from airflow.utils.operator_helpers import KeywordParameters
5256
from airflow.utils.process_utils import execute_in_subprocess
@@ -489,16 +493,16 @@ class PythonVirtualenvOperator(_BasePythonVirtualenvOperator):
489493
:ref:`howto/operator:PythonVirtualenvOperator`
490494
491495
:param python_callable: A python function with no references to outside variables,
492-
defined with def, which will be run in a virtualenv
496+
defined with def, which will be run in a virtual environment.
493497
:param requirements: Either a list of requirement strings, or a (templated)
494498
"requirements file" as specified by pip.
495-
:param python_version: The Python version to run the virtualenv with. Note that
499+
:param python_version: The Python version to run the virtual environment with. Note that
496500
both 2 and 2.7 are acceptable forms.
497501
:param use_dill: Whether to use dill to serialize
498502
the args and result (pickle is default). This allow more complex types
499503
but requires you to include dill in your requirements.
500504
:param system_site_packages: Whether to include
501-
system_site_packages in your virtualenv.
505+
system_site_packages in your virtual environment.
502506
See virtualenv documentation for more information.
503507
:param pip_install_options: a list of pip install options when installing requirements
504508
See 'pip install -h' for available options
@@ -521,9 +525,15 @@ class PythonVirtualenvOperator(_BasePythonVirtualenvOperator):
521525
exit code will be treated as a failure.
522526
:param index_urls: an optional list of index urls to load Python packages from.
523527
If not provided the system pip conf will be used to source packages from.
528+
:param venv_cache_path: Optional path to the virtual environment parent folder in which the
529+
virtual environment will be cached, creates a sub-folder venv-{hash} whereas hash will be replaced
530+
with a checksum of requirements. If not provided the virtual environment will be created and deleted
531+
in a temp folder for every execution.
524532
"""
525533

526-
template_fields: Sequence[str] = tuple({"requirements"} | set(PythonOperator.template_fields))
534+
template_fields: Sequence[str] = tuple(
535+
{"requirements", "index_urls", "venv_cache_path"}.union(PythonOperator.template_fields)
536+
)
527537
template_ext: Sequence[str] = (".txt",)
528538

529539
def __init__(
@@ -543,6 +553,7 @@ def __init__(
543553
expect_airflow: bool = True,
544554
skip_on_exit_code: int | Container[int] | None = None,
545555
index_urls: None | Collection[str] | str = None,
556+
venv_cache_path: None | os.PathLike[str] = None,
546557
**kwargs,
547558
):
548559
if (
@@ -553,7 +564,7 @@ def __init__(
553564
raise AirflowException(
554565
"Passing op_args or op_kwargs is not supported across different Python "
555566
"major versions for PythonVirtualenvOperator. Please use string_args."
556-
f"Sys version: {sys.version_info}. Venv version: {python_version}"
567+
f"Sys version: {sys.version_info}. Virtual environment version: {python_version}"
557568
)
558569
if python_version is not None and not isinstance(python_version, str):
559570
warnings.warn(
@@ -579,6 +590,7 @@ def __init__(
579590
self.index_urls = list(index_urls)
580591
else:
581592
self.index_urls = None
593+
self.venv_cache_path = venv_cache_path
582594
super().__init__(
583595
python_callable=python_callable,
584596
use_dill=use_dill,
@@ -593,15 +605,15 @@ def __init__(
593605
)
594606

595607
def _requirements_list(self) -> list[str]:
596-
"""Prepare a list of requirements that need to be installed for the venv."""
608+
"""Prepare a list of requirements that need to be installed for the virtual environment."""
597609
requirements = [str(dependency) for dependency in self.requirements]
598610
if not self.system_site_packages and self.use_dill and "dill" not in requirements:
599611
requirements.append("dill")
600612
requirements.sort() # Ensure a hash is stable
601613
return requirements
602614

603615
def _prepare_venv(self, venv_path: Path) -> None:
604-
"""Prepare the requirements and installs the venv."""
616+
"""Prepare the requirements and installs the virtual environment."""
605617
requirements_file = venv_path / "requirements.txt"
606618
requirements_file.write_text("\n".join(self._requirements_list()))
607619
prepare_virtualenv(
@@ -613,7 +625,83 @@ def _prepare_venv(self, venv_path: Path) -> None:
613625
index_urls=self.index_urls,
614626
)
615627

628+
def _calculate_cache_hash(self) -> tuple[str, str]:
629+
"""Helper to generate the hash of the cache folder to use.
630+
631+
The following factors are used as input for the hash:
632+
- (sorted) list of requirements
633+
- pip install options
634+
- flag of system site packages
635+
- python version
636+
- Variable to override the hash with a cache key
637+
- Index URLs
638+
639+
Returns a hash and the data dict which is the base for the hash as text.
640+
"""
641+
hash_dict = {
642+
"requirements_list": self._requirements_list(),
643+
"pip_install_options": self.pip_install_options,
644+
"index_urls": self.index_urls,
645+
"cache_key": str(Variable.get("PythonVirtualenvOperator.cache_key", "")),
646+
"python_version": self.python_version,
647+
"system_site_packages": self.system_site_packages,
648+
}
649+
hash_text = json.dumps(hash_dict, sort_keys=True)
650+
hash_object = hashlib_wrapper.md5(hash_text.encode())
651+
requirements_hash = hash_object.hexdigest()
652+
return requirements_hash[:8], hash_text
653+
654+
def _ensure_venv_cache_exists(self, venv_cache_path: Path) -> Path:
655+
"""Helper to ensure a valid virtual environment is set up and will create inplace."""
656+
cache_hash, hash_data = self._calculate_cache_hash()
657+
venv_path = venv_cache_path / f"venv-{cache_hash}"
658+
self.log.info("Python virtual environment will be cached in %s", venv_path)
659+
venv_path.parent.mkdir(parents=True, exist_ok=True)
660+
with open(f"{venv_path}.lock", "w") as f:
661+
# Ensure that cache is not build by parallel workers
662+
fcntl.flock(f, fcntl.LOCK_EX)
663+
664+
hash_marker = venv_path / "install_complete_marker.json"
665+
try:
666+
if venv_path.exists():
667+
if hash_marker.exists():
668+
previous_hash_data = hash_marker.read_text(encoding="utf8")
669+
if previous_hash_data == hash_data:
670+
self.log.info("Re-using cached Python virtual environment in %s", venv_path)
671+
return venv_path
672+
673+
self.log.error(
674+
"Unicorn alert: Found a previous virtual environment in %s "
675+
"with the same hash but different parameters. Previous setup: '%s' / "
676+
"Requested venv setup: '%s'. Please report a bug to airflow!",
677+
venv_path,
678+
previous_hash_data,
679+
hash_data,
680+
)
681+
else:
682+
self.log.warning(
683+
"Found a previous (probably partial installed) virtual environment in %s, "
684+
"deleting and re-creating.",
685+
venv_path,
686+
)
687+
688+
shutil.rmtree(venv_path)
689+
690+
venv_path.mkdir(parents=True)
691+
self._prepare_venv(venv_path)
692+
hash_marker.write_text(hash_data, encoding="utf8")
693+
except Exception as e:
694+
shutil.rmtree(venv_path)
695+
raise AirflowException(f"Unable to create new virtual environment in {venv_path}") from e
696+
self.log.info("New Python virtual environment created in %s", venv_path)
697+
return venv_path
698+
616699
def execute_callable(self):
700+
if self.venv_cache_path:
701+
venv_path = self._ensure_venv_cache_exists(Path(self.venv_cache_path))
702+
python_path = venv_path / "bin" / "python"
703+
return self._execute_python_callable_in_subprocess(python_path, venv_path)
704+
617705
with TemporaryDirectory(prefix="venv") as tmp_dir:
618706
tmp_path = Path(tmp_dir)
619707
self._prepare_venv(tmp_path)
@@ -632,7 +720,7 @@ def _iter_serializable_context_keys(self):
632720

633721
class BranchPythonVirtualenvOperator(PythonVirtualenvOperator, BranchMixIn):
634722
"""
635-
A workflow can "branch" or follow a path after the execution of this task in a virtualenv.
723+
A workflow can "branch" or follow a path after the execution of this task in a virtual environment.
636724
637725
It derives the PythonVirtualenvOperator and expects a Python function that returns
638726
a single task_id or list of task_ids to follow. The task_id(s) returned
@@ -651,15 +739,15 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator):
651739
"""
652740
Run a function in a virtualenv that is not re-created.
653741
654-
Reused as is without the overhead of creating the virtualenv (with certain caveats).
742+
Reused as is without the overhead of creating the virtual environment (with certain caveats).
655743
656744
The function must be defined using def, and not be
657745
part of a class. All imports must happen inside the function
658746
and no variables outside the scope may be referenced. A global scope
659747
variable named virtualenv_string_args will be available (populated by
660748
string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
661749
can use a return value.
662-
Note that if your virtualenv runs in a different Python major version than Airflow,
750+
Note that if your virtual environment runs in a different Python major version than Airflow,
663751
you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
664752
Airflow through plugins. You can use string_args though.
665753
@@ -671,13 +759,13 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator):
671759
:ref:`howto/operator:ExternalPythonOperator`
672760
673761
:param python: Full path string (file-system specific) that points to a Python binary inside
674-
a virtualenv that should be used (in ``VENV/bin`` folder). Should be absolute path
762+
a virtual environment that should be used (in ``VENV/bin`` folder). Should be absolute path
675763
(so usually start with "/" or "X:/" depending on the filesystem/os used).
676764
:param python_callable: A python function with no references to outside variables,
677-
defined with def, which will be run in a virtualenv
765+
defined with def, which will be run in a virtual environment
678766
:param use_dill: Whether to use dill to serialize
679767
the args and result (pickle is default). This allow more complex types
680-
but if dill is not preinstalled in your venv, the task will fail with use_dill enabled.
768+
but if dill is not preinstalled in your virtual environment, the task will fail with use_dill enabled.
681769
:param op_args: A list of positional arguments to pass to python_callable.
682770
:param op_kwargs: A dict of keyword arguments to pass to python_callable.
683771
:param string_args: Strings that are present in the global var virtualenv_string_args,
@@ -697,7 +785,7 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator):
697785
exit code will be treated as a failure.
698786
"""
699787

700-
template_fields: Sequence[str] = tuple({"python"} | set(PythonOperator.template_fields))
788+
template_fields: Sequence[str] = tuple({"python"}.union(PythonOperator.template_fields))
701789

702790
def __init__(
703791
self,
@@ -749,7 +837,8 @@ def execute_callable(self):
749837
raise AirflowException(
750838
"Passing op_args or op_kwargs is not supported across different Python "
751839
"major versions for ExternalPythonOperator. Please use string_args."
752-
f"Sys version: {sys.version_info}. Venv version: {python_version_as_list_of_strings}"
840+
f"Sys version: {sys.version_info}. "
841+
f"Virtual environment version: {python_version_as_list_of_strings}"
753842
)
754843
with TemporaryDirectory(prefix="tmd") as tmp_dir:
755844
tmp_path = Path(tmp_dir)
@@ -776,11 +865,11 @@ def _is_pendulum_installed_in_target_env(self) -> bool:
776865
return True
777866
except Exception as e:
778867
if self.expect_pendulum:
779-
self.log.warning("When checking for Pendulum installed in venv got %s", e)
868+
self.log.warning("When checking for Pendulum installed in virtual environment got %s", e)
780869
self.log.warning(
781-
"Pendulum is not properly installed in the virtualenv "
870+
"Pendulum is not properly installed in the virtual environment "
782871
"Pendulum context keys will not be available. "
783-
"Please Install Pendulum or Airflow in your venv to access them."
872+
"Please Install Pendulum or Airflow in your virtual environment to access them."
784873
)
785874
return False
786875

@@ -805,7 +894,7 @@ def _get_airflow_version_from_target_env(self) -> str | None:
805894
return target_airflow_version
806895
except Exception as e:
807896
if self.expect_airflow:
808-
self.log.warning("When checking for Airflow installed in venv got %s", e)
897+
self.log.warning("When checking for Airflow installed in virtual environment got %s", e)
809898
self.log.warning(
810899
f"This means that Airflow is not properly installed by "
811900
f"{self.python}. Airflow context keys will not be available. "
@@ -819,8 +908,8 @@ class BranchExternalPythonOperator(ExternalPythonOperator, BranchMixIn):
819908
A workflow can "branch" or follow a path after the execution of this task.
820909
821910
Extends ExternalPythonOperator, so expects to get Python:
822-
virtualenv that should be used (in ``VENV/bin`` folder). Should be absolute path,
823-
so it can run on separate virtualenv similarly to ExternalPythonOperator.
911+
virtual environment that should be used (in ``VENV/bin`` folder). Should be absolute path,
912+
so it can run on separate virtual environment similarly to ExternalPythonOperator.
824913
"""
825914

826915
def execute(self, context: Context) -> Any:

0 commit comments

Comments
 (0)