You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2023/03/10 21:14:55 UTC

[hive] branch master updated: HIVE-27129: Add enhanced support for Hive Metastore Client http support (Junlin Zeng, reviewed by Vihang Karajgaonkar)

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

vihangk1 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 63a02465957 HIVE-27129: Add enhanced support for Hive Metastore Client http support (Junlin Zeng, reviewed by Vihang Karajgaonkar)
63a02465957 is described below

commit 63a02465957abe5311d287e79cd9cb949170dedf
Author: Junlin Zeng <69...@users.noreply.github.com>
AuthorDate: Fri Mar 10 13:14:43 2023 -0800

    HIVE-27129: Add enhanced support for Hive Metastore Client http support (Junlin Zeng, reviewed by Vihang Karajgaonkar)
    
    (Closes #4104)
---
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |  95 ++++++++++++------
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |   3 +
 .../metastore/TestHiveMetastoreHttpHeaders.java    | 108 +++++++++++++++++++++
 3 files changed, 176 insertions(+), 30 deletions(-)

diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 50412dd17be..e229ab1a0b4 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -46,6 +46,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NoSuchElementException;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
@@ -54,6 +55,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.security.auth.login.LoginException;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
 
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -621,6 +623,24 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return transport;
   }
 
+  private Map<String, String> getAdditionalHeaders() {
+    Map<String, String> headers = new HashMap<>();
+    String keyValuePairs = MetastoreConf.getVar(conf,
+        ConfVars.METASTORE_CLIENT_ADDITIONAL_HEADERS);
+    try {
+      List<String> headerKeyValues =
+          Splitter.on(',').trimResults().splitToList(keyValuePairs);
+      for (String header : headerKeyValues) {
+        String[] parts = header.split("=");
+        headers.put(parts[0].trim(), parts[1].trim());
+      }
+    } catch (Exception ex) {
+      LOG.warn("Could not parse the headers provided in "
+          + ConfVars.METASTORE_CLIENT_ADDITIONAL_HEADERS, ex);
+    }
+    return headers;
+  }
+
   /*
   Creates a THttpClient if HTTP mode is enabled. If Client auth mode is set to JWT,
   then the method fetches JWT from environment variable: HMS_JWT and sets in auth
@@ -629,10 +649,47 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   private THttpClient createHttpClient(URI store, boolean useSSL) throws MetaException,
       TTransportException {
     String path = MetaStoreUtils.getHttpPath(MetastoreConf.getVar(conf, ConfVars.THRIFT_HTTP_PATH));
-    String httpUrl = (useSSL ? "https://" : "http://") + store.getHost() + ":" + store.getPort() + path;
+    String urlScheme;
+    if (useSSL || Objects.equals(store.getScheme(), "https")) {
+      urlScheme = "https://";
+    } else {
+      urlScheme = "http://";
+    }
+    String httpUrl = urlScheme + store.getHost() + ":" + store.getPort() + path;
+
+    HttpClientBuilder httpClientBuilder = createHttpClientBuilder();
+    THttpClient tHttpClient;
+    try {
+      if (useSSL) {
+        String trustStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_PATH).trim();
+        if (trustStorePath.isEmpty()) {
+          throw new IllegalArgumentException(ConfVars.SSL_TRUSTSTORE_PATH + " Not configured for SSL connection");
+        }
+        String trustStorePassword = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.SSL_TRUSTSTORE_PASSWORD);
+        String trustStoreType = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_TYPE).trim();
+        String trustStoreAlgorithm = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTMANAGERFACTORY_ALGORITHM).trim();
+        tHttpClient =
+            SecurityUtils.getThriftHttpsClient(httpUrl, trustStorePath, trustStorePassword, trustStoreAlgorithm,
+                trustStoreType, httpClientBuilder);
+      } else {
+        tHttpClient = new THttpClient(httpUrl, httpClientBuilder.build());
+      }
+    } catch (Exception e) {
+      if (e instanceof TTransportException) {
+        throw (TTransportException) e;
+      } else {
+        throw new MetaException("Failed to create http transport client to url: " + httpUrl + ". Error:" + e);
+      }
+    }
+    LOG.debug("Created thrift http client for URL: " + httpUrl);
+    return configureThriftMaxMessageSize(tHttpClient);
+  }
 
+  @VisibleForTesting
+  protected HttpClientBuilder createHttpClientBuilder() throws MetaException {
     HttpClientBuilder httpClientBuilder = HttpClientBuilder.create();
     String authType = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_AUTH_MODE);
+    Map<String, String> additionalHeaders = getAdditionalHeaders();
     if (authType.equalsIgnoreCase("jwt")) {
       // fetch JWT token from environment and set it in Auth Header in HTTP request
       String jwtToken = System.getenv("HMS_JWT");
@@ -646,6 +703,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
         public void process(HttpRequest httpRequest, HttpContext httpContext)
             throws HttpException, IOException {
           httpRequest.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + jwtToken);
+          for (Map.Entry<String, String> entry : additionalHeaders.entrySet()) {
+            httpRequest.addHeader(entry.getKey(), entry.getValue());
+          }
         }
       });
     } else {
@@ -663,38 +723,13 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
         public void process(HttpRequest httpRequest, HttpContext httpContext)
             throws HttpException, IOException {
           httpRequest.addHeader(MetaStoreUtils.USER_NAME_HTTP_HEADER, httpUser);
+          for (Map.Entry<String, String> entry : additionalHeaders.entrySet()) {
+            httpRequest.addHeader(entry.getKey(), entry.getValue());
+          }
         }
       });
     }
-    THttpClient tHttpClient;
-    try {
-      if (useSSL) {
-        String trustStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_PATH).trim();
-        if (trustStorePath.isEmpty()) {
-          throw new IllegalArgumentException(ConfVars.SSL_TRUSTSTORE_PATH
-              + " Not configured for SSL connection");
-        }
-        String trustStorePassword =
-            MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.SSL_TRUSTSTORE_PASSWORD);
-        String trustStoreType =
-            MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_TYPE).trim();
-        String trustStoreAlgorithm =
-            MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTMANAGERFACTORY_ALGORITHM).trim();
-        tHttpClient = SecurityUtils.getThriftHttpsClient(httpUrl, trustStorePath, trustStorePassword,
-            trustStoreAlgorithm, trustStoreType, httpClientBuilder);
-      }  else {
-        tHttpClient = new THttpClient(httpUrl, httpClientBuilder.build());
-      }
-    } catch (Exception e) {
-      if (e instanceof TTransportException) {
-        throw (TTransportException)e;
-      } else {
-        throw new MetaException("Failed to create http transport client to url: " + httpUrl
-            + ". Error:" + e);
-      }
-    }
-    LOG.debug("Created thrift http client for URL: " + httpUrl);
-    return configureThriftMaxMessageSize(tHttpClient);
+    return httpClientBuilder;
   }
 
   private TTransport createBinaryClient(URI store, boolean useSSL) throws TTransportException,
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index 0578e4b1c00..db73f0a8514 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -1591,6 +1591,9 @@ public class MetastoreConf {
                     " and password. Any other value is ignored right now but may be used later."
                 + "If JWT- Supported only in HTTP transport mode. If set, HMS Client will pick the value of JWT from "
                 + "environment variable HMS_JWT and set it in Authorization header in http request"),
+    METASTORE_CLIENT_ADDITIONAL_HEADERS("metastore.client.http.additional.headers",
+        "hive.metastore.client.http.additional.headers", "",
+        "Comma separated list of headers which are passed to the metastore service in the http headers"),
     METASTORE_CLIENT_PLAIN_USERNAME("metastore.client.plain.username",
             "hive.metastore.client.plain.username",  "",
         "The username used by the metastore client when " +
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreHttpHeaders.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreHttpHeaders.java
new file mode 100644
index 00000000000..0f33f0a4154
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreHttpHeaders.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.http.Header;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.protocol.HttpContext;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+
+import org.junit.Assert;
+
+@Category(MetastoreCheckinTest.class)
+public class TestHiveMetastoreHttpHeaders {
+  private static Configuration conf;
+  private static HiveMetaStoreClient msc;
+  private static int port;
+  private static final String testHeaderKey1 = "X-XXXX";
+  private static final String testHeaderVal1 = "yyyy";
+  private static final String testHeaderKey2 = "X-ZZZZ";
+  private static final String testHeaderVal2 = "aaaa";
+
+  static class TestHiveMetaStoreClient extends HiveMetaStoreClient {
+    public TestHiveMetaStoreClient(Configuration conf) throws MetaException {
+      super(conf);
+    }
+
+    @Override
+    protected HttpClientBuilder createHttpClientBuilder() throws MetaException {
+      HttpClientBuilder builder = super.createHttpClientBuilder();
+      builder.addInterceptorLast(new HttpRequestInterceptor() {
+        @Override
+        public void process(HttpRequest httpRequest, HttpContext httpContext) throws HttpException, IOException {
+          Header header1 = httpRequest.getFirstHeader(testHeaderKey1);
+          Assert.assertEquals(testHeaderVal1, header1.getValue());
+          Header header2 = httpRequest.getFirstHeader(testHeaderKey2);
+          Assert.assertEquals(testHeaderVal2, header2.getValue());
+        }
+      });
+      return builder;
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.EXECUTE_SET_UGI, false);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_TRANSPORT_MODE, "http");
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.METASTORE_CLIENT_THRIFT_TRANSPORT_MODE, "http");
+    port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+  }
+
+  @Test
+  public void testHttpHeaders() throws Exception {
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.METASTORE_CLIENT_ADDITIONAL_HEADERS,
+        String.format("%s=%s,%s=%s", testHeaderKey1, testHeaderVal1, testHeaderKey2, testHeaderVal2));
+    msc = new TestHiveMetaStoreClient(conf);
+    Database db = new DatabaseBuilder().setName("testHttpHeader").create(msc, conf);
+    msc.dropDatabase(db.getName());
+  }
+
+  @Test
+  public void testIllegalHttpHeaders() throws Exception {
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.METASTORE_CLIENT_ADDITIONAL_HEADERS,
+        String.format("%s%s", testHeaderKey1, testHeaderVal1));
+    msc = new TestHiveMetaStoreClient(conf);
+    boolean exceptionThrown = false;
+    try {
+      Database db = new DatabaseBuilder().setName("testHttpHeader").create(msc, conf);
+      msc.dropDatabase(db.getName());
+    } catch (Exception e) {
+      exceptionThrown = true;
+    }
+    Assert.assertTrue("Illegal header should invoke thrown exception", exceptionThrown);
+  }
+}