This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new ddcd474a5e Fix check if `virtualenv` is installed in 
`PythonVirtualenvOperator` (#32939)
ddcd474a5e is described below

commit ddcd474a5e2ce4568cca646eb1f5bce32b4ba0ed
Author: Shahar Epstein <[email protected]>
AuthorDate: Sun Jul 30 07:57:22 2023 +0300

    Fix check if `virtualenv` is installed in `PythonVirtualenvOperator` 
(#32939)
---
 airflow/operators/python.py    |  4 ++--
 tests/operators/test_python.py | 10 ++++++++++
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/airflow/operators/python.py b/airflow/operators/python.py
index 574fe5b0e3..3a10cf4f73 100644
--- a/airflow/operators/python.py
+++ b/airflow/operators/python.py
@@ -17,11 +17,11 @@
 # under the License.
 from __future__ import annotations
 
+import importlib
 import inspect
 import logging
 import os
 import pickle
-import shutil
 import subprocess
 import sys
 import types
@@ -540,7 +540,7 @@ class 
PythonVirtualenvOperator(_BasePythonVirtualenvOperator):
                 "major versions for PythonVirtualenvOperator. Please use 
string_args."
                 f"Sys version: {sys.version_info}. Venv version: 
{python_version}"
             )
-        if not shutil.which("virtualenv"):
+        if importlib.util.find_spec("virtualenv") is None:
             raise AirflowException("PythonVirtualenvOperator requires 
virtualenv, please install it.")
         if not requirements:
             self.requirements: list[str] | str = []
diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py
index 534a6be5a9..ba296f77a4 100644
--- a/tests/operators/test_python.py
+++ b/tests/operators/test_python.py
@@ -847,6 +847,16 @@ class 
TestPythonVirtualenvOperator(BaseTestPythonVirtualenvOperator):
         kwargs["python_version"] = python_version
         return kwargs
 
+    @mock.patch("airflow.operators.python.importlib")
+    def test_virtuenv_not_installed(self, importlib):
+        importlib.util.find_spec.return_value = None
+        with pytest.raises(AirflowException, match="requires virtualenv"):
+
+            def f():
+                pass
+
+            self.run_as_task(f)
+
     def test_add_dill(self):
         def f():
             """Ensure dill is correctly installed."""

Reply via email to