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/03/14 19:29:48 UTC

[GitHub] [hive] sourabh912 opened a new pull request #3105: [WIP]HIVE-21456: Thrift over Http for Hive Metastore

sourabh912 opened a new pull request #3105:
URL: https://github.com/apache/hive/pull/3105


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This patch adds an initial version of supporting thrift over http in Hive Metastore. It supports SSL with HTTP. The following would be addressed in follow up patches:
   
   - JWT authentication mechanism. In the current patch, a user has to pass x-actor-username in the HTTP header request.
   - Cookie based authentication
   - XSRF-filters support
   - Data Compression like GZIP in HTTP mode
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   This is a new feature to support thrift over HTTP for Hive Metastore so as to let client connect to Hive metastore via Http
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Added unit and integration tests.


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


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

Posted by GitBox <gi...@apache.org>.
nrg4878 commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r828072598



##########
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:
       This seems to be an older version compared to what HS2 uses. Can we atleast use the same version as HS2? This will probably address some known CVEs

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveHttpMetaStore.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.hive.metastore.annotation.MetastoreUnitTest;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+
+@Category(MetastoreCheckinTest.class)
+public class TestRemoteHiveHttpMetaStore extends TestRemoteHiveMetaStore {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestRemoteHiveHttpMetaStore.class);
+
+  @Override
+  public void start() throws Exception {
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_TRANSPORT_MODE, "http");
+    //MetastoreConf.setBoolVar(conf, ConfVars.USE_SSL, true);
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYSTORE_PATH, "/home/sourabh/src/certs1/keystore");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYSTORE_PASSWORD, "password");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYSTORE_TYPE, "pkcs12");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYMANAGERFACTORY_ALGORITHM, "SunX509");

Review comment:
       nit: delete dead code.

