airflow-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From davy...@apache.org
Subject incubator-airflow git commit: [AIRFLOW-1893][AIRFLOW-1901] Propagate PYTHONPATH when using impersonation
Date Tue, 12 Dec 2017 00:48:11 GMT
Repository: incubator-airflow
Updated Branches:
  refs/heads/master c6f46b102 -> 9d9727a80


[AIRFLOW-1893][AIRFLOW-1901] Propagate PYTHONPATH when using impersonation

When using impersonation via `run_as_user`, the
PYTHONPATH environment
variable is not propagated hence there may be
issues when depending on
specific custom packages used in DAGs.
This PR propagates only the PYTHONPATH in the
process creating the
sub-process with impersonation, if any.

Tested in staging environment; impersonation tests
in airflow are not very portable and fixing them
would take additional work, leaving as TODO and
tracking with jira ticket: https://issues.apache.o
rg/jira/browse/AIRFLOW-1901.

Closes #2860 from edgarRd/erod-
pythonpath_run_as_user


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/9d9727a8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/9d9727a8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/9d9727a8

Branch: refs/heads/master
Commit: 9d9727a80a3948615a4085d5168c24394fde5c84
Parents: c6f46b1
Author: Edgar Rodriguez <edgar.rodriguez@airbnb.com>
Authored: Mon Dec 11 16:47:47 2017 -0800
Committer: Dan Davydov <dan.davydov@airbnb.com>
Committed: Mon Dec 11 16:47:49 2017 -0800

----------------------------------------------------------------------
 airflow/task_runner/base_task_runner.py |  8 +++++
 run_unit_tests.sh                       |  4 +++
 tests/dags/test_impersonation_custom.py | 47 ++++++++++++++++++++++++++++
 tests/impersonation.py                  | 20 ++++++++++++
 tests/test_utils/fake_datetime.py       |  2 +-
 5 files changed, 80 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/9d9727a8/airflow/task_runner/base_task_runner.py
----------------------------------------------------------------------
diff --git a/airflow/task_runner/base_task_runner.py b/airflow/task_runner/base_task_runner.py
index 849c0c7..664a873 100644
--- a/airflow/task_runner/base_task_runner.py
+++ b/airflow/task_runner/base_task_runner.py
@@ -25,6 +25,9 @@ from airflow import configuration as conf
 from tempfile import mkstemp
 
 
