You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/06 23:57:30 UTC

[kudu] 01/03: hms: allow for tooling to run without Kudu plugin

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

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

commit a76177f615ab1936ff37679e551986526ba062bd
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Mon Jun 3 21:36:59 2019 -0400

    hms: allow for tooling to run without Kudu plugin
    
    Currently, whenever an HMS client successfully connects to the HMS, it
    checks the various HMS service configurations (e.g. that it is using the
    Kudu-HMS plugin), returning an error if any are misconfigured. This is
    important to make it much more obvious when the Kudu Master's HMS
    synchronization is misconfigured.
    
    It is still useful for other HMS clients (e.g. that used by the HMS
    tooling) to operate on an HMS instance that is not configured with the
    Kudu-HMS plugin et al.
    
    This patch removes the requirement by plumbing the option to the HMS
    client as a member of ThriftOptions. This was the most straightforward
    way to plumb this option from the HmsCatalog to the HmsClient, given the
    templating layered in between them for HA. Besides, we can use this
    option in the future if we ever want to verify the configuration of
    Thrift-based clients for other services (e.g. Sentry).
    
    This patch additionally allows the -hive_metastore_sasl_enabled flag to
    be used without the -keytab_file flag if not running kudu-master. To get
    this behavior, I've moved the gflag validator into master_main.cc, which
    is not built by tooling. I manually tested that it works, i.e. that
    tooling will not validate and that a master will.
    
    To test, I added an HmsMode that starts the HMS without the Kudu-HMS
    plugin installed and used it in a couple of HMS tooling tests. I
    considered reusing the ENABLE_HIVE_METASTORE HmsMode, but opted not to
    since some tests are greatly simplified by ENABLE_HIVE_METASTORE having
    the Kudu-HMS plugin installed (e.g. restarting the HMS isn't required to
    enable the Kudu-HMS integration).
    
    Change-Id: I9b9968bf0f8a55859a14421beda05cab3496b6c0
    Reviewed-on: http://gerrit.cloudera.org:8080/13510
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/common/common.proto                   | 14 +++++++-
 src/kudu/hms/hms_catalog.cc                    |  3 +-
 src/kudu/hms/hms_catalog.h                     |  7 +++-
 src/kudu/hms/hms_client-test.cc                |  4 +++
 src/kudu/hms/hms_client.cc                     |  7 +++-
 src/kudu/hms/hms_client.h                      |  1 +
 src/kudu/hms/mini_hms.cc                       | 27 ++++++++------
 src/kudu/hms/mini_hms.h                        |  6 ++++
 src/kudu/master/catalog_manager.cc             |  3 +-
 src/kudu/master/master.cc                      | 22 ------------
 src/kudu/master/master_main.cc                 | 25 +++++++++++++
 src/kudu/mini-cluster/external_mini_cluster.cc |  7 +++-
 src/kudu/server/server_base.cc                 |  1 -
 src/kudu/thrift/client.h                       |  4 +++
 src/kudu/tools/kudu-tool-test.cc               | 49 ++++++++++++++------------
 15 files changed, 118 insertions(+), 62 deletions(-)

diff --git a/src/kudu/common/common.proto b/src/kudu/common/common.proto
index a51e806..0e2e77e 100644
--- a/src/kudu/common/common.proto
+++ b/src/kudu/common/common.proto
@@ -70,9 +70,21 @@ enum EncodingType {
   BIT_SHUFFLE = 6;
 }
 
+// Enums that specify the HMS-related configurations for a Kudu mini-cluster.
 enum HmsMode {
+  // No HMS will be started.
   NONE = 0;
+
+  // The HMS will be started, but will not be configured to use the Kudu
+  // plugin.
+  DISABLE_HIVE_METASTORE = 3;
+
+  // The HMS will be started and configured to use the Kudu plugin, but the
+  // Kudu mini-cluster will not be configured to synchronize with it.
   ENABLE_HIVE_METASTORE = 1;
+
+  // The HMS will be started and configured to use the Kudu plugin, and the
+  // Kudu mini-cluster will be configured to synchronize with it.
   ENABLE_METASTORE_INTEGRATION = 2;
 };
 
