Repository: kudu
Updated Branches:
  refs/heads/master a8d088e11 -> 2c572cbd4


Add unstable client APIs to fetch HMS integration configuration status

The Impala/Kudu integration must know whether the Kudu cluster is
configured with the HMS integration enabled. This commit adds an
unstable API to the Java client and a private API to the Kudu client to
lookup this information. The new C++ API is utilized by the HMS tools to
get the HMS connection information without having to use the GetFlags
API, which requires admin privileges.

The patch required adding support for enabling the HMS integration to
the Java mini cluster.

Change-Id: Iddb89787ed35c41e85f1d9bf953c4c228dcafcdb
Reviewed-on: http://gerrit.cloudera.org:8080/11121
Reviewed-by: Adar Dembo <a...@cloudera.com>
Tested-by: Kudu Jenkins


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/2c572cbd
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/2c572cbd
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/2c572cbd

Branch: refs/heads/master
Commit: 2c572cbd4dbf4c35d5d85521cae6067722e346a0
Parents: a8d088e
Author: Dan Burkert <danburk...@apache.org>
Authored: Fri Aug 3 16:14:03 2018 -0700
Committer: Dan Burkert <danburk...@apache.org>
Committed: Fri Aug 24 22:25:00 2018 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/AsyncKuduClient.java | 50 ++++++++++++++++++-
 .../apache/kudu/client/HiveMetastoreConfig.java | 52 ++++++++++++++++++++
 .../java/org/apache/kudu/client/KuduClient.java | 14 ++++++
 .../org/apache/kudu/client/MiniKuduCluster.java | 18 +++++--
 .../apache/kudu/client/TestMiniKuduCluster.java | 15 ++++++
 src/kudu/client/client-internal.cc              | 11 ++++-
 src/kudu/client/client-internal.h               |  9 +++-
 src/kudu/client/client.cc                       | 17 +++++++
 src/kudu/client/client.h                        | 19 +++++++
 src/kudu/hms/mini_hms.cc                        | 26 ++++++----
 src/kudu/hms/mini_hms.h                         | 10 +++-
 src/kudu/master/master.proto                    | 27 ++++++++++
 src/kudu/master/master_service.cc               | 11 +++++
 src/kudu/mini-cluster/external_mini_cluster.cc  |  1 +
 src/kudu/tools/tool_action_hms.cc               | 34 +++++--------
 15 files changed, 271 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
