You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/01/24 01:11:59 UTC

[kudu] 02/05: [sentry] add mini Sentry to the external mini cluster

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

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

commit 9098f442e3bc727ab7fc57508de6dd092fffd522
Author: Hao Hao <ha...@cloudera.com>
AuthorDate: Sun Nov 18 23:24:37 2018 -0800

    [sentry] add mini Sentry to the external mini cluster
    
    This commit enables external mini cluster to be able to start with both
    mini sentry and mini hms. A major challenge is the circular dependency
    between the HMS and Sentry in terms of configuring each with the other's
    address, including port which is currently discovered by polling lsof.
    To work around it, one approach would be to:
    
      1. Start the Sentry service. Find out which port it's on. At this
         stage, the Sentry service has no knowledge of any HMS service.
      2. Start the HMS, configured to talk to Sentry's port. Find out which
         port it's on.
      3. Restart the Sentry service on the same port, reconfigured to talk
         to the HMS's port.
    
    However, there could be a race condition where another program binds to
    the same port during the restart in step 3. One option is to use
    SO_REUSEPORT socket option, which permits multiple sockets to be bound
    to an identical socket address. Although both Sentry and HMS are written
    in Java, only JDK9 (and above) supports this socket option[1]. Methods
    such as Java reflection can be used to invoke private methods to set up
    this option, but they are hacky.
    
    This patch extends the UNIQUE_LOOPBACK usage in mini cluster, so that
    external servers such as Sentry can pick up a unique bind address,
    knowing that in doing so there'll never be any danger of a port
    collision.
    
    I looped master_sentry-itest 1000 times (which used UNIQUE_LOOPBACK
    mode) and all passed:
    http://dist-test.cloudera.org/job?job_id=hao.hao.1548189107.129474
    
    [1]: https://docs.oracle.com/javase/9/docs/api/java/net/StandardSocketOptions.html#SO_REUSEPORT
    
    Change-Id: I7f02e6085bd239570d10ec629f48856d37ed6e59
    Reviewed-on: http://gerrit.cloudera.org:8080/11956
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/hms/mini_hms.cc                          | 148 +++++++++++++++++++---
 src/kudu/hms/mini_hms.h                           |  27 ++--
 src/kudu/integration-tests/CMakeLists.txt         |   1 +
 src/kudu/integration-tests/master_sentry-itest.cc | 110 ++++++++++++++++
 src/kudu/mini-cluster/CMakeLists.txt              |   1 +
 src/kudu/mini-cluster/external_mini_cluster.cc    |  87 ++++++++++++-
 src/kudu/mini-cluster/external_mini_cluster.h     |  20 +++
 src/kudu/mini-cluster/mini_cluster.cc             |  19 ++-
 src/kudu/mini-cluster/mini_cluster.h              |   3 +-
 src/kudu/security/test/mini_kdc.cc                |   8 +-
 src/kudu/sentry/mini_sentry.cc                    |  53 ++++++--
 src/kudu/sentry/mini_sentry.h                     |  34 ++++-
 src/kudu/sentry/sentry-test-base.h                |   7 +-
 src/kudu/util/test_util.cc                        |  76 +++++++----
 src/kudu/util/test_util.h                         |  17 ++-
 15 files changed, 533 insertions(+), 78 deletions(-)

diff --git a/src/kudu/hms/mini_hms.cc b/src/kudu/hms/mini_hms.cc
index 771cc30..8cbb6de 100644
--- a/src/kudu/hms/mini_hms.cc
+++ b/src/kudu/hms/mini_hms.cc
@@ -24,6 +24,7 @@
 #include <ostream>
 #include <string>
 
+#include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 
 #include "kudu/gutil/strings/substitute.h"
@@ -36,6 +37,7 @@
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_util.h"
 
+using boost::none;
 using kudu::rpc::SaslProtection;
 using std::map;
 using std::string;
@@ -54,11 +56,6 @@ MiniHms::~MiniHms() {
   WARN_NOT_OK(Stop(), "Failed to stop MiniHms");
 }
 