##########
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:
       nit: delete commented code

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveHttpMetaStore.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.hive.metastore.annotation.MetastoreUnitTest;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+
+@Category(MetastoreCheckinTest.class)
+public class TestRemoteHiveHttpMetaStore extends TestRemoteHiveMetaStore {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestRemoteHiveHttpMetaStore.class);
+
+  @Override
+  public void start() throws Exception {
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_TRANSPORT_MODE, "http");
+    //MetastoreConf.setBoolVar(conf, ConfVars.USE_SSL, true);
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYSTORE_PATH, "/home/sourabh/src/certs1/keystore");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYSTORE_PASSWORD, "password");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYSTORE_TYPE, "pkcs12");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_KEYMANAGERFACTORY_ALGORITHM, "SunX509");
+    LOG.info("Attempting to start test remote metastore");
+    super.start();
+    LOG.info("Successfully started test remote metastore");
+  }
+
+  @Override
+  protected HiveMetaStoreClient createClient() throws Exception {
+    MetastoreConf.setVar(conf, ConfVars.METASTORE_CLIENT_THRIFT_TRANSPORT_MODE, "http");
+    //MetastoreConf.setBoolVar(conf, ConfVars.USE_SSL, true);
+    //MetastoreConf.setVar(conf, ConfVars.SSL_TRUSTSTORE_PATH, "/home/sourabh/src/certs1/truststore");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_TRUSTSTORE_PASSWORD, "password");
+    //MetastoreConf.setVar(conf, ConfVars.SSL_TRUSTMANAGERFACTORY_ALGORITHM, "SunX509");

Review comment:
       nit: delete dead code

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
##########
@@ -1356,6 +1356,13 @@ public static ConfVars getMetaConf(String name) {
         "Comma-separated list of tasks that will be started in separate threads.  These will be" +
             " started only when the metastore is running as a separate service.  They must " +
             "implement " + METASTORE_TASK_THREAD_CLASS),
+    THRIFT_TRANSPORT_MODE("metastore.server.thrift.transport.mode",
+        "hive.metastore.server.thrift.transport.mode", "binary",
+        "Transport mode for thrift server in Metastore. Can be binary or http"),
+    THRIFT_HTTP_PATH("metastore.server.thrift.http.path",

Review comment:
       So this config is not related to the transport. Its a path used for routing the service requests. I am ok leaving it as-as but if this is inconsistent with the HS2 side configuration for its http support, then I agree with you.

##########
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);
+
+    // This check is likely pointless, especially with the current state of the http
+    // servlet which respects whatever comes in. Putting this in place for the moment
+    // only to enable testing on an otherwise secure cluster.
+    LOG.info(" Checking if security is enabled");
+    if (UserGroupInformation.isSecurityEnabled()) {
+      LOG.info("Logging in via keytab while starting HTTP metastore");
+      // Handle renewal
+      String kerberosName = SecurityUtil.getServerPrincipal(MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL), "0.0.0.0");
+      String keyTabFile = MetastoreConf.getVar(conf, ConfVars.KERBEROS_KEYTAB_FILE);
+      UserGroupInformation.loginUserFromKeytab(kerberosName, keyTabFile);
+    } else {
+      LOG.info("Security is not enabled. Not logging in via keytab");
+    }
+
+    // TODO Bunch of http specific variables need to be defined here.
+    long maxMessageSize = MetastoreConf.getLongVar(conf, ConfVars.SERVER_MAX_MESSAGE_SIZE);
+    int minWorkerThreads = MetastoreConf.getIntVar(conf, ConfVars.SERVER_MIN_THREADS);
+    int maxWorkerThreads = MetastoreConf.getIntVar(conf, ConfVars.SERVER_MAX_THREADS);
+
+    boolean useCompactProtocol = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_COMPACT_PROTOCOL);
+
+    // Server thread pool
+    // Start with minWorkerThreads, expand till maxWorkerThreads and reject
+    // subsequent requests
+    String threadPoolName = "HiveServer2-HttpHandler-Pool";
+    ExecutorService executorService = new ThreadPoolExecutor(
+        minWorkerThreads, maxWorkerThreads, 60, TimeUnit.SECONDS,
+        new SynchronousQueue<>());
+
+    ExecutorThreadPool threadPool = new ExecutorThreadPool((ThreadPoolExecutor) executorService);
+
+    // HTTP Server
+    org.eclipse.jetty.server.Server server = new Server(threadPool);
+    server.setStopAtShutdown(true);
+
+    ServerConnector connector;
+
+    final HttpConfiguration httpServerConf = new HttpConfiguration();
+    // TODO: Read from Configuration

Review comment:
       nit: looks like this TODO is done. Can we delete the comment?

##########
File path: itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java
##########
@@ -67,6 +67,12 @@ public static void setMetastoreSslConf(HiveConf conf) {
             KEY_STORE_TRUST_STORE_PASSWORD);
   }
 
+  public static void setMetastoreHttpsConf(HiveConf conf) {
+    setMetastoreSslConf(conf);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.TRANSPORT_MODE, "http");

Review comment:
       +1