+PYTHONPATH_VAR = 'PYTHONPATH'
+
+
 class BaseTaskRunner(LoggingMixin):
     """
     Runs Airflow task instances by invoking the `airflow run` command with raw
@@ -77,8 +80,13 @@ class BaseTaskRunner(LoggingMixin):
             with os.fdopen(temp_fd, 'w') as temp_file:
                 json.dump(cfg_subset, temp_file)
 
+            # propagate PYTHONPATH environment variable
+            pythonpath_value = os.environ.get(PYTHONPATH_VAR, '')
             popen_prepend = ['sudo', '-H', '-u', self.run_as_user]
 
+            if pythonpath_value:
+                popen_prepend.append('{}={}'.format(PYTHONPATH_VAR, pythonpath_value))
+
         self._cfg_path = cfg_path
         self._command = popen_prepend + self._task_instance.command_as_list(
             raw=True,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/9d9727a8/run_unit_tests.sh
----------------------------------------------------------------------
diff --git a/run_unit_tests.sh b/run_unit_tests.sh
index d53842c..c78358d 100755
--- a/run_unit_tests.sh
+++ b/run_unit_tests.sh
@@ -25,6 +25,10 @@ export AIRFLOW__TESTSECTION__TESTKEY=testvalue
 # use Airflow 2.0-style imports
 export AIRFLOW_USE_NEW_IMPORTS=1
 
+# add test/contrib to PYTHONPATH
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+export PYTHONPATH=$PYTHONPATH:${DIR}/tests/test_utils
+
 # any argument received is overriding the default nose execution arguments:
 
 nose_args=$@

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/9d9727a8/tests/dags/test_impersonation_custom.py
----------------------------------------------------------------------
diff --git a/tests/dags/test_impersonation_custom.py b/tests/dags/test_impersonation_custom.py
new file mode 100644
index 0000000..6f35b38
--- /dev/null
+++ b/tests/dags/test_impersonation_custom.py
@@ -0,0 +1,47 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from airflow.models import DAG
+from airflow.operators.python_operator import PythonOperator
+from datetime import datetime
+
+# AIRFLOW-1893 - Originally, impersonation tests were incomplete missing the use case when
+# DAGs access custom packages usually made available through the PYTHONPATH environment
+# variable. This file includes a DAG that imports a custom package made available and if
+# run via the previous implementation of impersonation, will fail by not being able to
+# import the custom package.
+# This DAG is used to test that impersonation propagates the PYTHONPATH environment
+# variable correctly.
+from tests.test_utils.fake_datetime import FakeDatetime
+
+DEFAULT_DATE = datetime(2016, 1, 1)
+
+args = {
+    'owner': 'airflow',
+    'start_date': DEFAULT_DATE,
+    'run_as_user': 'airflow_test_user'
+}
+
+dag = DAG(dag_id='impersonation_with_custom_pkg', default_args=args)
+
+
+def print_today():
+    dt = FakeDatetime.utcnow()
+    print('Today is {}'.format(dt.strftime('%Y-%m-%d')))
+
+
+PythonOperator(
+    python_callable=print_today,
+    task_id='exec_python_fn',
+    dag=dag)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/9d9727a8/tests/impersonation.py
----------------------------------------------------------------------
diff --git a/tests/impersonation.py b/tests/impersonation.py
index 5355c9a..203769c 100644
--- a/tests/impersonation.py
+++ b/tests/impersonation.py
@@ -16,6 +16,7 @@ import errno
 import os
 import subprocess
 import unittest
+import logging
 
 from airflow import jobs, models
 from airflow.utils.state import State
@@ -27,6 +28,7 @@ TEST_DAG_FOLDER = os.path.join(
 DEFAULT_DATE = datetime(2015, 1, 1)
 TEST_USER = 'airflow_test_user'
 
+logger = logging.getLogger(__name__)
 
 # TODO(aoen): Adding/remove a user as part of a test is very bad (especially if the user
 # already existed to begin with on the OS), this logic should be moved into a test
@@ -35,12 +37,16 @@ TEST_USER = 'airflow_test_user'
 # without any manual modification of the sudoers file by the agent that is running these
 # tests.
 
+
 class ImpersonationTest(unittest.TestCase):
     def setUp(self):
         self.dagbag = models.DagBag(
             dag_folder=TEST_DAG_FOLDER,
             include_examples=False,
         )
+        logger.info('Loaded DAGS:')
+        logger.info(self.dagbag.dagbag_report())
+
         try:
             subprocess.check_output(['sudo', 'useradd', '-m', TEST_USER, '-g',
                                      str(os.getegid())])
@@ -74,6 +80,7 @@ class ImpersonationTest(unittest.TestCase):
             task=dag.get_task(task_id),
             execution_date=DEFAULT_DATE)
         ti.refresh_from_db()
+
         self.assertEqual(ti.state, State.SUCCESS)
 
     def test_impersonation(self):
@@ -109,3 +116,16 @@ class ImpersonationTest(unittest.TestCase):
             )
         finally:
             del os.environ['AIRFLOW__CORE__DEFAULT_IMPERSONATION']
+
+    def test_impersonation_custom(self):
+        """
+        Tests that impersonation using a unix user works with custom packages in
+        PYTHONPATH
+        """
+        # PYTHONPATH is already set in script triggering tests
+        assert 'PYTHONPATH' in os.environ
+
+        self.run_backfill(
+            'impersonation_with_custom_pkg',
+            'exec_python_fn'
+        )

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/9d9727a8/tests/test_utils/fake_datetime.py
----------------------------------------------------------------------
diff --git a/tests/test_utils/fake_datetime.py b/tests/test_utils/fake_datetime.py
index 9b8102f..a2856f6 100644
--- a/tests/test_utils/fake_datetime.py
+++ b/tests/test_utils/fake_datetime.py
@@ -21,4 +21,4 @@ class FakeDatetime(datetime):
     """
 
     def __new__(cls, *args, **kwargs):
-        return date.__new__(datetime, *args, **kwargs)
+        return datetime.__new__(datetime, *args, **kwargs)


Mime
View raw message