-void MiniHms::SetNotificationLogTtl(MonoDelta ttl) {
-  CHECK(hms_process_);
-  notification_log_ttl_ = ttl;
-}
-
 void MiniHms::EnableKerberos(string krb5_conf,
                              string service_principal,
                              string keytab_file,
@@ -73,7 +70,17 @@ void MiniHms::EnableKerberos(string krb5_conf,
   protection_ = protection;
 }
 
+void MiniHms::EnableSentry(const HostPort& sentry_address,
+                           string sentry_service_principal) {
+  CHECK(!hms_process_);
+  DCHECK(!sentry_service_principal.empty());
+  VLOG(1) << Substitute("Enabling Sentry, at $0, for HMS", sentry_address.ToString());
+  sentry_address_ = sentry_address.ToString();
+  sentry_service_principal_ = std::move(sentry_service_principal);
+}
+
 void MiniHms::SetDataRoot(string data_root) {
+  CHECK(!hms_process_);
   data_root_ = std::move(data_root);
 }
 
@@ -92,9 +99,11 @@ Status MiniHms::Start() {
   string hadoop_home;
   string hive_home;
   string java_home;
+  string sentry_home;
   RETURN_NOT_OK(FindHomeDir("hadoop", bin_dir, &hadoop_home));
   RETURN_NOT_OK(FindHomeDir("hive", bin_dir, &hive_home));
   RETURN_NOT_OK(FindHomeDir("java", bin_dir, &java_home));
+  RETURN_NOT_OK(FindHomeDir("sentry", bin_dir, &sentry_home));
 
   if (data_root_.empty()) {
     data_root_ = GetTestDataDirectory();
@@ -104,9 +113,10 @@ Status MiniHms::Start() {
   RETURN_NOT_OK(CreateCoreSite());
   RETURN_NOT_OK(CreateLogConfig());
 
-  // Comma-separated list of additional jars to add to the HMS classpath.
-  string aux_jars = Substitute("$0/hms-plugin.jar,$1/hcatalog/share/hcatalog/*",
-                               bin_dir, hive_home);
+  // Comma-separated list of additional jars to add to the HMS classpath, including
+  // the HMS plugins of Kudu and Sentry.
+  string aux_jars = Substitute("$0/hms-plugin.jar,$1/hcatalog/share/hcatalog/*,$2/lib/*",
+                               bin_dir, hive_home, sentry_home);
 
   // List of JVM environment options to pass to the HMS.
   string java_options =
@@ -144,9 +154,10 @@ Status MiniHms::Start() {
   hms_process_->SetEnvVars(env_vars);
   RETURN_NOT_OK(hms_process_->Start());
 
-  // Wait for HMS to start listening on its ports and commencing operation.
+  // Wait for HMS to start listening on its ports and commencing operation
+  // with a wildcard binding.
   VLOG(1) << "Waiting for HMS ports";
-  Status wait = WaitForTcpBind(hms_process_->pid(), &port_,
+  Status wait = WaitForTcpBind(hms_process_->pid(), &port_, /*addr=*/none,
                                MonoDelta::FromMilliseconds(kHmsStartTimeoutMs));
   if (!wait.ok()) {
     WARN_NOT_OK(hms_process_->Kill(SIGQUIT), "failed to send SIGQUIT to HMS");
@@ -183,6 +194,10 @@ string MiniHms::uris() const {
   return Substitute("thrift://127.0.0.1:$0", port_);
 }
 
+bool MiniHms::IsAuthorizationEnabled() const {
+  return !sentry_address_.empty() && IsKerberosEnabled();
+}
+
 Status MiniHms::CreateHiveSite() const {
 
   // - datanucleus.schema.autoCreateAll
@@ -206,9 +221,10 @@ Status MiniHms::CreateHiveSite() const {
   //     names.
   //
   // - hive.metastore.notifications.add.thrift.objects
-  //     Configured the HMS to add the entire thrift Table/Partition
+  //     Configures the HMS to add the entire thrift Table/Partition
   //     objects to the HMS notifications.
-  static const string kFileTemplate = R"(
+  //
+  static const string kHiveFileTemplate = R"(
 <configuration>
   <property>
     <name>hive.metastore.transactional.event.listeners</name>
@@ -277,19 +293,109 @@ Status MiniHms::CreateHiveSite() const {
     <name>hive.metastore.notifications.add.thrift.objects</name>
     <value>true</value>
   </property>
+
+  $6
+
 </configuration>
   )";
 
-  string file_contents = Substitute(kFileTemplate,
-                                    notification_log_ttl_.ToSeconds(),
-                                    data_root_,
-                                    !keytab_file_.empty(),
-                                    keytab_file_,
-                                    service_principal_,
-                                    SaslProtection::name_of(protection_));
+  string sentry_properties;
+  if (IsAuthorizationEnabled()) {
+
+    // - hive.sentry.conf.url
+    //     Configuration URL of the Sentry authorization plugin in the HMS.
+    //
+    // - hive.metastore.filter.hook
+    //     Configures the HMS to use the Sentry plugin for filtering
+    //     out information user has no privileges to access for operations
+    //     as SHOWTABLES and SHOWDATABASES.
+    //
+    // - hive.metastore.pre.event.listeners
+    //     Configures the HMS to use the Sentry event listener to
+    //     consult Sentry service for authorization metadata when servicing
+    //     requests.
+    //
+    // - hive.metastore.event.listeners
+    //     Configures the HMS to use the Sentry post-event listener, which
+    //     synchronizes the HMS events with the Sentry service. The Sentry
+    //     service will be made aware of events like table renames and
+    //     update itself accordingly.
+    static const string kHiveSentryFileTemplate = R"(
+<property>
+  <name>hive.sentry.conf.url</name>
+  <value>file://$0/hive-sentry-site.xml</value>
+</property>
+
+<property>
+  <name>hive.metastore.filter.hook</name>
+  <value>org.apache.sentry.binding.metastore.SentryMetaStoreFilterHook</value>
+</property>
+
+<property>
+  <name>hive.metastore.pre.event.listeners</name>
+  <value>org.apache.sentry.binding.metastore.MetastoreAuthzBinding</value>
+</property>
+
+<property>
+  <name>hive.metastore.event.listeners</name>
+  <value>org.apache.sentry.binding.metastore.SentrySyncHMSNotificationsPostEventListener</value>
+</property>
+    )";
+
+    sentry_properties = Substitute(kHiveSentryFileTemplate, data_root_);
+  }
+
+  string hive_file_contents = Substitute(kHiveFileTemplate,
+                                         notification_log_ttl_.ToSeconds(),
+                                         data_root_,
+                                         IsKerberosEnabled(),
+                                         keytab_file_,
+                                         service_principal_,
+                                         SaslProtection::name_of(protection_),
+                                         sentry_properties);
+
+  if (IsAuthorizationEnabled()) {
+    // - hive.sentry.server
+    //     Server namespace the HMS instance belongs to for defining
+    //     server-level privileges in Sentry.
+    //
+    // - sentry.metastore.service.users
+    //     Set of service users whose access will be excluded from
+    //     Sentry authorization checks.
+    static const string kSentryFileTemplate = R"(
+<configuration>
+  <property>
+    <name>sentry.service.client.server.rpc-addresses</name>
+    <value>$0</value>
+  </property>
+
+  <property>
+    <name>sentry.service.server.principal</name>
+    <value>$1</value>
+  </property>
+
+  <property>
+    <name>hive.sentry.server</name>
+    <value>$2</value>
+  </property>
+
+  <property>
+    <name>sentry.metastore.service.users</name>
+    <value>kudu</value>
+  </property>
+</configuration>
+  )";
+    string sentry_file_contents = Substitute(kSentryFileTemplate,
+                                             sentry_address_,
+                                             sentry_service_principal_,
+                                             "server1");
+    RETURN_NOT_OK(WriteStringToFile(Env::Default(),
+                                    sentry_file_contents,
+                                    JoinPathSegments(data_root_, "hive-sentry-site.xml")));
+  }
 
   return WriteStringToFile(Env::Default(),
-                           file_contents,
+                           hive_file_contents,
                            JoinPathSegments(data_root_, "hive-site.xml"));
 }
 
@@ -311,7 +417,7 @@ Status MiniHms::CreateCoreSite() const {
 </configuration>
   )";
 
-  string file_contents = Substitute(kFileTemplate, keytab_file_.empty() ? "simple" : "kerberos");
+  string file_contents = Substitute(kFileTemplate, IsKerberosEnabled() ? "kerberos" : "simple");
 
   return WriteStringToFile(Env::Default(),
                            file_contents,
diff --git a/src/kudu/hms/mini_hms.h b/src/kudu/hms/mini_hms.h
index b98fa4b..0adcb21 100644
--- a/src/kudu/hms/mini_hms.h
+++ b/src/kudu/hms/mini_hms.h
@@ -40,15 +40,17 @@ class MiniHms {
 
   ~MiniHms();
 
-  // Configures the notification log TTL. Must be called before Start().
-  void SetNotificationLogTtl(MonoDelta ttl);
-
   // Configures the mini HMS to use Kerberos.
   void EnableKerberos(std::string krb5_conf,
                       std::string service_principal,
                       std::string keytab_file,
                       rpc::SaslProtection::Type protection);
 
+  // Configures the mini HMS to enable the Sentry plugin, passing the
+  // Sentry service's principal to be used in Kerberos environment.
+  void EnableSentry(const HostPort& sentry_address,
+                    std::string sentry_service_principal);
+
   // 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);
@@ -74,10 +76,18 @@ class MiniHms {
     return HostPort("127.0.0.1", port_);
   }
 
-  /// Returns the Metastore URIs, in the format that the Hive
-  /// hive.metastore.uris configuration expects.
+  // Returns the Metastore URIs, in the format that the Hive
+  // hive.metastore.uris configuration expects.
   std::string uris() const;
 
+  // Returns true when Sentry as well as Kerberos is enabled.
+  bool IsAuthorizationEnabled() const;
+
+  // Returns true when Kerberos is enabled.
+  bool IsKerberosEnabled() const {
+    return !keytab_file_.empty();
+  }
+
  private:
 
   // Creates a hive-site.xml for the mini HMS.
@@ -89,9 +99,6 @@ class MiniHms {
   // Creates a log4j2 configuration properties file for the mini HMS.
   Status CreateLogConfig() const WARN_UNUSED_RESULT;
 
-  // Waits for the metastore process to bind to a port.
-  Status WaitForHmsPorts() WARN_UNUSED_RESULT;
-
   std::unique_ptr<Subprocess> hms_process_;
   MonoDelta notification_log_ttl_ = MonoDelta::FromSeconds(86400);
   uint16_t port_ = 0;
@@ -103,6 +110,10 @@ class MiniHms {
   std::string service_principal_;
   std::string keytab_file_;
   rpc::SaslProtection::Type protection_ = rpc::SaslProtection::kAuthentication;
+
+  // Sentry configuration
+  std::string sentry_address_;
+  std::string sentry_service_principal_;
 };
 
 } // namespace hms
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 429283b..2de1a9f 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -88,6 +88,7 @@ ADD_KUDU_TEST(master_migration-itest)
 ADD_KUDU_TEST_DEPENDENCIES(master_migration-itest
   kudu)
 ADD_KUDU_TEST(master_replication-itest)
+ADD_KUDU_TEST(master_sentry-itest RUN_SERIAL true PROCESSORS 4)
 ADD_KUDU_TEST(master-stress-test RUN_SERIAL true)
 ADD_KUDU_TEST(multidir_cluster-itest)
 ADD_KUDU_TEST(open-readonly-fs-itest PROCESSORS 4)
diff --git a/src/kudu/integration-tests/master_sentry-itest.cc b/src/kudu/integration-tests/master_sentry-itest.cc
new file mode 100644
index 0000000..ef2ad2e
--- /dev/null
+++ b/src/kudu/integration-tests/master_sentry-itest.cc
@@ -0,0 +1,110 @@
+// 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.
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <gtest/gtest.h>
+
+#include "kudu/client/client.h"
+#include "kudu/client/schema.h"
+#include "kudu/client/shared_ptr.h"
+#include "kudu/common/common.pb.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/hms/hive_metastore_types.h"
+#include "kudu/hms/hms_client.h"
+#include "kudu/hms/mini_hms.h"
+#include "kudu/integration-tests/external_mini_cluster-itest-base.h"
+#include "kudu/mini-cluster/external_mini_cluster.h"
+#include "kudu/security/test/mini_kdc.h"
+#include "kudu/thrift/client.h"
+#include "kudu/util/test_macros.h"
+
+namespace kudu {
+
+using client::KuduColumnSchema;
+using client::KuduSchema;
+using client::KuduSchemaBuilder;
+using client::KuduTableCreator;
+using cluster::ExternalMiniClusterOptions;
+using hms::HmsClient;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+// Test Master authorization enforcement with Sentry and HMS
+// integration enabled.
+class MasterSentryTest : public ExternalMiniClusterITestBase,
+                         public ::testing::WithParamInterface<bool> {
+ public:
+
+  void SetUp() override {
+    ExternalMiniClusterITestBase::SetUp();
+    ExternalMiniClusterOptions opts;
+    opts.hms_mode = HmsMode::ENABLE_METASTORE_INTEGRATION;
+    opts.enable_sentry = true;
+    opts.enable_kerberos = enable_kerberos_;
+    StartClusterWithOpts(std::move(opts));
+  }
+
+ protected:
+    const bool enable_kerberos_ = GetParam();
+};
+INSTANTIATE_TEST_CASE_P(KerberosEnabled, MasterSentryTest, ::testing::Bool());
+
+// TODO(hao): Write a proper test when master authorization enforcement is done.
+TEST_P(MasterSentryTest, TestFoo) {
+  const string kDatabaseName = "default";
+  const string kTableName = "foo";
+  const string kTableIdentifier = Substitute("$0.$1", kDatabaseName, kTableName);
+
+  KuduSchema schema;
+  KuduSchemaBuilder b;
+  b.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
+  b.AddColumn("val")->Type(KuduColumnSchema::INT32);
+  ASSERT_OK(b.Build(&schema));
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  ASSERT_OK(table_creator->table_name(kTableIdentifier)
+                           .schema(&schema)
+                           .num_replicas(1)
+                           .set_range_partition_columns({ "key" })
+                           .Create());
+
+  // Verify the table exists in Kudu and the HMS.
+
+  bool exists;
+  ASSERT_OK(client_->TableExists(kTableIdentifier, &exists));
+  ASSERT_TRUE(exists);
+
+  thrift::ClientOptions hms_client_opts;
+  if (enable_kerberos_) {
+    // Create a principal 'kudu' and configure to use it.
+    ASSERT_OK(cluster_->kdc()->CreateUserPrincipal("kudu"));
+    ASSERT_OK(cluster_->kdc()->Kinit("kudu"));
+    ASSERT_OK(cluster_->kdc()->SetKrb5Environment());
+    hms_client_opts.enable_kerberos = true;
+    hms_client_opts.service_principal = "hive";
+  }
+  hms::HmsClient hms_client(cluster_->hms()->address(), hms_client_opts);
+  ASSERT_OK(hms_client.Start());
+
+  hive::Table hms_table;
+  ASSERT_OK(hms_client.GetTable(kDatabaseName, kTableName, &hms_table));
+}
+
+} // namespace kudu
diff --git a/src/kudu/mini-cluster/CMakeLists.txt b/src/kudu/mini-cluster/CMakeLists.txt
index d5cc44a..17e37ae 100644
--- a/src/kudu/mini-cluster/CMakeLists.txt
+++ b/src/kudu/mini-cluster/CMakeLists.txt
@@ -36,6 +36,7 @@ target_link_libraries(mini_cluster
   master_proto
   mini_hms
   mini_kdc
+  mini_sentry
   server_base_proto
   tablet_proto
   tserver
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index a7e3d6b..e578997 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -48,6 +48,7 @@
 #include "kudu/rpc/sasl_common.h"
 #include "kudu/rpc/user_credentials.h"
 #include "kudu/security/test/mini_kdc.h"
+#include "kudu/sentry/mini_sentry.h"
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
 #include "kudu/tablet/metadata.pb.h"
@@ -108,6 +109,7 @@ ExternalMiniClusterOptions::ExternalMiniClusterOptions()
       num_data_dirs(1),
       enable_kerberos(false),
       hms_mode(HmsMode::NONE),
+      enable_sentry(false),
       logtostderr(true),
       start_process_timeout(MonoDelta::FromSeconds(30)),
       rpc_negotiation_timeout(MonoDelta::FromSeconds(3)) {
@@ -153,6 +155,30 @@ Status ExternalMiniCluster::HandleOptions() {
   return Status::OK();
 }
 
+Status ExternalMiniCluster::StartSentry() {
+  sentry_->SetDataRoot(opts_.cluster_root);
+
+  if (hms_) {
+    sentry_->EnableHms(hms_->uris());
+  }
+
+  if (opts_.enable_kerberos) {
+    string spn = Substitute("sentry/$0", sentry_->address().host());
+    string ktpath;
+    RETURN_NOT_OK_PREPEND(kdc_->CreateServiceKeytab(spn, &ktpath),
+                          "could not create keytab");
+    sentry_->EnableKerberos(kdc_->GetEnvVars()["KRB5_CONFIG"],
+                            Substitute("$0@KRBTEST.COM", spn),
+                            ktpath);
+  }
+
+  return sentry_->Start();
+}
+
+Status ExternalMiniCluster::StopSentry() {
+  return sentry_->Stop();
+}
+
 Status ExternalMiniCluster::Start() {
   CHECK(masters_.empty()) << "Masters are not empty (size: " << masters_.size()
       << "). Maybe you meant Restart()?";
@@ -193,13 +219,46 @@ Status ExternalMiniCluster::Start() {
                           "could not set krb5 client env");
   }
 
+  // Start the Sentry service and the HMS in the following steps, in order
+  // to deal with the circular dependency in terms of configuring each
+  // with the other's IP/port.
+  // 1. Pick a bind IP using UNIQUE_LOOPBACK mode for the Sentry service.
+  //    Statically choose a random port. Since the Sentry service will
+  //    live on its own IP address, there's no danger of collision.
+  // 2. Start the HMS, configured to talk to the Sentry service. Find out
+  //    which port it's on.
+  // 3. Start the Sentry service with the chosen address/port from step 1.
+  //
+  // We can also pick a random port for the HMS in step 2, however, due to
+  // HIVE-18998 (which is addressed in Hive 4.0.0 by HIVE-20794), this is not
+  // an option.
+  // TODO(hao): Pick a static port for the HMS to bind to when we move to Hive 4.
+  //
+  // Note that when UNIQUE_LOOPBACK mode is not supported (i.e. on macOS),
+  // we cannot choose a port at random as that can cause a port collision.
+  // In that case, we start the Sentry service with the picked IP address
+  // and port 0 in step 1. Find out which port it's on by polling lsof.
+  // In step 3, restart the Sentry service and reconfigure it to talk to
+  // the HMS's port.
+
+  if (opts_.enable_sentry) {
+    sentry_.reset(new sentry::MiniSentry());
+    string host = GetBindIpForExternalServer(0);
+    uint16_t port = opts_.bind_mode == UNIQUE_LOOPBACK ? 10000 : 0;
+    sentry_->SetAddress(HostPort(host, port));
+    if (opts_.bind_mode != UNIQUE_LOOPBACK) {
+      RETURN_NOT_OK_PREPEND(StartSentry(), "Failed to start the Sentry service");
+    }
+  }
+
+  // Start the HMS.
   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";
+      string spn = Substitute("hive/$0", hms_->address().host());
       string ktpath;
       RETURN_NOT_OK_PREPEND(kdc_->CreateServiceKeytab(spn, &ktpath),
                             "could not create keytab");
@@ -207,8 +266,23 @@ Status ExternalMiniCluster::Start() {
                            rpc::SaslProtection::kAuthentication);
     }
 
+    if (opts_.enable_sentry) {
+      string sentry_spn = Substitute("sentry/$0@KRBTEST.COM", sentry_->address().host());
+      hms_->EnableSentry(sentry_->address(), sentry_spn);
+    }
+
     RETURN_NOT_OK_PREPEND(hms_->Start(),
                           "Failed to start the Hive Metastore");
+
+    // (Re)start Sentry with the HMS address.
+    if (opts_.enable_sentry) {
+      if (opts_.bind_mode != UNIQUE_LOOPBACK) {
+        RETURN_NOT_OK_PREPEND(StopSentry(),
+                              "Failed to stop the Sentry service");
+      }
+      RETURN_NOT_OK_PREPEND(StartSentry(),
+                            "Failed to start the Sentry service");
+    }
   }
 
   RETURN_NOT_OK_PREPEND(StartMasters(), "failed to start masters");
@@ -425,6 +499,13 @@ Status ExternalMiniCluster::StartMasters() {
         opts.extra_flags.emplace_back("--hive_metastore_sasl_enabled=true");
       }
     }
+    if (opts_.enable_sentry) {
+      opts.extra_flags.emplace_back(Substitute("--sentry_service_rpc_addresses=$0",
+                                               sentry_->address().ToString()));
+      if (!opts_.enable_kerberos) {
+        opts.extra_flags.emplace_back("--sentry_service_security_mode=none");
+      }
+    }
     opts.logtostderr = opts_.logtostderr;
 
     scoped_refptr<ExternalMaster> peer = new ExternalMaster(opts);
@@ -448,6 +529,10 @@ string ExternalMiniCluster::GetBindIpForMaster(int index) const {
   return MiniCluster::GetBindIpForDaemon(MiniCluster::MASTER, index, opts_.bind_mode);
 }
 
+string ExternalMiniCluster::GetBindIpForExternalServer(int index) const {
+  return MiniCluster::GetBindIpForDaemon(MiniCluster::EXTERNAL_SERVER, index, opts_.bind_mode);
+}
+
 Status ExternalMiniCluster::AddTabletServer() {
   CHECK(leader_master() != nullptr)
       << "Must have started at least 1 master before adding tablet servers";
diff --git a/src/kudu/mini-cluster/external_mini_cluster.h b/src/kudu/mini-cluster/external_mini_cluster.h
index 8addfff..ea218af 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.h
+++ b/src/kudu/mini-cluster/external_mini_cluster.h
@@ -66,6 +66,10 @@ namespace rpc {
 class Messenger;
 } // namespace rpc
 
+namespace sentry {
+class MiniSentry;
+} // namespace sentry
+
 namespace server {
 class ServerStatusPB;
 } // namespace server
@@ -151,6 +155,11 @@ struct ExternalMiniClusterOptions {
   // Default: HmsMode::NONE.
   HmsMode hms_mode;
 
+  // If true, set up a Sentry service as part of this ExternalMiniCluster.
+  //
+  // Default: false.
+  bool enable_sentry;
+
   // If true, sends logging output to stderr instead of a log file.
   //
   // Default: true.
@@ -214,6 +223,9 @@ class ExternalMiniCluster : public MiniCluster {
   // Same as above but for a master.
   std::string GetBindIpForMaster(int index) const;
 
+  // Same as above but for a external server, e.g. Sentry service or Hive Metastore.
+  std::string GetBindIpForExternalServer(int index) const;
+
   // Return a pointer to the running leader master. This may be NULL
   // if the cluster is not started.
   //
@@ -270,6 +282,10 @@ class ExternalMiniCluster : public MiniCluster {
     return hms_.get();
   }
 
+  sentry::MiniSentry* sentry() const {
+    return sentry_.get();
+  }
+
   const std::string& cluster_root() const {
     return opts_.cluster_root;
   }
@@ -385,6 +401,9 @@ class ExternalMiniCluster : public MiniCluster {
 
   Status StartMasters();
 
+  Status StartSentry();
+  Status StopSentry();
+
   Status DeduceBinRoot(std::string* ret);
   Status HandleOptions();
 
@@ -394,6 +413,7 @@ class ExternalMiniCluster : public MiniCluster {
   std::vector<scoped_refptr<ExternalTabletServer>> tablet_servers_;
   std::unique_ptr<MiniKdc> kdc_;
   std::unique_ptr<hms::MiniHms> hms_;
+  std::unique_ptr<sentry::MiniSentry> sentry_;
 
   std::shared_ptr<rpc::Messenger> messenger_;
 
diff --git a/src/kudu/mini-cluster/mini_cluster.cc b/src/kudu/mini-cluster/mini_cluster.cc
index f67b432..2bcac30 100644
--- a/src/kudu/mini-cluster/mini_cluster.cc
+++ b/src/kudu/mini-cluster/mini_cluster.cc
@@ -49,7 +49,24 @@ string MiniCluster::GetBindIpForDaemon(DaemonType type, int index, BindMode bind
       uint32_t pid = getpid();
       CHECK_LT(pid, 1 << kPidBits) << Substitute(
           "PID $0 is more than $1 bits wide", pid, kPidBits);
-      int idx = (type == TSERVER) ? index + 1 : kServersMaxNum - index;
+      int idx;
+      // Partition the index space 'kServersMaxNum' into three portions, one for each
+      // daemon type, to get unique address. If a daemon index spans over the portion
+      // reserved for another type, then duplicate address can be generated. Though this
+      // should be enough for our current tests.
+      switch (type) {
+        case MASTER:
+          idx = index + 1;
+          break;
+        case TSERVER:
+          idx = kServersMaxNum - index;
+          break;
+        case EXTERNAL_SERVER:
+          idx = kServersMaxNum / 3 + index;
+          break;
+        default:
+          LOG(FATAL) << type;
+      }
       uint32_t ip = (pid << kServerIdxBits) | static_cast<uint32_t>(idx);
       uint8_t octets[] = {
           static_cast<uint8_t>((ip >> 16) & 0xff),
diff --git a/src/kudu/mini-cluster/mini_cluster.h b/src/kudu/mini-cluster/mini_cluster.h
index 627a683..2fe83e7 100644
--- a/src/kudu/mini-cluster/mini_cluster.h
+++ b/src/kudu/mini-cluster/mini_cluster.h
@@ -103,7 +103,8 @@ class MiniCluster {
 
   enum DaemonType {
     MASTER,
-    TSERVER
+    TSERVER,
+    EXTERNAL_SERVER
   };
 
   static constexpr const char* const kWildcardIpAddr = "0.0.0.0";
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index f4151e6..f61b3ef 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -25,6 +25,7 @@
 #include <string>
 #include <utility>
 
+#include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 
 #include "kudu/gutil/strings/strip.h"
@@ -37,6 +38,7 @@
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_util.h"
 
+using boost::none;
 using std::map;
 using std::string;
 using std::unique_ptr;
@@ -150,8 +152,10 @@ Status MiniKdc::Start() {
   RETURN_NOT_OK(kdc_process_->Start());
 
   const bool need_config_update = (options_.port == 0);
-  // Wait for KDC to start listening on its ports and commencing operation.
-  RETURN_NOT_OK(WaitForUdpBind(kdc_process_->pid(), &options_.port, MonoDelta::FromSeconds(1)));
+  // Wait for KDC to start listening on its ports and commencing operation
+  // with a wildcard binding.
+  RETURN_NOT_OK(WaitForUdpBind(kdc_process_->pid(), &options_.port,
+                               /*addr=*/none, MonoDelta::FromSeconds(1)));
 
   if (need_config_update) {
     // If we asked for an ephemeral port, grab the actual ports and
diff --git a/src/kudu/sentry/mini_sentry.cc b/src/kudu/sentry/mini_sentry.cc
index 5a7ad7d..2380d4f 100644
--- a/src/kudu/sentry/mini_sentry.cc
+++ b/src/kudu/sentry/mini_sentry.cc
@@ -65,6 +65,22 @@ void MiniSentry::EnableKerberos(std::string krb5_conf,
   keytab_file_ = std::move(keytab_file);
 }
 
+void MiniSentry::EnableHms(string hms_uris) {
+  CHECK(!sentry_process_);
+  hms_uris_ = std::move(hms_uris);
+}
+
+void MiniSentry::SetDataRoot(string data_root) {
+  CHECK(!sentry_process_);
+  data_root_ = std::move(data_root);
+}
+
+void MiniSentry::SetAddress(const HostPort& address) {
+  CHECK(!sentry_process_);
+  ip_ = address.host();
+  port_ = address.port();
+}
+
 Status MiniSentry::Start() {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, kSentryStartTimeoutMs / 2, "Starting Sentry");
   CHECK(!sentry_process_);
@@ -84,15 +100,23 @@ Status MiniSentry::Start() {
   RETURN_NOT_OK(FindHomeDir("sentry", bin_dir, &sentry_home));
   RETURN_NOT_OK(FindHomeDir("java", bin_dir, &java_home));
 
-  auto tmp_dir = GetTestDataDirectory();
+  if (data_root_.empty()) {
+    data_root_ = GetTestDataDirectory();
+  }
 
-  RETURN_NOT_OK(CreateSentryConfigs(tmp_dir));
+  RETURN_NOT_OK(CreateSentryConfigs(data_root_));
 
   // List of JVM environment options to pass to the Sentry service.
   string java_options;
   if (!krb5_conf_.empty()) {
     java_options += Substitute(" -Djava.security.krb5.conf=$0", krb5_conf_);
   }
+  if (IsHmsEnabled()) {
+    java_options += Substitute(" -Dhive.metastore.uris=$0"
+        " -Dhive.metastore.sasl.enabled=$1"
+        " -Dhive.metastore.kerberos.principal=hive/127.0.0.1@KRBTEST.COM",
+        hms_uris_, IsKerberosEnabled());
+  }
 
   map<string, string> env_vars {
       { "JAVA_HOME", java_home },
@@ -103,9 +127,9 @@ Status MiniSentry::Start() {
   // Start Sentry.
   sentry_process_.reset(new Subprocess({
       Substitute("$0/bin/sentry", sentry_home),
-      "--log4jConf", JoinPathSegments(tmp_dir, "log4j.properties"),
+      "--log4jConf", JoinPathSegments(data_root_, "log4j.properties"),
       "--command", "service",
-      "--conffile", JoinPathSegments(tmp_dir, "sentry-site.xml"),
+      "--conffile", JoinPathSegments(data_root_, "sentry-site.xml"),
   }));
 
   sentry_process_->SetEnvVars(env_vars);
@@ -113,8 +137,13 @@ Status MiniSentry::Start() {
 
   // Wait for Sentry to start listening on its ports and commencing operation.
   VLOG(1) << "Waiting for Sentry ports";
-  Status wait = WaitForTcpBind(sentry_process_->pid(), &port_,
+
+  uint16_t orig_port = port_;
+  Status wait = WaitForTcpBind(sentry_process_->pid(), &port_, ip_,
                                MonoDelta::FromMilliseconds(kSentryStartTimeoutMs));
+  // Check that the port number only changed if the original port was 0
+  // (i.e. if we asked to bind to an ephemeral port)
+  CHECK(orig_port == 0 || port_ == orig_port);
   if (!wait.ok()) {
     WARN_NOT_OK(sentry_process_->Kill(SIGQUIT), "failed to send SIGQUIT to Sentry");
   }
@@ -173,6 +202,8 @@ Status MiniSentry::CreateSentryConfigs(const string& tmp_dir) const {
   // - sentry.service.server.rpc-port
   //     Port number that the Sentry service starts with.
   //
+  // - sentry.service.server.rpc-address
+  //     IP address that the Sentry service starts with.
   static const string kFileTemplate = R"(
 <configuration>
 
@@ -227,13 +258,18 @@ Status MiniSentry::CreateSentryConfigs(const string& tmp_dir) const {
   </property>
 
   <property>
+    <name>sentry.service.server.rpc-address</name>
+    <value>$6</value>
+  </property>
+
+  <property>
     <name>sentry.service.admin.group</name>
     <value>admin</value>
   </property>
 
   <property>
     <name>sentry.service.allow.connect</name>
-    <value>kudu</value>
+    <value>kudu,hive</value>
   </property>
 
 </configuration>
@@ -242,12 +278,13 @@ Status MiniSentry::CreateSentryConfigs(const string& tmp_dir) const {
   string users_ini_path = JoinPathSegments(tmp_dir, "users.ini");
   string file_contents = Substitute(
       kFileTemplate,
-      keytab_file_.empty() ? "none" : "kerberos",
+      IsKerberosEnabled() ? "kerberos" : "none",
       service_principal_,
       keytab_file_,
       tmp_dir,
       users_ini_path,
-      port_);
+      port_,
+      ip_);
   RETURN_NOT_OK(WriteStringToFile(Env::Default(),
                                   file_contents,
                                   JoinPathSegments(tmp_dir, "sentry-site.xml")));
diff --git a/src/kudu/sentry/mini_sentry.h b/src/kudu/sentry/mini_sentry.h
index 169c9e6..36635c7 100644
--- a/src/kudu/sentry/mini_sentry.h
+++ b/src/kudu/sentry/mini_sentry.h
@@ -43,6 +43,17 @@ class MiniSentry {
                       std::string service_principal,
                       std::string keytab_file);
 
+  // Configures the mini Sentry service to connect to a Hive Metastore instance.
+  void EnableHms(std::string hms_uris);
+
+  // Configures the mini Sentry service to store its data in the provided path.
+  // If not set, it uses a test-only temporary directory.
+  void SetDataRoot(std::string data_root);
+
+  // Configures the mini Sentry service to start with the provided address.
+  // If not set, it uses the default Ip and port number.
+  void SetAddress(const HostPort& address);
+
   // Starts the mini Sentry service.
   //
   // If the MiniSentry has already been started and stopped, it will be restarted
@@ -58,10 +69,20 @@ class MiniSentry {
   // Unpause the Sentry service.
   Status Resume() WARN_UNUSED_RESULT;
 
-  // Returns the address of the Sentry service. Should only be called after the
+  // Returns the address of the mini Sentry service. Should only be called after the
   // Sentry service is started.
   HostPort address() const {
-    return HostPort("127.0.0.1", port_);
+    return HostPort(ip_, port_);
+  }
+
+  // Returns true when Kerberos is enabled.
+  bool IsKerberosEnabled() const {
+    return !keytab_file_.empty();
+  }
+
+  // Returns true when the HMS is enabled.
+  bool IsHmsEnabled() const {
+    return !hms_uris_.empty();
   }
 
  private:
@@ -70,18 +91,21 @@ class MiniSentry {
   // configuration files.
   Status CreateSentryConfigs(const std::string& tmp_dir) const WARN_UNUSED_RESULT;
 
-  // Waits for the metastore process to bind to a port.
-  Status WaitForSentryPorts() WARN_UNUSED_RESULT;
-
   std::unique_ptr<Subprocess> sentry_process_;
 
   // Port number of the mini Sentry service. Default to 0.
   uint16_t port_ = 0;
+  // Ip address of the mini Sentry service. Default to 0.0.0.0.
+  std::string ip_ = "0.0.0.0";
+  std::string data_root_;
 
   // Kerberos configuration
   std::string krb5_conf_;
   std::string service_principal_;
   std::string keytab_file_;
+
+  // HMS configuration
+  std::string hms_uris_;
 };
 
 } // namespace sentry
diff --git a/src/kudu/sentry/sentry-test-base.h b/src/kudu/sentry/sentry-test-base.h
index a67f74f..7d50b45 100644
--- a/src/kudu/sentry/sentry-test-base.h
+++ b/src/kudu/sentry/sentry-test-base.h
@@ -19,6 +19,7 @@
 
 #include <string>
 
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/sasl_common.h"
 #include "kudu/security/test/mini_kdc.h"
 #include "kudu/sentry/mini_sentry.h"
@@ -43,12 +44,12 @@ class SentryTestBase : public KuduTest,
       ASSERT_OK(kdc_->Start());
 
       // Create a service principal for the Sentry, and configure it to use it.
-      std::string spn = "sentry/127.0.0.1@KRBTEST.COM";
+      std::string spn = strings::Substitute("sentry/$0", sentry_->address().host());
       std::string ktpath;
-      ASSERT_OK(kdc_->CreateServiceKeytab("sentry/127.0.0.1", &ktpath));
+      ASSERT_OK(kdc_->CreateServiceKeytab(spn, &ktpath));
 
       sentry_->EnableKerberos(kdc_->GetEnvVars()["KRB5_CONFIG"],
-                              spn,
+                              strings::Substitute("$0@KRBTEST.COM", spn),
                               ktpath);
 
       ASSERT_OK(rpc::SaslInit());
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index 214964d..c514d0e 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -36,6 +36,7 @@
 #include <sys/param.h> // for MAXPATHLEN
 #endif
 
+#include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <gtest/gtest-spi.h>
@@ -43,6 +44,7 @@
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/strcat.h"
+#include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
@@ -63,6 +65,7 @@ DEFINE_string(test_leave_files, "on_failure",
 
 DEFINE_int32(test_random_seed, 0, "Random seed to use for randomized tests");
 
+using boost::optional;
 using std::string;
 using std::vector;
 using strings::Substitute;
@@ -379,7 +382,9 @@ int CountOpenFds(Env* env, const string& path_pattern) {
 }
 
 namespace {
-Status WaitForBind(pid_t pid, uint16_t* port, const char* kind, MonoDelta timeout) {
+Status WaitForBind(pid_t pid, uint16_t* port,
+                   const optional<const string&>& addr,
+                   const char* kind, MonoDelta timeout) {
   // In general, processes do not expose the port they bind to, and
   // reimplementing lsof involves parsing a lot of files in /proc/. So,
   // requiring lsof for tests and parsing its output seems more
@@ -395,15 +400,52 @@ Status WaitForBind(pid_t pid, uint16_t* port, const char* kind, MonoDelta timeou
     "-a", "-i", kind
   };
 
+  // The '-Ffn' flag gets lsof to output something like:
+  //   p5801
+  //   f548
+  //   n127.0.0.1:43954->127.0.0.1:43617
+  //   f549
+  //   n*:8038
+  //
+  // The first line is the pid. We ignore it.
+  // Subsequent lines come in pairs. In each pair, the first half of the pair
+  // is file descriptor number, we ignore it.
+  // The second half has the bind address and port.
+  //
+  // In this example, the first pair is an outbound TCP socket. We ignore it.
+  // The second pair is the listening TCP socket bind address and port.
+  //
+  // We use the first encountered listening TCP socket, since that's most likely
+  // to be the primary service port. When searching, we use the provided bind
+  // address if there is any, otherwise we use '*' (same as '0.0.0.0') which
+  // matches all addresses on the local machine.
+  string addr_pattern = Substitute("n$0:", (!addr || *addr == "0.0.0.0") ? "*" : *addr);
   MonoTime deadline = MonoTime::Now() + timeout;
   string lsof_out;
+  int32_t p = -1;
 
   for (int64_t i = 1; ; i++) {
     lsof_out.clear();
-    Status s = Subprocess::Call(cmd, "", &lsof_out);
+    Status s = Subprocess::Call(cmd, "", &lsof_out).AndThen([&] () {
+      StripTrailingNewline(&lsof_out);
+      vector<string> lines = strings::Split(lsof_out, "\n");
+      for (int index = 2; index < lines.size(); index += 2) {
+        StringPiece cur_line(lines[index]);
+        if (HasPrefixString(cur_line.ToString(), addr_pattern) &&
+            !cur_line.contains("->")) {
+          cur_line.remove_prefix(addr_pattern.size());
+          if (!safe_strto32(cur_line.data(), cur_line.size(), &p)) {
+            return Status::RuntimeError("unexpected lsof output", lsof_out);
+          }
+
+          return Status::OK();
+        }
+      }
+
+      return Status::RuntimeError("unexpected lsof output", lsof_out);
+    });
 
     if (s.ok()) {
-      StripTrailingNewline(&lsof_out);
       break;
     }
     if (deadline < MonoTime::Now()) {
@@ -413,22 +455,6 @@ Status WaitForBind(pid_t pid, uint16_t* port, const char* kind, MonoDelta timeou
     SleepFor(MonoDelta::FromMilliseconds(i * 10));
   }
 
-  // The '-Ffn' flag gets lsof to output something like:
-  //   p19730
-  //   f123
-  //   n*:41254
-  // The first line is the pid. We ignore it.
-  // The second line is the file descriptor number. We ignore it.
-  // The third line has the bind address and port.
-  // Subsequent lines show active connections.
-  vector<string> lines = strings::Split(lsof_out, "\n");
-  int32_t p = -1;
-  if (lines.size() < 3 ||
-      lines[2].substr(0, 3) != "n*:" ||
-      !safe_strto32(lines[2].substr(3), &p) ||
-      p <= 0) {
-    return Status::RuntimeError("unexpected lsof output", lsof_out);
-  }
   CHECK(p > 0 && p < std::numeric_limits<uint16_t>::max()) << "parsed invalid port: " << p;
   VLOG(1) << "Determined bound port: " << p;
   *port = p;
@@ -436,12 +462,16 @@ Status WaitForBind(pid_t pid, uint16_t* port, const char* kind, MonoDelta timeou
 }
 } // anonymous namespace
 
-Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
-  return WaitForBind(pid, port, "4TCP", timeout);
+Status WaitForTcpBind(pid_t pid, uint16_t* port,
+                      const optional<const string&>& addr,
+                      MonoDelta timeout) {
+  return WaitForBind(pid, port, addr, "4TCP", timeout);
 }
 
-Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
-  return WaitForBind(pid, port, "4UDP", timeout);
+Status WaitForUdpBind(pid_t pid, uint16_t* port,
+                      const optional<const string&>& addr,
+                      MonoDelta timeout) {
+  return WaitForBind(pid, port, addr, "4UDP", timeout);
 }
 
 Status FindHomeDir(const string& name, const string& bin_dir, string* home_dir) {
diff --git a/src/kudu/util/test_util.h b/src/kudu/util/test_util.h
index 32dfacd..d74d7d4 100644
--- a/src/kudu/util/test_util.h
+++ b/src/kudu/util/test_util.h
@@ -26,6 +26,7 @@
 #include <memory>
 #include <string>
 
+#include <boost/optional/optional.hpp>
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/port.h"
@@ -136,11 +137,17 @@ void AssertEventually(const std::function<void(void)>& f,
 // unlike the usual behavior of path globs.
 int CountOpenFds(Env* env, const std::string& path_pattern);
 
-// Waits for the subprocess to bind to any listening TCP port, and returns the port.
-Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
-
-// Waits for the subprocess to bind to any listening UDP port, and returns the port.
-Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
+// Waits for the subprocess to bind to any listening TCP port on the provided
+// IP address (if the address is not provided, it is a wildcard binding), and
+// returns the port.
+Status WaitForTcpBind(pid_t pid, uint16_t* port,
+                      const boost::optional<const std::string&>& addr,
+                      MonoDelta timeout) WARN_UNUSED_RESULT;
+
+// Similar to above but binds to any listening UDP port.
+Status WaitForUdpBind(pid_t pid, uint16_t* port,
+                      const boost::optional<const std::string&>& addr,
+                      MonoDelta timeout) WARN_UNUSED_RESULT;
 
 // Find the home directory of a Java-style application, e.g. JAVA_HOME or
 // HADOOP_HOME.