##########
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);
+
+    // This check is likely pointless, especially with the current state of the http
+    // servlet which respects whatever comes in. Putting this in place for the moment
+    // only to enable testing on an otherwise secure cluster.
+    LOG.info(" Checking if security is enabled");
+    if (UserGroupInformation.isSecurityEnabled()) {
+      LOG.info("Logging in via keytab while starting HTTP metastore");
+      // Handle renewal
+      String kerberosName = SecurityUtil.getServerPrincipal(MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL), "0.0.0.0");
+      String keyTabFile = MetastoreConf.getVar(conf, ConfVars.KERBEROS_KEYTAB_FILE);
+      UserGroupInformation.loginUserFromKeytab(kerberosName, keyTabFile);
+    } else {
+      LOG.info("Security is not enabled. Not logging in via keytab");
+    }
+
+    // TODO Bunch of http specific variables need to be defined here.
+    long maxMessageSize = MetastoreConf.getLongVar(conf, ConfVars.SERVER_MAX_MESSAGE_SIZE);
+    int minWorkerThreads = MetastoreConf.getIntVar(conf, ConfVars.SERVER_MIN_THREADS);
+    int maxWorkerThreads = MetastoreConf.getIntVar(conf, ConfVars.SERVER_MAX_THREADS);
+
+    boolean useCompactProtocol = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_COMPACT_PROTOCOL);
+
+    // Server thread pool
+    // Start with minWorkerThreads, expand till maxWorkerThreads and reject
+    // subsequent requests
+    String threadPoolName = "HiveServer2-HttpHandler-Pool";
+    ExecutorService executorService = new ThreadPoolExecutor(
+        minWorkerThreads, maxWorkerThreads, 60, TimeUnit.SECONDS,
+        new SynchronousQueue<>());
+
+    ExecutorThreadPool threadPool = new ExecutorThreadPool((ThreadPoolExecutor) executorService);
+
+    // HTTP Server
+    org.eclipse.jetty.server.Server server = new Server(threadPool);
+    server.setStopAtShutdown(true);
+
+    ServerConnector connector;
+
+    final HttpConfiguration httpServerConf = new HttpConfiguration();
+    // TODO: Read from Configuration
+    httpServerConf.setRequestHeaderSize(
+        MetastoreConf.getIntVar(conf, ConfVars.METASTORE_THRIFT_HTTP_REQUEST_HEADER_SIZE));
+    httpServerConf.setResponseHeaderSize(
+        MetastoreConf.getIntVar(conf, ConfVars.METASTORE_THRIFT_HTTP_RESPONSE_HEADER_SIZE));
+
+    final HttpConnectionFactory http = new HttpConnectionFactory(httpServerConf);
+
+    boolean useSsl  = MetastoreConf.getBoolVar(conf, ConfVars.USE_SSL);
+    String schemeName = useSsl ? "https" : "http";
+    if (useSsl) {
+      String keyStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_KEYSTORE_PATH).trim();
+      if (keyStorePath.isEmpty()) {
+        throw new IllegalArgumentException(ConfVars.SSL_KEYSTORE_PATH.toString()
+            + " Not configured for SSL connection");
+      }
+      String keyStorePassword =
+          MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.SSL_KEYSTORE_PASSWORD);
+      String keyStoreType =
+          MetastoreConf.getVar(conf, ConfVars.SSL_KEYSTORE_TYPE).trim();
+      String keyStoreAlgorithm =
+          MetastoreConf.getVar(conf, ConfVars.SSL_KEYMANAGERFACTORY_ALGORITHM).trim();
+
+      SslContextFactory sslContextFactory = new SslContextFactory();
+      // TODO: Add support for excluding protocols?

Review comment:
       nit: Looks like this TODO has already been implemented as welll

##########
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:
       Should this not be in HiveConf.java instead of MetastoreConf.java? Its my understanding that this is used by HMS client like HS2. So it should be set in the client side config?

##########
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",

Review comment:
       Should this not be in HiveConf.java instead of MetastoreConf.java? Its my understanding that this is used by HMS client like HS2. So it should be set in the client side config?




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


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

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r837245803



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -803,6 +765,114 @@ private void open() throws MetaException {
     snapshotActiveConf();
   }
 
+  private String[] getHttpUrlAndUser(URI store) throws MetaException {
+    boolean useSSL = MetastoreConf.getBoolVar(conf, ConfVars.USE_SSL);
+    String path = MetaStoreUtils.getHttpPath(MetastoreConf.getVar(conf, ConfVars.THRIFT_HTTP_PATH));
+    String httpUrl = (useSSL ? "https://" : "http://") + store.getHost() + ":" + store.getPort() + path;
+    String httpUser = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_PLAIN_USERNAME);

Review comment:
       we probably want to make sure that this config can only be changed by an admin




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


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

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r840360794



