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 5f8481c799 Move hive_cli_params to hook parameters (#28101)
5f8481c799 is described below

commit 5f8481c799ea6bd742a5ccc194b2ff8dbe01eab5
Author: Jarek Potiuk <[email protected]>
AuthorDate: Tue Dec 6 23:00:45 2022 +0100

    Move hive_cli_params to hook parameters (#28101)
    
    Those parameters belong to the Hook, not to the connection
    definition, so we should actually be able to specify them there.
    
    You can also now specify ``hive_cli_params`` in the HiveOperator
    and it will pass the parameters to the HiveCliHook created under
    the hood.
---
 airflow/providers/apache/hive/CHANGELOG.rst                   |  9 +++++++++
 airflow/providers/apache/hive/hooks/hive.py                   | 11 ++++++-----
 airflow/providers/apache/hive/operators/hive.py               |  3 +++
 airflow/providers/apache/hive/provider.yaml                   |  1 +
 .../connections/hive_cli.rst                                  |  2 --
 5 files changed, 19 insertions(+), 7 deletions(-)

diff --git a/airflow/providers/apache/hive/CHANGELOG.rst 
b/airflow/providers/apache/hive/CHANGELOG.rst
index a6d98da752..7086ab4a06 100644
--- a/airflow/providers/apache/hive/CHANGELOG.rst
+++ b/airflow/providers/apache/hive/CHANGELOG.rst
@@ -24,6 +24,15 @@
 Changelog
 ---------
 
+5.0.0
+.....
+
+Breaking changes
+~~~~~~~~~~~~~~~~
+
+The ``hive_cli_params`` from connection were moved to the Hook. If you have 
extra parameters defined in your
+connections as ``hive_cli_params`` extra, you should move them to the DAG 
where your HiveOperator is used.
+
 4.1.1
 .....
 
diff --git a/airflow/providers/apache/hive/hooks/hive.py 
b/airflow/providers/apache/hive/hooks/hive.py
index 430032d8f1..8621098de4 100644
--- a/airflow/providers/apache/hive/hooks/hive.py
+++ b/airflow/providers/apache/hive/hooks/hive.py
@@ -63,10 +63,8 @@ class HiveCliHook(BaseHook):
     traditional CLI. To enable ``beeline``, set the use_beeline param in the
     extra field of your connection as in ``{ "use_beeline": true }``
 
-    Note that you can also set default hive CLI parameters using the
-    ``hive_cli_params`` to be used in your connection as in
-    ``{"hive_cli_params": "-hiveconf mapred.job.tracker=some.jobtracker:444"}``
-    Parameters passed here can be overridden by run_cli's hive_conf param
+    Note that you can also set default hive CLI parameters by passing 
``hive_cli_params``
+    space separated list of parameters to add to the hive command.
 
     The extra connection parameter ``auth`` gets passed as in the ``jdbc``
     connection string as is.
@@ -78,6 +76,8 @@ class HiveCliHook(BaseHook):
         Possible settings include: VERY_HIGH, HIGH, NORMAL, LOW, VERY_LOW
     :param mapred_job_name: This name will appear in the jobtracker.
         This can make monitoring easier.
+    :param hive_cli_params: Space separated list of hive command parameters to 
add to the
+        hive command.
     """
 
     conn_name_attr = "hive_cli_conn_id"
@@ -92,10 +92,11 @@ class HiveCliHook(BaseHook):
         mapred_queue: str | None = None,
         mapred_queue_priority: str | None = None,
         mapred_job_name: str | None = None,
+        hive_cli_params: str = "",
     ) -> None:
         super().__init__()
         conn = self.get_connection(hive_cli_conn_id)
-        self.hive_cli_params: str = conn.extra_dejson.get("hive_cli_params", 
"")
+        self.hive_cli_params: str = hive_cli_params
         self.use_beeline: bool = conn.extra_dejson.get("use_beeline", False)
         self.auth = conn.extra_dejson.get("auth", "noSasl")
         self.conn = conn
diff --git a/airflow/providers/apache/hive/operators/hive.py 
b/airflow/providers/apache/hive/operators/hive.py
index 23f6c32edd..776ada6979 100644
--- a/airflow/providers/apache/hive/operators/hive.py
+++ b/airflow/providers/apache/hive/operators/hive.py
@@ -87,6 +87,7 @@ class HiveOperator(BaseOperator):
         mapred_queue: str | None = None,
         mapred_queue_priority: str | None = None,
         mapred_job_name: str | None = None,
+        hive_cli_params: str = "",
         **kwargs: Any,
     ) -> None:
         super().__init__(**kwargs)
@@ -102,6 +103,7 @@ class HiveOperator(BaseOperator):
         self.mapred_queue = mapred_queue
         self.mapred_queue_priority = mapred_queue_priority
         self.mapred_job_name = mapred_job_name
+        self.hive_cli_params = hive_cli_params
 
         job_name_template = conf.get_mandatory_value(
             "hive",
@@ -124,6 +126,7 @@ class HiveOperator(BaseOperator):
             mapred_queue=self.mapred_queue,
             mapred_queue_priority=self.mapred_queue_priority,
             mapred_job_name=self.mapred_job_name,
+            hive_cli_params=self.hive_cli_params,
         )
 
     def prepare_template(self) -> None:
diff --git a/airflow/providers/apache/hive/provider.yaml 
b/airflow/providers/apache/hive/provider.yaml
index df936955a3..926808f3d8 100644
--- a/airflow/providers/apache/hive/provider.yaml
+++ b/airflow/providers/apache/hive/provider.yaml
@@ -22,6 +22,7 @@ description: |
   `Apache Hive <https://hive.apache.org/>`__
 
 versions:
+  - 5.0.0
   - 4.1.1
   - 4.1.0
   - 4.0.1
diff --git a/docs/apache-airflow-providers-apache-hive/connections/hive_cli.rst 
b/docs/apache-airflow-providers-apache-hive/connections/hive_cli.rst
index 1c7af80d51..cd9e61ccaf 100644
--- a/docs/apache-airflow-providers-apache-hive/connections/hive_cli.rst
+++ b/docs/apache-airflow-providers-apache-hive/connections/hive_cli.rst
@@ -68,8 +68,6 @@ Extra (optional)
     Specify the extra parameters (as json dictionary) that can be used in Hive 
CLI connection.
     The following parameters are all optional:
 
-    * ``hive_cli_params``
-      Specify an object CLI params for use with Beeline CLI and Hive CLI.
     * ``use_beeline``
       Specify as ``True`` if using the Beeline CLI. Default is ``False``.
     * ``auth``

Reply via email to