You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2022/04/01 00:32:55 UTC

[GitHub] [hive] sourabh912 commented on a change in pull request #3105: [WIP]HIVE-21456: Thrift over Http for Hive Metastore

sourabh912 commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r840095321



##########
File path: standalone-metastore/pom.xml
##########
@@ -103,6 +103,7 @@
     <spotbugs.version>4.0.3</spotbugs.version>
     <caffeine.version>2.8.4</caffeine.version>
     <slf4j.version>1.7.30</slf4j.version>
+    <httpcomponents.core.version>4.4.10</httpcomponents.core.version>

Review comment:
       Sure. I will upgrade it to 4.4.13 which is what HS2 has as of today.

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
##########
@@ -343,21 +366,162 @@ public static void startMetaStore(int port, HadoopThriftAuthBridge bridge,
     startMetaStore(port, bridge, conf, false, null);
   }
 
-  /**
-   * Start Metastore based on a passed {@link HadoopThriftAuthBridge}.
-   *
-   * @param port The port on which the Thrift server will start to serve
-   * @param bridge
-   * @param conf Configuration overrides
-   * @param startMetaStoreThreads Start the background threads (initiator, cleaner, statsupdater, etc.)
-   * @param startedBackgroundThreads If startMetaStoreThreads is true, this AtomicBoolean will be switched to true,
-   *  when all of the background threads are scheduled. Useful for testing purposes to wait
-   *  until the MetaStore is fully initialized.
-   * @throws Throwable
-   */
-  public static void startMetaStore(int port, HadoopThriftAuthBridge bridge,
-      Configuration conf, boolean startMetaStoreThreads, AtomicBoolean startedBackgroundThreads) throws Throwable {
-    isMetaStoreRemote = true;
+  public static boolean isThriftServerRunning() {
+    return thriftServer != null && thriftServer.isRunning();
+  }
+
+  // TODO: Is it worth trying to use a server that supports HTTP/2?
+  //  Does the Thrift http client support this?
+
+  public static ThriftServer startHttpMetastore(int port, Configuration conf)
+      throws Exception {
+    LOG.info("Attempting to start http metastore server on port: {}", port);

Review comment:
       I didn't get the context of the question but we don't disable TRACE for the server.

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
##########
@@ -1496,6 +1503,23 @@ public static ConfVars getMetaConf(String name) {
     USERS_IN_ADMIN_ROLE("metastore.users.in.admin.role", "hive.users.in.admin.role", "", false,
         "Comma separated list of users who are in admin role for bootstrapping.\n" +
             "More users can be added in ADMIN role later."),
+    // TODO: Should we have a separate config for the metastoreclient or THRIFT_TRANSPORT_MODE
+    // would suffice ?
+    METASTORE_CLIENT_THRIFT_TRANSPORT_MODE("metastore.client.thrift.transport.mode",
+        "hive.metastore.client.thrift.transport.mode", "binary",
+        "Transport mode to be used by the metastore client. It should be the same as " + THRIFT_TRANSPORT_MODE),
+    METASTORE_CLIENT_THRIFT_HTTP_PATH("metastore.client.thrift.http.path",

Review comment:
       The reason I kept it here is because there are other client configs like METASTORE_CLIENT_AUTH_MODE, METASTORE_CLIENT_PLAIN_USERNAME that are defined in this conf. If you think we should move all client side confs to HiveConf, we can do it in a follow up patch. Thoughts? 

##########
File path: itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java
##########
@@ -437,15 +439,36 @@ public void testConnectionWrongCertCN() throws Exception {
    * Test HMS server with SSL
    * @throws Exception
    */
+  @Ignore
   @Test
   public void testMetastoreWithSSL() throws Exception {
     testSSLHMS(true);
   }
 
+  /**
+   * Test HMS server with Http + SSL
+   * @throws Exception
+   */
+  @Test
+  public void testMetastoreWithHttps() throws Exception {
+    // MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.EVENT_DB_NOTIFICATION_API_AUTH, false);
+    //MetastoreConf.setVar(conf, MetastoreConf.ConfVars.METASTORE_CLIENT_TRANSPORT_MODE, "http");

Review comment:
       Done.

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HmsThriftHttpServlet.java
##########
@@ -0,0 +1,116 @@
+/* * 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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Enumeration;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.TServlet;
+
+public class HmsThriftHttpServlet extends TServlet {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(HmsThriftHttpServlet.class);
+
+  private static final String X_USER = MetaStoreUtils.USER_NAME_HTTP_HEADER;
+
+  private final boolean isSecurityEnabled;
+
+  public HmsThriftHttpServlet(TProcessor processor,
+      TProtocolFactory inProtocolFactory, TProtocolFactory outProtocolFactory) {
+    super(processor, inProtocolFactory, outProtocolFactory);
+    // This should ideally be reveiving an instance of the Configuration which is used for the check
+    isSecurityEnabled = UserGroupInformation.isSecurityEnabled();
+  }
+
+  public HmsThriftHttpServlet(TProcessor processor,
+      TProtocolFactory protocolFactory) {
+    super(processor, protocolFactory);
+    isSecurityEnabled = UserGroupInformation.isSecurityEnabled();
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request,
+      HttpServletResponse response) throws ServletException, IOException {
+
+    Enumeration<String> headerNames = request.getHeaderNames();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(" Logging headers in request");

Review comment:
       Done 




-- 
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