##########
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:
       https://issues.apache.org/jira/browse/HIVE-23498




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


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

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r827532603



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/SecurityUtils.java
##########
@@ -271,6 +286,31 @@ public static TTransport getSSLSocket(String host, int port, int loginTimeout,
     return getSSLSocketWithHttps(tSSLSocket);
   }
 
+  public static THttpClient getHttpSSLSocket(String httpsUrl, String trustStorePath,
+      String trustStorePasswd, String trustStoreAlgorithm, String trustStoreType)

Review comment:
       I did not find http server related code, for example, the server ssl  socket that can handle http request. Will the servlet handle that?

##########
File path: itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java
##########
@@ -67,6 +67,12 @@ public static void setMetastoreSslConf(HiveConf conf) {
             KEY_STORE_TRUST_STORE_PASSWORD);
   }
 
+  public static void setMetastoreHttpsConf(HiveConf conf) {
+    setMetastoreSslConf(conf);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.TRANSPORT_MODE, "http");

Review comment:
       Should it be THRIFT_TRANSPORT_MODE?

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -591,6 +592,31 @@ public void renamePartition(String catName, String dbname, String tableName, Lis
     client.rename_partition_req(req);
   }
 
+  /**
+   * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on.
+   * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*"
+   * @param httpPath
+   * @return
+   */
+  private String getHttpPath(String httpPath) {

Review comment:
       It is duplicated and has no use, should be removed?




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


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

Posted by GitBox <gi...@apache.org>.
sourabh912 commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r832503645



##########
File path: itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java
##########
@@ -67,6 +67,12 @@ public static void setMetastoreSslConf(HiveConf conf) {
             KEY_STORE_TRUST_STORE_PASSWORD);
   }
 
+  public static void setMetastoreHttpsConf(HiveConf conf) {
+    setMetastoreSslConf(conf);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.TRANSPORT_MODE, "http");

Review comment:
       Yes, it should be THRIFT_TRANSPORT_MODE.




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


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

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r837259862



##########
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:
       nit: Extra space?




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


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

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r837256170



