You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "vihangk1 (via GitHub)" <gi...@apache.org> on 2023/03/09 05:46:52 UTC

[GitHub] [hive] vihangk1 commented on a diff in pull request #4104: [HIVE-27129] Add enhanced support for Hive Metastore Client http support

vihangk1 commented on code in PR #4104:
URL: https://github.com/apache/hive/pull/4104#discussion_r1130492724


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -35,25 +35,15 @@
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;

Review Comment:
   We generally don't use wild-card imports. Can you please revert this part and explicitly add the necessary imports (You may have to change your IDE settings so that it doesn't do this for automatically).



##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -621,31 +611,84 @@ private <T extends TTransport> T configureThriftMaxMessageSize(T transport) {
     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());
+        LOG.warn(parts[0].trim() + "=" + parts[1].trim());

Review Comment:
   Is this log needed? If not, let's remove it or change it to debug.



##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -621,31 +611,84 @@ private <T extends TTransport> T configureThriftMaxMessageSize(T transport) {
     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());
+        LOG.warn(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
   header in http request
    */
-  private THttpClient createHttpClient(URI store, boolean useSSL) throws MetaException,
-      TTransportException {
+  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);
+  }
 
+  protected HttpClientBuilder createHttpClientBuilder() throws MetaException {

Review Comment:
   Please annotate this with `@VisibleForTesting`



##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -659,42 +702,16 @@ public void process(HttpRequest httpRequest, HttpContext httpContext)
       }
       final String httpUser = user;
       httpClientBuilder.addInterceptorFirst(new HttpRequestInterceptor() {
-        @Override
-        public void process(HttpRequest httpRequest, HttpContext httpContext)
+        @Override public void process(HttpRequest httpRequest, HttpContext httpContext)

Review Comment:
   same here.



##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreHttpHeaders.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+@Category(MetastoreCheckinTest.class) public class TestHiveMetastoreHttpHeaders {

Review Comment:
   Please reformat this file once you have your IDE setup with the formatter above to fix code-style issues.



##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreHttpHeaders.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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 static 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);
+          assertEquals(testHeaderVal1, header1.getValue());
+          Header header2 = httpRequest.getFirstHeader(testHeaderKey2);
+          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);

Review Comment:
   This method can be in a `BeforeClass` I guess. Also, you probably need a `AfterClass` to shutdown the metastore that you started just to make sure that you clean up after the test completes.



##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -621,31 +611,84 @@ private <T extends TTransport> T configureThriftMaxMessageSize(T transport) {
     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());
+        LOG.warn(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
   header in http request
    */
-  private THttpClient createHttpClient(URI store, boolean useSSL) throws MetaException,
-      TTransportException {
+  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);
+  }
 
+  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");
       if (jwtToken == null || jwtToken.isEmpty()) {
         LOG.debug("No jwt token set in environment variable: HMS_JWT");
-        throw new MetaException("For auth mode JWT, valid signed jwt token must be provided in the "
-            + "environment variable HMS_JWT");
+        throw new MetaException(

Review Comment:
   nit, unintentional new line?



##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -621,31 +611,84 @@ private <T extends TTransport> T configureThriftMaxMessageSize(T transport) {
     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());
+        LOG.warn(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
   header in http request
    */
-  private THttpClient createHttpClient(URI store, boolean useSSL) throws MetaException,
-      TTransportException {
+  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);
+  }
 
+  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");
       if (jwtToken == null || jwtToken.isEmpty()) {
         LOG.debug("No jwt token set in environment variable: HMS_JWT");
-        throw new MetaException("For auth mode JWT, valid signed jwt token must be provided in the "
-            + "environment variable HMS_JWT");
+        throw new MetaException(
+            "For auth mode JWT, valid signed jwt token must be provided in the " + "environment variable HMS_JWT");
       }
       httpClientBuilder.addInterceptorFirst(new HttpRequestInterceptor() {
-        @Override
-        public void process(HttpRequest httpRequest, HttpContext httpContext)
+        @Override public void process(HttpRequest httpRequest, HttpContext httpContext)

Review Comment:
   You may want set up your IDE to change the it's default formatting style. There are instructions in https://cwiki.apache.org/confluence/display/Hive/HowToContribute#HowToContribute-CodingConventions
   
   Generally the `@Override` is in a separate line above `public void process` as per the convention.
   



##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreHttpHeaders.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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 static org.junit.Assert.*;

Review Comment:
   avoid wildcard imports.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org