@@ -425,4 +437,4 @@ message KeyRangePB {
   optional bytes stop_primary_key = 2 [(kudu.REDACT) = true];
   // Number of bytes in chunk.
   required uint64 size_bytes_estimates = 3;
-}
\ No newline at end of file
+}
diff --git a/src/kudu/hms/hms_catalog.cc b/src/kudu/hms/hms_catalog.cc
index 7c2219a..6f03f5c 100644
--- a/src/kudu/hms/hms_catalog.cc
+++ b/src/kudu/hms/hms_catalog.cc
@@ -110,7 +110,7 @@ HmsCatalog::~HmsCatalog() {
   Stop();
 }
 
-Status HmsCatalog::Start() {
+Status HmsCatalog::Start(HmsClientVerifyKuduSyncConfig verify_service_config) {
   vector<HostPort> addresses;
   RETURN_NOT_OK(ParseUris(FLAGS_hive_metastore_uris, &addresses));
 
@@ -122,6 +122,7 @@ Status HmsCatalog::Start() {
   options.service_principal = FLAGS_hive_metastore_kerberos_principal;
   options.max_buf_size = FLAGS_hive_metastore_max_message_size_bytes;
   options.retry_count = FLAGS_hive_metastore_retry_count;
+  options.verify_service_config = verify_service_config == VERIFY;
 
   return ha_client_.Start(std::move(addresses), std::move(options));
 }
diff --git a/src/kudu/hms/hms_catalog.h b/src/kudu/hms/hms_catalog.h
index f4e92fd..e878f4c 100644
--- a/src/kudu/hms/hms_catalog.h
+++ b/src/kudu/hms/hms_catalog.h
@@ -37,6 +37,11 @@ class Schema;
 
 namespace hms {
 
+enum HmsClientVerifyKuduSyncConfig {
+  DONT_VERIFY,
+  VERIFY,
+};
+
 // A high-level API above the HMS which handles converting to and from
 // Kudu-specific types, retries, reconnections, HA, error handling, and
 // concurrent requests.
@@ -49,7 +54,7 @@ class HmsCatalog {
   ~HmsCatalog();
 
   // Starts the HmsCatalog instance.
-  Status Start();
+  Status Start(HmsClientVerifyKuduSyncConfig verify_service_config = DONT_VERIFY);
 
   // Stops the HmsCatalog instance.
   void Stop();
diff --git a/src/kudu/hms/hms_client-test.cc b/src/kudu/hms/hms_client-test.cc
index ca64c6b..6fbc121 100644
--- a/src/kudu/hms/hms_client-test.cc
+++ b/src/kudu/hms/hms_client-test.cc
@@ -397,6 +397,10 @@ TEST_F(HmsClientTest, TestHmsConnect) {
   options.send_timeout = MonoDelta::FromMilliseconds(100);
   options.conn_timeout = MonoDelta::FromMilliseconds(100);
 
+  // This test will attempt to connect and transfer data upon starting the
+  // client.
+  options.verify_service_config = true;
+
   auto start_client = [&options] (Sockaddr addr) -> Status {
     HmsClient client(HostPort(addr), options);
     return client.Start();
diff --git a/src/kudu/hms/hms_client.cc b/src/kudu/hms/hms_client.cc
index 61d87db..13e2237 100644
--- a/src/kudu/hms/hms_client.cc
+++ b/src/kudu/hms/hms_client.cc
@@ -126,7 +126,8 @@ const int kSlowExecutionWarningThresholdMs = 1000;
 const char* const HmsClient::kServiceName = "Hive Metastore";
 
 HmsClient::HmsClient(const HostPort& address, const ClientOptions& options)
-      : client_(hive::ThriftHiveMetastoreClient(CreateClientProtocol(address, options))) {
+      : verify_kudu_sync_config_(options.verify_service_config),
+        client_(hive::ThriftHiveMetastoreClient(CreateClientProtocol(address, options))) {
 }
 
 HmsClient::~HmsClient() {
@@ -138,6 +139,10 @@ Status HmsClient::Start() {
   HMS_RET_NOT_OK(client_.getOutputProtocol()->getTransport()->open(),
                  "failed to open Hive Metastore connection");
 
+  if (!verify_kudu_sync_config_) {
+    return Status::OK();
+  }
+
   // Immediately after connecting to the HMS, check that it is configured with
   // the required event listeners.
   string event_listener_config;
diff --git a/src/kudu/hms/hms_client.h b/src/kudu/hms/hms_client.h
index 67dcbed..c79f401 100644
--- a/src/kudu/hms/hms_client.h
+++ b/src/kudu/hms/hms_client.h
@@ -190,6 +190,7 @@ class HmsClient {
   static Status DeserializeJsonTable(Slice json, hive::Table* table) WARN_UNUSED_RESULT;
 
  private:
+  bool verify_kudu_sync_config_;
   hive::ThriftHiveMetastoreClient client_;
 };
 } // namespace hms
diff --git a/src/kudu/hms/mini_hms.cc b/src/kudu/hms/mini_hms.cc
index 7ccb4a9..1400f60 100644
--- a/src/kudu/hms/mini_hms.cc
+++ b/src/kudu/hms/mini_hms.cc
@@ -83,6 +83,10 @@ void MiniHms::EnableSentry(const HostPort& sentry_address,
   sentry_client_rpc_retry_interval_ms_ = sentry_client_rpc_retry_interval_ms;
 }
 
+void MiniHms::EnableKuduPlugin(bool enable) {
+  enable_kudu_plugin_ = enable;
+}
+
 void MiniHms::SetDataRoot(string data_root) {
   CHECK(!hms_process_);
   data_root_ = std::move(data_root);
@@ -208,6 +212,9 @@ bool MiniHms::IsAuthorizationEnabled() const {
 
 Status MiniHms::CreateHiveSite() const {
 
+  const string listeners = Substitute("org.apache.hive.hcatalog.listener.DbNotificationListener$0",
+      enable_kudu_plugin_ ? ",org.apache.kudu.hive.metastore.KuduMetastorePlugin" : "");
+
   // - datanucleus.schema.autoCreateAll
   // - hive.metastore.schema.verification
   //     Allow Hive to startup and run without first running the schemaTool.
@@ -237,8 +244,7 @@ Status MiniHms::CreateHiveSite() const {
   <property>
     <name>hive.metastore.transactional.event.listeners</name>
     <value>
-      org.apache.hive.hcatalog.listener.DbNotificationListener,
-      org.apache.kudu.hive.metastore.KuduMetastorePlugin
+      $0
     </value>
   </property>
 
@@ -254,37 +260,37 @@ Status MiniHms::CreateHiveSite() const {
 
   <property>
     <name>hive.metastore.warehouse.dir</name>
-    <value>file://$1/warehouse/</value>
+    <value>file://$2/warehouse/</value>
   </property>
 
   <property>
     <name>javax.jdo.option.ConnectionURL</name>
-    <value>jdbc:derby:$1/metadb;create=true</value>
+    <value>jdbc:derby:$2/metadb;create=true</value>
   </property>
 
   <property>
     <name>hive.metastore.event.db.listener.timetolive</name>
-    <value>$0s</value>
+    <value>$1s</value>
   </property>
 
   <property>
     <name>hive.metastore.sasl.enabled</name>
-    <value>$2</value>
+    <value>$3</value>
   </property>
 
   <property>
     <name>hive.metastore.kerberos.keytab.file</name>
-    <value>$3</value>
+    <value>$4</value>
   </property>
 
   <property>
     <name>hive.metastore.kerberos.principal</name>
-    <value>$4</value>
+    <value>$5</value>
   </property>
 
   <property>
     <name>hadoop.rpc.protection</name>
-    <value>$5</value>
+    <value>$6</value>
   </property>
 
   <property>
@@ -302,7 +308,7 @@ Status MiniHms::CreateHiveSite() const {
     <value>true</value>
   </property>
 
-  $6
+  $7
 
 </configuration>
   )";
@@ -354,6 +360,7 @@ Status MiniHms::CreateHiveSite() const {
   }
 
   string hive_file_contents = Substitute(kHiveFileTemplate,
+                                         listeners,
                                          notification_log_ttl_.ToSeconds(),
                                          data_root_,
                                          IsKerberosEnabled(),
diff --git a/src/kudu/hms/mini_hms.h b/src/kudu/hms/mini_hms.h
index 7fb1c22..73ac2df 100644
--- a/src/kudu/hms/mini_hms.h
+++ b/src/kudu/hms/mini_hms.h
@@ -59,6 +59,9 @@ class MiniHms {
                     int sentry_client_rpc_retry_num = 3,
                     int sentry_client_rpc_retry_interval_ms = 500);
 
+  // Configures the mini HMS to enable or disable the Kudu plugin.
+  void EnableKuduPlugin(bool enable);
+
   // 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);
@@ -124,6 +127,9 @@ class MiniHms {
   std::string sentry_service_principal_;
   int sentry_client_rpc_retry_num_;
   int sentry_client_rpc_retry_interval_ms_;
+
+  // Whether to enable the Kudu listener plugin.
+  bool enable_kudu_plugin_ = true;
 };
 
 } // namespace hms
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index b933ad5..eeb351d 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -280,6 +280,7 @@ using kudu::consensus::RaftConsensus;
 using kudu::consensus::RaftPeerPB;
 using kudu::consensus::StartTabletCopyRequestPB;
 using kudu::consensus::kMinimumTerm;
+using kudu::hms::HmsClientVerifyKuduSyncConfig;
 using kudu::pb_util::SecureDebugString;
 using kudu::pb_util::SecureShortDebugString;
 using kudu::rpc::RpcContext;
@@ -752,7 +753,7 @@ Status CatalogManager::Init(bool is_first_run) {
     std::lock_guard<RWMutex> leader_lock_guard(leader_lock_);
 
     hms_catalog_.reset(new hms::HmsCatalog(std::move(master_addresses)));
-    RETURN_NOT_OK_PREPEND(hms_catalog_->Start(),
+    RETURN_NOT_OK_PREPEND(hms_catalog_->Start(HmsClientVerifyKuduSyncConfig::VERIFY),
                           "failed to start Hive Metastore catalog");
 
     hms_notification_log_listener_.reset(new HmsNotificationLogListenerTask(this));
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index 5d90381..4fb8b8f 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -25,7 +25,6 @@
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/cfile/block_cache.h"
@@ -56,7 +55,6 @@
 #include "kudu/tserver/tablet_copy_service.h"
 #include "kudu/tserver/tablet_service.h"
 #include "kudu/util/flag_tags.h"
-#include "kudu/util/flag_validators.h"
 #include "kudu/util/maintenance_manager.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
@@ -95,8 +93,6 @@ DEFINE_string(location_mapping_cmd, "",
               "characters from the set [a-zA-Z0-9_-.]. If the cluster is not "
               "using location awareness features this flag should not be set.");
 
-DECLARE_bool(hive_metastore_sasl_enabled);
-DECLARE_string(keytab_file);
 
 using std::min;
 using std::shared_ptr;
@@ -113,24 +109,6 @@ using strings::Substitute;
 namespace kudu {
 namespace master {
 
-namespace {
-// Validates that if the HMS is configured with SASL enabled, the server has a
-// keytab available. This is located in master.cc because the HMS module (where
-// --hive_metastore_sasl_enabled is defined) doesn't link to the server module
-// (where --keytab_file is defined), and vice-versa. The master module is the
-// first module which links to both.
-bool ValidateHiveMetastoreSaslEnabled() {
-  if (FLAGS_hive_metastore_sasl_enabled && FLAGS_keytab_file.empty()) {
-    LOG(ERROR) << "When the Hive Metastore has SASL enabled "
-                  "(--hive_metastore_sasl_enabled), Kudu must be configured with "
-                  "a keytab (--keytab_file).";
-    return false;
-  }
-  return true;
-}
-GROUP_FLAG_VALIDATOR(hive_metastore_sasl_enabled, ValidateHiveMetastoreSaslEnabled);
-} // anonymous namespace
-
 Master::Master(const MasterOptions& opts)
   : KuduServer("Master", opts, "kudu.master"),
     state_(kStopped),
diff --git a/src/kudu/master/master_main.cc b/src/kudu/master/master_main.cc
index 583a185..d2f784a 100644
--- a/src/kudu/master/master_main.cc
+++ b/src/kudu/master/master_main.cc
@@ -25,6 +25,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.h"
 #include "kudu/master/master_options.h"
+#include "kudu/util/flag_validators.h"
 #include "kudu/util/flags.h"
 #include "kudu/util/init.h"
 #include "kudu/util/logging.h"
@@ -38,9 +39,33 @@ DECLARE_bool(evict_failed_followers);
 DECLARE_int32(webserver_port);
 DECLARE_string(rpc_bind_addresses);
 
+DECLARE_bool(hive_metastore_sasl_enabled);
+DECLARE_string(keytab_file);
+
 namespace kudu {
 namespace master {
 
+namespace {
+// Validates that if the HMS is configured with SASL enabled, the server has a
+// keytab available. This is located in master.cc because the HMS module (where
+// -hive_metastore_sasl_enabled is defined) doesn't link to the server module
+// (where --keytab_file is defined), and vice-versa. The master module is the
+// first module which links to both.
+// Note: this check only needs to be run on a server. E.g. tools that run with
+// the HMS don't need to pass in a keytab.
+bool ValidateHiveMetastoreSaslEnabled() {
+  if (FLAGS_hive_metastore_sasl_enabled &&
+      FLAGS_keytab_file.empty()) {
+    LOG(ERROR) << "When the Hive Metastore has SASL enabled "
+                  "(--hive_metastore_sasl_enabled), Kudu must be configured with "
+                  "a keytab (--keytab_file).";
+    return false;
+  }
+  return true;
+}
+GROUP_FLAG_VALIDATOR(hive_metastore_sasl_enabled, ValidateHiveMetastoreSaslEnabled);
+} // anonymous namespace
+
 static int MasterMain(int argc, char** argv) {
   InitKuduOrDie();
 
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index b60fcb2..bffb048 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -252,11 +252,16 @@ Status ExternalMiniCluster::Start() {
   }
 
   // Start the HMS.
-  if (opts_.hms_mode == HmsMode::ENABLE_HIVE_METASTORE ||
+  if (opts_.hms_mode == HmsMode::DISABLE_HIVE_METASTORE ||
+      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_.hms_mode == HmsMode::DISABLE_HIVE_METASTORE) {
+      hms_->EnableKuduPlugin(false);
+    }
+
     if (opts_.enable_kerberos) {
       string spn = Substitute("hive/$0", hms_->address().host());
       string ktpath;
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 7a57f33..cdc3bc9 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -130,7 +130,6 @@ TAG_FLAG(principal, experimental);
 // See KUDU-1884.
 TAG_FLAG(principal, unsafe);
 
-
 DEFINE_string(keytab_file, "",
               "Path to the Kerberos Keytab file for this server. Specifying a "
               "keytab file will cause the server to kinit, and enable Kerberos "
diff --git a/src/kudu/thrift/client.h b/src/kudu/thrift/client.h
index d01f213..49ac621 100644
--- a/src/kudu/thrift/client.h
+++ b/src/kudu/thrift/client.h
@@ -76,6 +76,10 @@ struct ClientOptions {
   // Number of times an RPC is retried by the HA client after encountering
   // retriable failures, such as network failures.
   int32_t retry_count = 1;
+
+  // Whether the client should, upon connecting, verify the Thrift service
+  // configuration is correct.
+  bool verify_service_config = false;
 };
 
 std::shared_ptr<apache::thrift::protocol::TProtocol> CreateClientProtocol(
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 65829e9..3fe573e 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -3402,7 +3402,7 @@ TEST_P(ToolTestKerberosParameterized, TestHmsDowngrade) {
 TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   string kUsername = "alice";
   ExternalMiniClusterOptions opts;
-  opts.hms_mode = HmsMode::ENABLE_HIVE_METASTORE;
+  opts.hms_mode = HmsMode::DISABLE_HIVE_METASTORE;
   opts.enable_kerberos = EnableKerberos();
   NO_FATALS(StartExternalMiniCluster(std::move(opts)));
 
@@ -3410,6 +3410,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   thrift::ClientOptions hms_opts;
   hms_opts.enable_kerberos = EnableKerberos();
   hms_opts.service_principal = "hive";
+  hms_opts.verify_service_config = false;
   HmsClient hms_client(cluster_->hms()->address(), hms_opts);
   ASSERT_OK(hms_client.Start());
   ASSERT_TRUE(hms_client.IsConnected());
@@ -3521,11 +3522,6 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(hms_client.CreateDatabase(db));
   ASSERT_OK(CreateKuduTable(kudu_client, "my_db.table"));
 
-  // Enable the HMS integration.
-  cluster_->ShutdownNodes(cluster::ClusterNodes::MASTERS_ONLY);
-  cluster_->EnableMetastoreIntegration();
-  ASSERT_OK(cluster_->Restart());
-
   unordered_set<string> consistent_tables = {
     "default.control",
   };
@@ -3554,12 +3550,16 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
     consistent_tables.insert(tables.begin(), tables.end());
   };
 
+  const string hms_flags = Substitute("-hive_metastore_uris=$0 -hive_metastore_sasl_enabled=$1",
+                                      FLAGS_hive_metastore_uris, FLAGS_hive_metastore_sasl_enabled);
+
   // Run the HMS check tool and verify that the consistent tables are not
   // reported, and the inconsistent tables are reported.
   auto check = [&] () {
     string out;
     string err;
-    Status s = RunActionStdoutStderrString(Substitute("hms check $0", master_addr), &out, &err);
+    Status s = RunActionStdoutStderrString(Substitute("hms check $0 $1", master_addr, hms_flags),
+                                           &out, &err);
     SCOPED_TRACE(strings::CUnescapeOrDie(out));
     if (inconsistent_tables.empty()) {
       ASSERT_OK(s);
@@ -3581,13 +3581,14 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
 
   // 'hms fix --dryrun should not change the output of 'hms check'.
   NO_FATALS(RunActionStdoutNone(
-        Substitute("hms fix $0 --dryrun --drop_orphan_hms_tables", master_addr)));
+        Substitute("hms fix $0 --dryrun --drop_orphan_hms_tables $1", master_addr, hms_flags)));
   NO_FATALS(check());
 
   // Drop orphan hms tables.
   NO_FATALS(RunActionStdoutNone(
         Substitute("hms fix $0 --drop_orphan_hms_tables --nocreate_missing_hms_tables "
-                   "--noupgrade_hms_tables --nofix_inconsistent_tables", master_addr)));
+                   "--noupgrade_hms_tables --nofix_inconsistent_tables $1",
+                   master_addr, hms_flags)));
   make_consistent({
     "default.orphan_hms_table",
     "default.orphan_hms_table_legacy_managed",
@@ -3596,7 +3597,8 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
 
   // Create missing hms tables.
   NO_FATALS(RunActionStdoutNone(
-        Substitute("hms fix $0 --noupgrade_hms_tables --nofix_inconsistent_tables", master_addr)));
+        Substitute("hms fix $0 --noupgrade_hms_tables --nofix_inconsistent_tables $1",
+                   master_addr, hms_flags)));
   make_consistent({
     "default.kudu_orphan",
     "my_db.table",
@@ -3605,7 +3607,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
 
   // Upgrade legacy HMS tables.
   NO_FATALS(RunActionStdoutNone(
-        Substitute("hms fix $0 --nofix_inconsistent_tables", master_addr)));
+        Substitute("hms fix $0 --nofix_inconsistent_tables $1", master_addr, hms_flags)));
   make_consistent({
     "default.legacy_managed",
     "legacy_external",
@@ -3615,7 +3617,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   NO_FATALS(check());
 
   // Refresh stale HMS tables.
-  NO_FATALS(RunActionStdoutNone(Substitute("hms fix $0", master_addr)));
+  NO_FATALS(RunActionStdoutNone(Substitute("hms fix $0 $1", master_addr, hms_flags)));
   make_consistent({
     "default.UPPERCASE",
     "default.inconsistent_schema",
@@ -3679,7 +3681,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
 TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   string kUsername = "alice";
   ExternalMiniClusterOptions opts;
-  opts.hms_mode = HmsMode::ENABLE_HIVE_METASTORE;
+  opts.hms_mode = HmsMode::DISABLE_HIVE_METASTORE;
   opts.enable_kerberos = EnableKerberos();
   NO_FATALS(StartExternalMiniCluster(std::move(opts)));
 
@@ -3687,6 +3689,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   thrift::ClientOptions hms_opts;
   hms_opts.enable_kerberos = EnableKerberos();
   hms_opts.service_principal = "hive";
+  hms_opts.verify_service_config = false;
   HmsClient hms_client(cluster_->hms()->address(), hms_opts);
   ASSERT_OK(hms_client.Start());
   ASSERT_TRUE(hms_client.IsConnected());
@@ -3727,16 +3730,15 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "impala::default.conflicting_legacy_table"));
   ASSERT_OK(CreateKuduTable(kudu_client, "default.conflicting_legacy_table"));
 
-  // Enable the HMS integration.
-  cluster_->ShutdownNodes(cluster::ClusterNodes::MASTERS_ONLY);
-  cluster_->EnableMetastoreIntegration();
-  ASSERT_OK(cluster_->Restart());
+  const string hms_flags = Substitute("-hive_metastore_uris=$0 -hive_metastore_sasl_enabled=$1",
+                                      FLAGS_hive_metastore_uris, FLAGS_hive_metastore_sasl_enabled);
 
   // Run the HMS check tool and verify that the inconsistent tables are reported.
   auto check = [&] () {
     string out;
     string err;
-    Status s = RunActionStdoutStderrString(Substitute("hms check $0", master_addr), &out, &err);
+    Status s = RunActionStdoutStderrString(Substitute("hms check $0 $1", master_addr, hms_flags),
+                                           &out, &err);
     SCOPED_TRACE(strings::CUnescapeOrDie(out));
     for (const string& table : vector<string>({
       "duplicate_hms_tables",
@@ -3757,7 +3759,8 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   {
     string out;
     string err;
-    Status s = RunActionStdoutStderrString(Substitute("hms fix $0", master_addr), &out, &err);
+    Status s = RunActionStdoutStderrString(Substitute("hms fix $0 $1", master_addr, hms_flags),
+                                           &out, &err);
     SCOPED_TRACE(strings::CUnescapeOrDie(out));
     ASSERT_FALSE(s.ok());
   }
@@ -3783,14 +3786,14 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
 
   // Rename the conflicting table.
   NO_FATALS(RunActionStdoutNone(Substitute(
-          "table rename-table --nomodify-external-catalogs $0 "
-          "default.conflicting_legacy_table default.non_conflicting_legacy_table", master_addr)));
+            "table rename-table --nomodify-external-catalogs $0 "
+            "default.conflicting_legacy_table default.non_conflicting_legacy_table", master_addr)));
 
   // Run the automatic fixer to create missing HMS table entries.
-  NO_FATALS(RunActionStdoutNone(Substitute("hms fix $0", master_addr)));
+  NO_FATALS(RunActionStdoutNone(Substitute("hms fix $0 $1", master_addr, hms_flags)));
 
   // Check should now be clean.
-  NO_FATALS(RunActionStdoutNone(Substitute("hms check $0", master_addr)));
+  NO_FATALS(RunActionStdoutNone(Substitute("hms check $0 $1", master_addr, hms_flags)));
 
   // Ensure the tables are available.
   vector<string> kudu_tables;