##########
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);
+
+    // This check is likely pointless, especially with the current state of the http
+    // servlet which respects whatever comes in. Putting this in place for the moment
+    // only to enable testing on an otherwise secure cluster.
+    LOG.info(" Checking if security is enabled");
+    if (UserGroupInformation.isSecurityEnabled()) {
+      LOG.info("Logging in via keytab while starting HTTP metastore");
+      // Handle renewal
+      String kerberosName = SecurityUtil.getServerPrincipal(MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL), "0.0.0.0");
+      String keyTabFile = MetastoreConf.getVar(conf, ConfVars.KERBEROS_KEYTAB_FILE);
+      UserGroupInformation.loginUserFromKeytab(kerberosName, keyTabFile);
+    } else {
+      LOG.info("Security is not enabled. Not logging in via keytab");
+    }
+
+    // TODO Bunch of http specific variables need to be defined here.
+    long maxMessageSize = MetastoreConf.getLongVar(conf, ConfVars.SERVER_MAX_MESSAGE_SIZE);
+    int minWorkerThreads = MetastoreConf.getIntVar(conf, ConfVars.SERVER_MIN_THREADS);
+    int maxWorkerThreads = MetastoreConf.getIntVar(conf, ConfVars.SERVER_MAX_THREADS);
+
+    boolean useCompactProtocol = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_COMPACT_PROTOCOL);
+
+    // Server thread pool
+    // Start with minWorkerThreads, expand till maxWorkerThreads and reject
+    // subsequent requests
+    String threadPoolName = "HiveServer2-HttpHandler-Pool";
+    ExecutorService executorService = new ThreadPoolExecutor(
+        minWorkerThreads, maxWorkerThreads, 60, TimeUnit.SECONDS,
+        new SynchronousQueue<>());
+
+    ExecutorThreadPool threadPool = new ExecutorThreadPool((ThreadPoolExecutor) executorService);
+
+    // HTTP Server
+    org.eclipse.jetty.server.Server server = new Server(threadPool);
+    server.setStopAtShutdown(true);
+
+    ServerConnector connector;
+
+    final HttpConfiguration httpServerConf = new HttpConfiguration();
+    // TODO: Read from Configuration
+    httpServerConf.setRequestHeaderSize(
+        MetastoreConf.getIntVar(conf, ConfVars.METASTORE_THRIFT_HTTP_REQUEST_HEADER_SIZE));
+    httpServerConf.setResponseHeaderSize(
+        MetastoreConf.getIntVar(conf, ConfVars.METASTORE_THRIFT_HTTP_RESPONSE_HEADER_SIZE));
+
+    final HttpConnectionFactory http = new HttpConnectionFactory(httpServerConf);
+
+    boolean useSsl  = MetastoreConf.getBoolVar(conf, ConfVars.USE_SSL);
+    String schemeName = useSsl ? "https" : "http";
+    if (useSsl) {
+      String keyStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_KEYSTORE_PATH).trim();
+      if (keyStorePath.isEmpty()) {
+        throw new IllegalArgumentException(ConfVars.SSL_KEYSTORE_PATH.toString()
+            + " Not configured for SSL connection");
+      }
+      String keyStorePassword =
+          MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.SSL_KEYSTORE_PASSWORD);
+      String keyStoreType =
+          MetastoreConf.getVar(conf, ConfVars.SSL_KEYSTORE_TYPE).trim();
+      String keyStoreAlgorithm =
+          MetastoreConf.getVar(conf, ConfVars.SSL_KEYMANAGERFACTORY_ALGORITHM).trim();
+
+      SslContextFactory sslContextFactory = new SslContextFactory();
+      // TODO: Add support for excluding protocols?
+      String[] excludedProtocols = MetastoreConf.getVar(conf, ConfVars.SSL_PROTOCOL_BLACKLIST).split(",");
+      LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols));
+      sslContextFactory.addExcludeProtocols(excludedProtocols);
+      LOG.info("HTTP Server SSL: SslContextFactory.getExcludeProtocols = "
+          + Arrays.toString(sslContextFactory.getExcludeProtocols()));
+      sslContextFactory.setKeyStorePath(keyStorePath);
+      sslContextFactory.setKeyStorePassword(keyStorePassword);
+      sslContextFactory.setKeyStoreType(keyStoreType);
+      sslContextFactory.setKeyManagerFactoryAlgorithm(keyStoreAlgorithm);
+      connector = new ServerConnector(server, sslContextFactory, http);
+    } else {
+      connector = new ServerConnector(server, http);
+    }
+    connector.setPort(port);
+    connector.setReuseAddress(true);
+    // TODO: What should the idle timeout be for the metastore. 30 minutes seems a little too long.
+    connector.setIdleTimeout(120 * 1000);
+    // TODO: AcceptQueueSize needs to be higher for HMS
+    connector.setAcceptQueueSize(maxWorkerThreads);
+    // TODO: Connection keepalive configuration?
+
+    server.addConnector(connector);
+
+    TProcessor processor;
+
+    // All of this code can be re-used.
+    //  Eventually move the HTTP and Binary parts into separate
+    //  classes.
+    final TProtocolFactory protocolFactory;
+    final TProtocolFactory inputProtoFactory;
+    if (useCompactProtocol) {
+      protocolFactory = new TCompactProtocol.Factory();
+      inputProtoFactory = new TCompactProtocol.Factory(maxMessageSize, maxMessageSize);
+    } else {
+      protocolFactory = new TBinaryProtocol.Factory();
+      inputProtoFactory = new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize);
+    }
+
+    // TODO ZZZ: HMS seems to have it's own set of handlers. Not sure if the threadpool here is actually required.
+    HMSHandler baseHandler = new HMSHandler("new db based metaserver",
+        conf);
+    IHMSHandler handler = newRetryingHMSHandler(baseHandler, conf);
+    processor = new ThriftHiveMetastore.Processor<>(handler);
+    LOG.info("Starting DB backed MetaStore Server with generic processor");