----------------------------------------------------------------------
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
index 8e97baf..8c1e032 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
@@ -294,6 +294,10 @@ public class AsyncKuduClient implements AutoCloseable {
   @GuardedBy("sessions")
   private final Set<AsyncKuduSession> sessions = new HashSet<>();
 
+  /** The Hive Metastore configuration of the most recently connected-to 
master. */
+  @GuardedBy("this")
+  private HiveMetastoreConfig hiveMetastoreConfig = null;
+
   // Since RPCs to the masters also go through RpcProxy, we need to treat them 
as if they were a
   // normal table. We'll use the following fake table name to identify places 
where we need special
   // handling.
@@ -815,7 +819,9 @@ public class AsyncKuduClient implements AutoCloseable {
     // We have no authn data -- connect to the master, which will fetch
     // new info.
     return getMasterTableLocationsPB(null)
-        .addCallback(new MasterLookupCB(masterTable, null, 1))
+        .addCallback(new MasterLookupCB(masterTable,
+                                        /* partitionKey */ null,
+                                        /* requestedBatchSize */ 1))
         .addCallback(new Callback<byte[], Object>() {
           @Override
           public byte[] call(Object ignored) {
@@ -827,6 +833,35 @@ public class AsyncKuduClient implements AutoCloseable {
   }
 
   /**
+   * Get the Hive Metastore configuration of the most recently connected-to 
leader master, or
+   * {@code null} if the Hive Metastore integration is not enabled.
+   */
+  @InterfaceAudience.LimitedPrivate("Impala")
+  @InterfaceStability.Unstable
+  public Deferred<HiveMetastoreConfig> getHiveMetastoreConfig() {
+    // If we've already connected to the master, use the config we received 
when we connected.
+    if (hasConnectedToMaster) {
+      synchronized (this) {
+        return Deferred.fromResult(hiveMetastoreConfig);
+      }
+    }
+    // We have no Metastore config -- connect to the master, which will fetch 
new info.
+    return getMasterTableLocationsPB(null)
+        .addCallback(new MasterLookupCB(masterTable,
+                                        /* partitionKey */ null,
+                                        /* requestedBatchSize */ 1))
+        .addCallback(new Callback<HiveMetastoreConfig, Object>() {
+          @Override
+          public HiveMetastoreConfig call(Object ignored) {
+            // Connecting to the cluster should have also fetched the 
metastore config.
+            synchronized (AsyncKuduClient.this) {
+              return hiveMetastoreConfig;
+            }
+          }
+        });
+  }
+
+  /**
    * Import data allowing this client to authenticate to the cluster.
    * This will typically be used before making any connections to servers
    * in the cluster.
@@ -1543,6 +1578,19 @@ public class AsyncKuduClient implements AutoCloseable {
                         e.getMessage());
                   }
                 }
+
+                HiveMetastoreConfig hiveMetastoreConfig = null;
+                Master.ConnectToMasterResponsePB respPb = 
resp.getConnectResponse();
+                if (respPb.hasHmsConfig()) {
+                  Master.HiveMetastoreConfig metastoreConf = 
respPb.getHmsConfig();
+                  hiveMetastoreConfig = new 
HiveMetastoreConfig(metastoreConf.getHmsUris(),
+                                                                
metastoreConf.getHmsSaslEnabled(),
+                                                                
metastoreConf.getHmsUuid());
+                }
+                synchronized (AsyncKuduClient.this) {
+                  AsyncKuduClient.this.hiveMetastoreConfig = 
hiveMetastoreConfig;
+                }
+
                 hasConnectedToMaster = true;
 
                 // Translate the located master into a TableLocations

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/java/kudu-client/src/main/java/org/apache/kudu/client/HiveMetastoreConfig.java
----------------------------------------------------------------------
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/HiveMetastoreConfig.java
 
b/java/kudu-client/src/main/java/org/apache/kudu/client/HiveMetastoreConfig.java
new file mode 100644
index 0000000..cb479cd
--- /dev/null
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/HiveMetastoreConfig.java
@@ -0,0 +1,52 @@
+// 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.kudu.client;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * The Hive Metastore configuration of a Kudu cluster.
+ */
+@InterfaceAudience.LimitedPrivate("Impala")
+@InterfaceStability.Unstable
+class HiveMetastoreConfig {
+  private final String hiveMetastoreUris;
+  private final boolean hiveMetastoreSaslEnabled;
+  private final String hiveMetastoreUuid;
+
+  HiveMetastoreConfig(String hiveMetastoreUris,
+                      boolean hiveMetastoreSaslEnabled,
+                      String hiveMetastoreUuid) {
+    this.hiveMetastoreUris = hiveMetastoreUris;
+    this.hiveMetastoreSaslEnabled = hiveMetastoreSaslEnabled;
+    this.hiveMetastoreUuid = hiveMetastoreUuid;
+  }
+
+  public String getHiveMetastoreUris() {
+    return hiveMetastoreUris;
+  }
+
+  public boolean getHiveMetastoreSaslEnabled() {
+    return hiveMetastoreSaslEnabled;
+  }
+
+  public String getHiveMetastoreUuid() {
+    return hiveMetastoreUuid;
+  }
+}

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
----------------------------------------------------------------------
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
index 55c0920..dfe1e40 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Executor;
 
+import com.stumbleupon.async.Callback;
 import com.stumbleupon.async.Deferred;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -81,6 +82,19 @@ public class KuduClient implements AutoCloseable {
     return asyncClient.hasLastPropagatedTimestamp();
   }
 
+
+  /**
+   * Returns the Hive Metastore configuration of the cluster.
+   *
+   * @return the Hive Metastore configuration of the cluster
+   * @throws KuduException if the configuration can not be retrieved
+   */
+  @InterfaceAudience.LimitedPrivate("Impala")
+  @InterfaceStability.Unstable
+  public HiveMetastoreConfig getHiveMetastoreConfig() throws KuduException {
+    return joinAndHandleException(asyncClient.getHiveMetastoreConfig());
+  }
+
   /**
    * Create a table on the cluster with the specified name, schema, and table 
configurations.
    * @param name the table's name

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
----------------------------------------------------------------------
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
index d0714b2..dccddff 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
@@ -36,6 +36,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.net.HostAndPort;
 
+import org.apache.kudu.Common;
 import org.apache.kudu.Common.HostPortPB;
 import org.apache.kudu.tools.Tool.ControlShellRequestPB;
 import org.apache.kudu.tools.Tool.ControlShellResponsePB;
@@ -96,6 +97,7 @@ public class MiniKuduCluster implements AutoCloseable {
   private final String clusterRoot;
 
   private MiniKdcOptionsPB kdcOptionsPb;
+  private final Common.HmsMode hmsMode;
 
   private MiniKuduCluster(boolean enableKerberos,
       int numMasters,
@@ -103,13 +105,15 @@ public class MiniKuduCluster implements AutoCloseable {
       List<String> extraTserverFlags,
       List<String> extraMasterFlags,
       MiniKdcOptionsPB kdcOptionsPb,
-      String clusterRoot) {
+      String clusterRoot,
+      Common.HmsMode hmsMode) {
     this.enableKerberos = enableKerberos;
     this.numMasters = numMasters;
     this.numTservers = numTservers;
     this.extraTserverFlags = ImmutableList.copyOf(extraTserverFlags);
     this.extraMasterFlags = ImmutableList.copyOf(extraMasterFlags);
     this.kdcOptionsPb = kdcOptionsPb;
+    this.hmsMode = hmsMode;
 
     if (clusterRoot == null) {
       // If a cluster root was not set, create a  unique temp directory to use.
@@ -205,6 +209,7 @@ public class MiniKuduCluster implements AutoCloseable {
             .setNumMasters(numMasters)
             .setNumTservers(numTservers)
             .setEnableKerberos(enableKerberos)
+            .setHmsMode(hmsMode)
             .addAllExtraMasterFlags(extraMasterFlags)
             .addAllExtraTserverFlags(extraTserverFlags)
             .setMiniKdcOptions(kdcOptionsPb)
@@ -531,8 +536,8 @@ public class MiniKuduCluster implements AutoCloseable {
     private final List<String> extraMasterFlags = new ArrayList<>();
     private String clusterRoot = null;
 
-    private MiniKdcOptionsPB.Builder kdcOptionsPb =
-        MiniKdcOptionsPB.newBuilder();
+    private MiniKdcOptionsPB.Builder kdcOptionsPb = 
MiniKdcOptionsPB.newBuilder();
+    private Common.HmsMode hmsMode = Common.HmsMode.NONE;
 
     public MiniKuduClusterBuilder numMasters(int numMasters) {
       this.numMasters = numMasters;
@@ -553,6 +558,11 @@ public class MiniKuduCluster implements AutoCloseable {
       return this;
     }
 
+    public MiniKuduClusterBuilder enableHiveMetastoreIntegration() {
+      hmsMode = Common.HmsMode.ENABLE_METASTORE_INTEGRATION;
+      return this;
+    }
+
     /**
      * Adds a new flag to be passed to the Tablet Server daemons on start.
      * @return this instance
@@ -600,7 +610,7 @@ public class MiniKuduCluster implements AutoCloseable {
           new MiniKuduCluster(enableKerberos,
               numMasters, numTservers,
               extraTserverFlags, extraMasterFlags,
-              kdcOptionsPb.build(), clusterRoot);
+              kdcOptionsPb.build(), clusterRoot, hmsMode);
       try {
         cluster.start();
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
----------------------------------------------------------------------
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
index 0adafee..9579c0e 100644
--- 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
+++ 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
@@ -14,6 +14,8 @@
 package org.apache.kudu.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -82,6 +84,19 @@ public class TestMiniKuduCluster {
       KuduClient client = new 
KuduClientBuilder(cluster.getMasterAddresses()).build();
       ListTablesResponse resp = client.getTablesList();
       assertTrue(resp.getTablesList().isEmpty());
+      assertNull(client.getHiveMetastoreConfig());
+    }
+  }
+
+  @Test(timeout = 50000)
+  public void testHiveMetastoreIntegration() throws Exception {
+    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
+                                                      .numMasters(NUM_MASTERS)
+                                                      
.numTservers(NUM_TABLET_SERVERS)
+                                                      
.enableHiveMetastoreIntegration()
+                                                      .build()) {
+      KuduClient client = new 
KuduClientBuilder(cluster.getMasterAddresses()).build();
+      assertNotNull(client.getHiveMetastoreConfig());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/client/client-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.cc 
b/src/kudu/client/client-internal.cc
index 335a23a..ecea7e9 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -357,7 +357,8 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
     vector<uint32_t> required_feature_flags);
 
 KuduClient::Data::Data()
-    : latest_observed_timestamp_(KuduClient::kNoTimestamp) {
+    : hive_metastore_sasl_enabled_(false),
+      latest_observed_timestamp_(KuduClient::kNoTimestamp) {
 }
 
 KuduClient::Data::~Data() {
@@ -707,6 +708,12 @@ void KuduClient::Data::ConnectedToClusterCb(
       for (const auto& hostport : connect_response.master_addrs()) {
         master_hostports_.emplace_back(HostPort(hostport.host(), 
hostport.port()));
       }
+
+      const auto& hive_config = connect_response.hms_config();
+      hive_metastore_uris_ = hive_config.hms_uris();
+      hive_metastore_sasl_enabled_ = hive_config.hms_sasl_enabled();
+      hive_metastore_uuid_ = hive_config.hms_uuid();
+
       master_proxy_.reset(new MasterServiceProxy(messenger_, leader_addr, 
leader_hostname));
       master_proxy_->set_user_credentials(user_credentials_);
     }
@@ -827,7 +834,7 @@ HostPort KuduClient::Data::leader_master_hostport() const {
   return leader_master_hostport_;
 }
 
-const vector<HostPort>& KuduClient::Data::master_hostports() const {
+vector<HostPort> KuduClient::Data::master_hostports() const {
   std::lock_guard<simple_spinlock> l(leader_master_lock_);
   return master_hostports_;
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/client/client-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.h 
b/src/kudu/client/client-internal.h
index 467d5ec..9c2ecfe 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -191,7 +191,7 @@ class KuduClient::Data {
 
   HostPort leader_master_hostport() const;
 
-  const std::vector<HostPort>& master_hostports() const;
+  std::vector<HostPort> master_hostports() const;
 
   uint64_t GetLatestObservedTimestamp() const;
 
@@ -260,6 +260,13 @@ class KuduClient::Data {
   // leader master in ConnectedToClusterCb.
   std::vector<HostPort> master_hostports_;
 
+  // The Hive Metastore configuration of the most recently connected to leader
+  // master, or an empty string if the leader master is not configured to
+  // integrate with a Hive Metastore.
+  std::string hive_metastore_uris_;
+  bool hive_metastore_sasl_enabled_;
+  std::string hive_metastore_uuid_;
+
   // Proxy to the leader master.
   std::shared_ptr<master::MasterServiceProxy> master_proxy_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/client/client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index b2b4f1b..1c22df0 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -19,6 +19,7 @@
 
 #include <cstdlib>
 #include <memory>
+#include <mutex>
 #include <ostream>
 #include <set>
 #include <string>
@@ -145,6 +146,7 @@ MAKE_ENUM_LIMITS(kudu::client::KuduScanner::OrderMode,
 struct tm;
 
 namespace kudu {
+class simple_spinlock;
 namespace client {
 
 class ResourceMetrics;
@@ -635,6 +637,21 @@ Status KuduClient::ExportAuthenticationCredentials(string* 
authn_creds) const {
   return Status::OK();
 }
 
+string KuduClient::GetHiveMetastoreUris() const {
+  std::lock_guard<simple_spinlock> l(data_->leader_master_lock_);
+  return data_->hive_metastore_uris_;
+}
+
+bool KuduClient::GetHiveMetastoreSaslEnabled() const {
+  std::lock_guard<simple_spinlock> l(data_->leader_master_lock_);
+  return data_->hive_metastore_sasl_enabled_;
+}
+
+string KuduClient::GetHiveMetastoreUuid() const {
+  std::lock_guard<simple_spinlock> l(data_->leader_master_lock_);
+  return data_->hive_metastore_uuid_;
+}
+
 ////////////////////////////////////////////////////////////
 // KuduTableCreator
 ////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index cfe62eb..aec3aef 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -536,6 +536,25 @@ class KUDU_EXPORT KuduClient : public 
sp::enable_shared_from_this<KuduClient> {
   /// @return Status object for the operation.
   Status ExportAuthenticationCredentials(std::string* authn_creds) const;
 
+  // @return the configured Hive Metastore URIs on the most recently connected 
to
+  //    leader master, or an empty string if the Hive Metastore integration is 
not
+  //    enabled.
+  std::string KUDU_NO_EXPORT GetHiveMetastoreUris() const;
+
+  // @return the configured Hive Metastore SASL (Kerberos) configuration on 
the most
+  //    recently connected to leader master, or an arbitrary value if the Hive
+  //    Metastore integration is not enabled.
+  bool KUDU_NO_EXPORT GetHiveMetastoreSaslEnabled() const;
+
+  // @return a unique ID which identifies the Hive Metastore instance, if the
+  //    cluster is configured with the Hive Metastore integration, or an
+  //    arbitrary value if the Hive Metastore integration is not enabled.
+  //
+  // @note this is provided on a best-effort basis, as not all Hive Metastore
+  //    versions which Kudu is compatible with include the necessary APIs. See
+  //    HIVE-16452 for more info.
+  std::string KUDU_NO_EXPORT GetHiveMetastoreUuid() const;
+
  private:
   class KUDU_NO_EXPORT Data;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/hms/mini_hms.cc
----------------------------------------------------------------------
diff --git a/src/kudu/hms/mini_hms.cc b/src/kudu/hms/mini_hms.cc
index e52247d..e27615a 100644
--- a/src/kudu/hms/mini_hms.cc
+++ b/src/kudu/hms/mini_hms.cc
@@ -75,6 +75,10 @@ void MiniHms::EnableKerberos(string krb5_conf,
   protection_ = protection;
 }
 
+void MiniHms::SetDataRoot(string data_root) {
+  data_root_ = data_root;
+}
+
 namespace {
 Status FindHomeDir(const char* name, const string& bin_dir, string* home_dir) {
   string name_upper;
@@ -110,10 +114,12 @@ Status MiniHms::Start() {
   RETURN_NOT_OK(FindHomeDir("hive", bin_dir, &hive_home));
   RETURN_NOT_OK(FindHomeDir("java", bin_dir, &java_home));
 
-  auto tmp_dir = GetTestDataDirectory();
+  if (data_root_.empty()) {
+    data_root_ = GetTestDataDirectory();
+  }
 
-  RETURN_NOT_OK(CreateHiveSite(tmp_dir));
-  RETURN_NOT_OK(CreateCoreSite(tmp_dir));
+  RETURN_NOT_OK(CreateHiveSite());
+  RETURN_NOT_OK(CreateCoreSite());
 
   // Comma-separated list of additional jars to add to the HMS classpath.
   string aux_jars = 
Substitute("$0/hms-plugin.jar,$1/hcatalog/share/hcatalog/*",
@@ -143,9 +149,9 @@ Status MiniHms::Start() {
       { "JAVA_HOME", java_home },
       { "HADOOP_HOME", hadoop_home },
       { "HIVE_AUX_JARS_PATH", aux_jars },
-      { "HIVE_CONF_DIR", tmp_dir },
+      { "HIVE_CONF_DIR", data_root_ },
       { "JAVA_TOOL_OPTIONS", java_options },
-      { "HADOOP_CONF_DIR", tmp_dir },
+      { "HADOOP_CONF_DIR", data_root_ },
   };
 
   // Start the HMS.
@@ -198,7 +204,7 @@ string MiniHms::uris() const {
   return Substitute("thrift://127.0.0.1:$0", port_);
 }
 
-Status MiniHms::CreateHiveSite(const string& tmp_dir) const {
+Status MiniHms::CreateHiveSite() const {
 
   // - datanucleus.schema.autoCreateAll
   // - hive.metastore.schema.verification
@@ -279,7 +285,7 @@ Status MiniHms::CreateHiveSite(const string& tmp_dir) const 
{
 
   string file_contents = strings::Substitute(kFileTemplate,
                                              notification_log_ttl_.ToSeconds(),
-                                             tmp_dir,
+                                             data_root_,
                                              !keytab_file_.empty(),
                                              keytab_file_,
                                              service_principal_,
@@ -287,10 +293,10 @@ Status MiniHms::CreateHiveSite(const string& tmp_dir) 
const {
 
   return WriteStringToFile(Env::Default(),
                            file_contents,
-                           JoinPathSegments(tmp_dir, "hive-site.xml"));
+                           JoinPathSegments(data_root_, "hive-site.xml"));
 }
 
-Status MiniHms::CreateCoreSite(const string& tmp_dir) const {
+Status MiniHms::CreateCoreSite() const {
 
   // - hadoop.security.authentication
   //     The HMS uses Hadoop's UGI contraption which will refuse to login a 
user
@@ -313,7 +319,7 @@ Status MiniHms::CreateCoreSite(const string& tmp_dir) const 
{
 
   return WriteStringToFile(Env::Default(),
                            file_contents,
-                           JoinPathSegments(tmp_dir, "core-site.xml"));
+                           JoinPathSegments(data_root_, "core-site.xml"));
 }
 
 } // namespace hms

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/hms/mini_hms.h
----------------------------------------------------------------------
diff --git a/src/kudu/hms/mini_hms.h b/src/kudu/hms/mini_hms.h
index a45d9de..6528e0b 100644
--- a/src/kudu/hms/mini_hms.h
+++ b/src/kudu/hms/mini_hms.h
@@ -49,6 +49,10 @@ class MiniHms {
                       std::string keytab_file,
                       rpc::SaslProtection::Type protection);
 
+  // Configures the mini HMS to store its data in the provided path. If not 
set,
+  // it uses a test-only temporary directory.
+  void SetDataRoot(std::string data_root);
+
   // Starts the mini Hive metastore.
   //
   // If the MiniHms has already been started and stopped, it will be restarted
@@ -77,10 +81,10 @@ class MiniHms {
  private:
 
   // Creates a hive-site.xml for the mini HMS.
-  Status CreateHiveSite(const std::string& tmp_dir) const WARN_UNUSED_RESULT;
+  Status CreateHiveSite() const WARN_UNUSED_RESULT;
 
   // Creates a core-site.xml for the mini HMS.
-  Status CreateCoreSite(const std::string& tmp_dir) const WARN_UNUSED_RESULT;
+  Status CreateCoreSite() const WARN_UNUSED_RESULT;
 
   // Waits for the metastore process to bind to a port.
   Status WaitForHmsPorts() WARN_UNUSED_RESULT;
@@ -89,6 +93,8 @@ class MiniHms {
   MonoDelta notification_log_ttl_ = MonoDelta::FromSeconds(86400);
   uint16_t port_ = 0;
 
+  std::string data_root_;
+
   // Kerberos configuration
   std::string krb5_conf_;
   std::string service_principal_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/master/master.proto
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index 6fba23b..6ad72cf 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -659,6 +659,33 @@ message ConnectToMasterResponsePB {
   // an HA setup. If that master then fails, the applications would go
   // offline.
   repeated HostPortPB master_addrs = 5;
+
+  // If the master is configured with the Hive Metastore integration enabled,
+  // this field will include the configuration options.
+  optional HiveMetastoreConfig hms_config = 6;
+}
+
+// Hive Metastore integration options and configuration.
+message HiveMetastoreConfig {
+
+  // Address(es) of the Hive Metastore instance(s).
+  //
+  // For more info see the Kudu master --hive_metastore_uris flag for more 
info,
+  // or the Hive Metastore hive.metastore.uris configuration.
+  optional string hms_uris = 1;
+
+  // Whether the Hive Metastore instance uses SASL (Kerberos) security.
+  //
+  // For more info see the Kudu master --hive_metastore_sasl_enabled flag, or
+  // the Hive Metastore hive.metastore.sasl.enabled configuration.
+  optional bool hms_sasl_enabled = 2;
+
+  // An ID which uniquely identifies the Hive Metastore instance.
+  //
+  // NOTE: this is provided on a best-effort basis, as not all Hive Metastore
+  // versions which Kudu is compatible with include the necessary APIs. See
+  // HIVE-16452 for more info.
+  optional string hms_uuid = 3;
 }
 
 // ============================================================================

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc 
b/src/kudu/master/master_service.cc
index b71f470..1df802e 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -33,6 +33,7 @@
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/consensus/replica_management.pb.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/hms/hms_catalog.h"
 #include "kudu/master/catalog_manager.h"
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
@@ -52,7 +53,9 @@
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 
+DECLARE_bool(hive_metastore_sasl_enabled);
 DECLARE_bool(raft_prepare_replacement_before_eviction);
+DECLARE_string(hive_metastore_uris);
 
 DEFINE_int32(master_inject_latency_on_tablet_lookups_ms, 0,
              "Number of milliseconds that the master will sleep before 
responding to "
@@ -509,6 +512,14 @@ void MasterServiceImpl::ConnectToMaster(const 
ConnectToMasterRequestPB* /*req*/,
     }
   }
 
+  // Add Hive Metastore information.
+  if (hms::HmsCatalog::IsEnabled()) {
+    auto* metastore_config = resp->mutable_hms_config();
+    metastore_config->set_hms_uris(FLAGS_hive_metastore_uris);
+    metastore_config->set_hms_sasl_enabled(FLAGS_hive_metastore_sasl_enabled);
+    // TODO(dan): set the hms_uuid field.
+  }
+
   // Rather than consulting the current consensus role, instead base it
   // on the catalog manager's view. This prevents us from advertising LEADER
   // until we have taken over all the associated responsibilities.

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/mini-cluster/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc 
b/src/kudu/mini-cluster/external_mini_cluster.cc
index 3d72375..6061f83 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -190,6 +190,7 @@ Status ExternalMiniCluster::Start() {
   if (opts_.hms_mode == HmsMode::ENABLE_HIVE_METASTORE ||
       opts_.hms_mode == HmsMode::ENABLE_METASTORE_INTEGRATION) {
     hms_.reset(new hms::MiniHms());
+    hms_->SetDataRoot(opts_.cluster_root);
 
     if (opts_.enable_kerberos) {
       string spn = "hive/127.0.0.1";

http://git-wip-us.apache.org/repos/asf/kudu/blob/2c572cbd/src/kudu/tools/tool_action_hms.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_hms.cc 
b/src/kudu/tools/tool_action_hms.cc
index e709c82..65ae2e0 100644
--- a/src/kudu/tools/tool_action_hms.cc
+++ b/src/kudu/tools/tool_action_hms.cc
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <algorithm>
 #include <iostream>
 #include <map>
 #include <memory>
@@ -40,8 +39,6 @@
 #include "kudu/hms/hive_metastore_types.h"
 #include "kudu/hms/hms_catalog.h"
 #include "kudu/hms/hms_client.h"
-#include "kudu/master/master.h"
-#include "kudu/server/server_base.pb.h"
 #include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/util/monotime.h"
@@ -68,6 +65,11 @@ DEFINE_bool(upgrade_hms_tables, true,
     "new Kudu metadata format.");
 
 namespace kudu {
+
+namespace server {
+class GetFlagsResponsePB_Flag;
+} // namespace server
+
 namespace tools {
 
 using client::KuduClient;
@@ -119,30 +121,16 @@ Status Init(const RunnerContext& context,
   *master_addrs = GetMasterAddresses(**kudu_client);
 
   if (FLAGS_hive_metastore_uris.empty()) {
-    // Lookup the HMS URIs and SASL config from the master configuration.
-    vector<GetFlagsResponsePB_Flag> flags;
-    RETURN_NOT_OK(GetServerFlags(vector<string>(Split(*master_addrs, ","))[0],
-                                 master::Master::kDefaultPort, false, {}, 
&flags));
-
-    auto hms_uris = std::find_if(flags.begin(), flags.end(),
-        [] (const GetFlagsResponsePB_Flag& flag) {
-          return flag.name() == "hive_metastore_uris";
-        });
-    if (hms_uris == flags.end()) {
+    string hive_metastore_uris = (*kudu_client)->GetHiveMetastoreUris();
+    if (hive_metastore_uris.empty()) {
       return Status::ConfigurationError(
-          "the Kudu master is not configured with the Hive Metastore 
integration", master_addrs[0]);
+          "the Kudu leader master is not configured with the Hive Metastore 
integration");
     }
-    auto hms_sasl_enabled = std::find_if(flags.begin(), flags.end(),
-        [] (const GetFlagsResponsePB_Flag& flag) {
-          return flag.name() == "hive_metastore_sasl_enabled";
-        });
+    bool hive_metastore_sasl_enabled = 
(*kudu_client)->GetHiveMetastoreSaslEnabled();
 
     // Override the flag values.
-    FLAGS_hive_metastore_uris = hms_uris->value();
-    // SetCommandLineOption avoids needing to parse the value into a bool.
-    CHECK(!gflags::SetCommandLineOption(
-        "hive_metastore_sasl_enabled",
-        hms_sasl_enabled == flags.end() ? "false" : 
hms_sasl_enabled->value().c_str()).empty());
+    FLAGS_hive_metastore_uris = hive_metastore_uris;
+    FLAGS_hive_metastore_sasl_enabled = hive_metastore_sasl_enabled;
   }
 
   // Create HMS catalog.

Reply via email to