You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by da...@apache.org on 2018/01/08 19:25:38 UTC

incubator-airflow git commit: [AIRFLOW-1971] Propagate hive config on impersonation

Repository: incubator-airflow
Updated Branches:
  refs/heads/master e46cde418 -> e9c1ac588


[AIRFLOW-1971] Propagate hive config on impersonation

Currently, if hive specific settings are defined
in the configuration
file, they are not being propagated when using
impersonation.
We need to propagate this configuration down to
the impersonated
process.

Closes #2920 from edgarRd/erod-propagate-hive-conf


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

Branch: refs/heads/master
Commit: e9c1ac588a698b88f916d6f47531d7e0dc63237d
Parents: e46cde4
Author: Edgar Rodriguez <ed...@airbnb.com>
Authored: Mon Jan 8 11:25:12 2018 -0800
Committer: Dan Davydov <da...@airbnb.com>
Committed: Mon Jan 8 11:25:16 2018 -0800

----------------------------------------------------------------------
 airflow/task/task_runner/base_task_runner.py |  1 +
 tests/dags/test_impersonation_custom.py      | 10 ++++++++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e9c1ac58/airflow/task/task_runner/base_task_runner.py
----------------------------------------------------------------------
diff --git a/airflow/task/task_runner/base_task_runner.py b/airflow/task/task_runner/base_task_runner.py
index 664a873..11a4745 100644
--- a/airflow/task/task_runner/base_task_runner.py
+++ b/airflow/task/task_runner/base_task_runner.py
@@ -64,6 +64,7 @@ class BaseTaskRunner(LoggingMixin):
                 'smtp': cfg_dict.get('smtp', {}),
                 'scheduler': cfg_dict.get('scheduler', {}),
                 'webserver': cfg_dict.get('webserver', {}),
+                'hive': cfg_dict.get('hive', {}),  # we should probably generalized this
             }
             temp_fd, cfg_path = mkstemp()
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e9c1ac58/tests/dags/test_impersonation_custom.py
----------------------------------------------------------------------
diff --git a/tests/dags/test_impersonation_custom.py b/tests/dags/test_impersonation_custom.py
index 6f35b38..b6dd9c8 100644
--- a/tests/dags/test_impersonation_custom.py
+++ b/tests/dags/test_impersonation_custom.py
@@ -41,7 +41,17 @@ def print_today():
     print('Today is {}'.format(dt.strftime('%Y-%m-%d')))
 
 
+def check_hive_conf():
+    from airflow import configuration as conf
+    assert conf.get('hive', 'default_hive_mapred_queue') == 'airflow'
+
+
 PythonOperator(
     python_callable=print_today,
     task_id='exec_python_fn',
     dag=dag)
+
+PythonOperator(
+    python_callable=check_hive_conf,
+    task_id='exec_check_hive_conf_fn',
+    dag=dag)