Review comment:
       Could we have some more logs for DEBUG level about the configuration we used to start the HMS?




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


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

Posted by GitBox <gi...@apache.org>.
sourabh912 commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r832506838



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -591,6 +592,31 @@ public void renamePartition(String catName, String dbname, String tableName, Lis
     client.rename_partition_req(req);
   }
 
+  /**
+   * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on.
+   * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*"
+   * @param httpPath
+   * @return
+   */
+  private String getHttpPath(String httpPath) {

Review comment:
       Sure. I will remove it.




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r837249852



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
##########
@@ -1356,6 +1356,13 @@ public static ConfVars getMetaConf(String name) {
         "Comma-separated list of tasks that will be started in separate threads.  These will be" +
             " started only when the metastore is running as a separate service.  They must " +
             "implement " + METASTORE_TASK_THREAD_CLASS),
+    THRIFT_TRANSPORT_MODE("metastore.server.thrift.transport.mode",

Review comment:
       We should consider which of these should kept secret, and which of these could be changed on session level




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


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

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r837253159



##########
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:
       Do we disable the TRACE request for our server?




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


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

Posted by GitBox <gi...@apache.org>.
sourabh912 commented on pull request #3105:
URL: https://github.com/apache/hive/pull/3105#issuecomment-1067222894


   @harishjp @nrg4878 @hsnusonic @yongzhi @saihemanth-cloudera : Please review the PR and share your feedback.


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


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

Posted by GitBox <gi...@apache.org>.
sourabh912 commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r832503645



##########
File path: itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java
##########
@@ -67,6 +67,12 @@ public static void setMetastoreSslConf(HiveConf conf) {
             KEY_STORE_TRUST_STORE_PASSWORD);
   }
 
+  public static void setMetastoreHttpsConf(HiveConf conf) {
+    setMetastoreSslConf(conf);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.TRANSPORT_MODE, "http");

Review comment:
       Yes, THRIFT_TRANSPORT_MODE is better variable name. 




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


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

Posted by GitBox <gi...@apache.org>.
sourabh912 commented on a change in pull request #3105:
URL: https://github.com/apache/hive/pull/3105#discussion_r832508524



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
##########
@@ -1356,6 +1356,13 @@ public static ConfVars getMetaConf(String name) {
         "Comma-separated list of tasks that will be started in separate threads.  These will be" +
             " started only when the metastore is running as a separate service.  They must " +
             "implement " + METASTORE_TASK_THREAD_CLASS),
+    THRIFT_TRANSPORT_MODE("metastore.server.thrift.transport.mode",
+        "hive.metastore.server.thrift.transport.mode", "binary",
+        "Transport mode for thrift server in Metastore. Can be binary or http"),
+    THRIFT_HTTP_PATH("metastore.server.thrift.http.path",

Review comment:
       I think a better config name would be `metastore.server.thrift.transport.http.path`. Thoughts? 

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/SecurityUtils.java
##########
@@ -271,6 +286,31 @@ public static TTransport getSSLSocket(String host, int port, int loginTimeout,
     return getSSLSocketWithHttps(tSSLSocket);
   }
 
+  public static THttpClient getHttpSSLSocket(String httpsUrl, String trustStorePath,
+      String trustStorePasswd, String trustStoreAlgorithm, String trustStoreType)

Review comment:
       The server side code for this is in HiveMetaStore.java at https://github.com/apache/hive/pull/3105/files#diff-00e70b6958060aa36762b21bf16676f83af01c1e09b56816aecc6abe7c8ac866R683




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