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

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new 9aa5be398 [hive] Support HiveCatalog for Hive 3.x and certain 
metastore client classes with constructors only for 2.x (#2828)
9aa5be398 is described below

commit 9aa5be398d0a0f2fffe85a649d3f00c77f521dfc
Author: tsreaper <[email protected]>
AuthorDate: Thu Feb 22 13:32:14 2024 +0800

    [hive] Support HiveCatalog for Hive 3.x and certain metastore client 
classes with constructors only for 2.x (#2828)
---
 .../hive/RetryingMetaStoreClientFactory.java       | 64 ++++++++++++++--------
 1 file changed, 42 insertions(+), 22 deletions(-)

diff --git 
a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java
 
b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java
index abf3598e3..bc5a88705 100644
--- 
a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java
+++ 
b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java
@@ -93,6 +93,33 @@ public class RetryingMetaStoreClientFactory {
                                                     new ConcurrentHashMap<>(),
                                                     clientClassName,
                                                     true))
+                    // for hive 3.x,
+                    // and some metastore client classes providing 
constructors only for 2.x
+                    .put(
+                            new Class<?>[] {
+                                Configuration.class,
+                                Class[].class,
+                                Object[].class,
+                                ConcurrentHashMap.class,
+                                String.class
+                            },
+                            (getProxyMethod, hiveConf, clientClassName) ->
+                                    (IMetaStoreClient)
+                                            getProxyMethod.invoke(
+                                                    null,
+                                                    hiveConf,
+                                                    new Class[] {
+                                                        HiveConf.class,
+                                                        
HiveMetaHookLoader.class,
+                                                        Boolean.class
+                                                    },
+                                                    new Object[] {
+                                                        hiveConf,
+                                                        (HiveMetaHookLoader) 
(tbl -> null),
+                                                        true
+                                                    },
+                                                    new ConcurrentHashMap<>(),
+                                                    clientClassName))
                     .build();
 
     // If clientClassName is HiveMetaStoreClient,
@@ -116,38 +143,31 @@ public class RetryingMetaStoreClientFactory {
                     .build();
 
     public IMetaStoreClient createClient(HiveConf hiveConf, String 
clientClassName) {
-        Method getProxy = null;
-        HiveMetastoreProxySupplier supplier = null;
-        RuntimeException methodNotFound =
-                new RuntimeException(
-                        "Failed to find desired getProxy method from 
RetryingMetaStoreClient");
         Map<Class<?>[], HiveMetastoreProxySupplier> suppliers =
                 new LinkedHashMap<>(PROXY_SUPPLIERS);
         if (HiveMetaStoreClient.class.getName().equals(clientClassName)) {
             suppliers.putAll(PROXY_SUPPLIERS_SHADED);
         }
+
+        RuntimeException failToCreate =
+                new RuntimeException(
+                        "Failed to create the desired metastore client (class 
name: "
+                                + clientClassName
+                                + ")");
         for (Entry<Class<?>[], HiveMetastoreProxySupplier> entry : 
suppliers.entrySet()) {
             Class<?>[] classes = entry.getKey();
             try {
-                getProxy = RetryingMetaStoreClient.class.getMethod("getProxy", 
classes);
-                supplier = entry.getValue();
-            } catch (NoSuchMethodException e) {
-                methodNotFound.addSuppressed(e);
+                Method getProxy = 
RetryingMetaStoreClient.class.getMethod("getProxy", classes);
+                HiveMetastoreProxySupplier supplier = entry.getValue();
+                IMetaStoreClient client = supplier.get(getProxy, hiveConf, 
clientClassName);
+                return 
isNullOrWhitespaceOnly(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
+                        ? client
+                        : HiveMetaStoreClient.newSynchronizedClient(client);
+            } catch (Exception e) {
+                failToCreate.addSuppressed(e);
             }
         }
-        if (getProxy == null) {
-            throw methodNotFound;
-        }
-
-        IMetaStoreClient client;
-        try {
-            client = supplier.get(getProxy, hiveConf, clientClassName);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        return 
isNullOrWhitespaceOnly(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
-                ? client
-                : HiveMetaStoreClient.newSynchronizedClient(client);
+        throw failToCreate;
     }
 
     /** Function interface for creating hive metastore proxy. */

Reply via email to