You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/06/28 18:44:07 UTC

[2/5] kudu git commit: Use dynamic bind address for internal + external mini clusters

Use dynamic bind address for internal + external mini clusters

This technique helps avoid port conflicts. It assigns a unique loopback
IP, based on pid and per-cluster numbering, to every InternalMiniCluster
or ExternalMiniCluster component (including masters and tablet servers).

Changes:

1. Apply the "dynamic bind address" technique to internal mini clusters
   (previously it was only used for external mini clusters).
2. Apply it to masters (previously it was only used for tablet servers).

I will post a follow-up patch to remove some of the CMakeLists.txt
limitations related to port conflicts, although we may need to put some
back due to lack of macOS support for unique loopback addresses.

I had to refactor a few host-related APIs to do this in a relatively
non-hacky manner. All tests still pass (and should more often).

Additional changes:

* Move BindMode to MiniCluster base class.
* Fix a few brittle tests that started failing because of this change.

Change-Id: I35eff9fbf5ccf8822cfe061673bc36598dff56f0
Reviewed-on: http://gerrit.cloudera.org:8080/7274
Tested-by: Mike Percy <mp...@apache.org>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: e6758739a90adeb2e4d0c6cf76185cf90cc7d2b0
Parents: 42a0749
Author: Mike Percy <mp...@apache.org>
Authored: Tue Jun 20 00:28:39 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Jun 28 18:12:42 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc                  |   7 +-
 src/kudu/integration-tests/CMakeLists.txt       |   1 +
 .../external_mini_cluster-test.cc               |  12 +-
 .../integration-tests/external_mini_cluster.cc  | 114 +++++++++----------
 .../integration-tests/external_mini_cluster.h   |  67 ++++-------
 .../integration-tests/internal_mini_cluster.cc  |  45 +++++---
 .../integration-tests/internal_mini_cluster.h   |  25 +++-
 .../integration-tests/master_migration-itest.cc |   2 +-
 .../master_replication-itest.cc                 |   6 +-
 src/kudu/integration-tests/mini_cluster.cc      |  52 +++++++++
 src/kudu/integration-tests/mini_cluster.h       |  66 +++++++++++
 src/kudu/master/master-test.cc                  |   4 +-
 src/kudu/master/mini_master.cc                  |  50 ++++----
 src/kudu/master/mini_master.h                   |  26 ++---
 src/kudu/master/sys_catalog-test.cc             |   2 +-
 src/kudu/tools/kudu-tool-test.cc                |   2 +-
 src/kudu/tserver/mini_tablet_server.cc          |   9 +-
 src/kudu/tserver/mini_tablet_server.h           |   3 +-
 src/kudu/tserver/tablet_server-test-base.h      |   6 +-
 src/kudu/tserver/ts_tablet_manager-test.cc      |   9 +-
 20 files changed, 316 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 404d1e2..26baf56 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -2131,10 +2131,9 @@ TEST_F(ClientTest, TestWriteTimeout) {
     unique_ptr<KuduError> error = GetSingleErrorFromSession(session.get());
     const Status& status = error->status();
     ASSERT_TRUE(status.IsTimedOut()) << status.ToString();
-    ASSERT_STR_CONTAINS(status.ToString(),
-                        "Failed to write batch of 1 ops to tablet");
-    ASSERT_STR_CONTAINS(status.ToString(), "Write RPC to 127.0.0.1:");
-    ASSERT_STR_CONTAINS(status.ToString(), "after 1 attempt");
+    ASSERT_STR_MATCHES(status.ToString(),
+                       R"(Failed to write batch of 1 ops to tablet.*after 1 attempt.*)"
+                       R"(Write RPC to 127\.[0-9]{1,3}\.[0-9]{1,3}\.[0-9]{1,3}:.*timed out)");
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 018f80c..bbe56f1 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -28,6 +28,7 @@ set(INTEGRATION_TESTS_SRCS
   external_mini_cluster_fs_inspector.cc
   internal_mini_cluster.cc
   log_verifier.cc
+  mini_cluster.cc
   test_workload.cc
 )
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/external_mini_cluster-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster-test.cc b/src/kudu/integration-tests/external_mini_cluster-test.cc
index 79b2a7f..cd0c44a 100644
--- a/src/kudu/integration-tests/external_mini_cluster-test.cc
+++ b/src/kudu/integration-tests/external_mini_cluster-test.cc
@@ -120,10 +120,14 @@ TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
     SCOPED_TRACE(i);
     ExternalMaster* master = CHECK_NOTNULL(cluster.master(i));
     HostPort master_rpc = master->bound_rpc_hostport();
-    EXPECT_TRUE(HasPrefixString(master_rpc.ToString(), "127.0.0.1:")) << master_rpc.ToString();
+    string expected_prefix = Substitute("$0:", cluster.GetBindIpForMaster(i));
+    if (cluster.bind_mode() == MiniCluster::UNIQUE_LOOPBACK) {
+      EXPECT_NE(expected_prefix, "127.0.0.1:") << "Should bind to unique per-server hosts";
+    }
+    EXPECT_TRUE(HasPrefixString(master_rpc.ToString(), expected_prefix)) << master_rpc.ToString();
 
     HostPort master_http = master->bound_http_hostport();
-    EXPECT_TRUE(HasPrefixString(master_http.ToString(), "127.0.0.1:")) << master_http.ToString();
+    EXPECT_TRUE(HasPrefixString(master_http.ToString(), expected_prefix)) << master_http.ToString();
 
     // Retrieve a thread metric, which should always be present on any master.
     int64_t value;
@@ -141,7 +145,9 @@ TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
     ExternalTabletServer* ts = CHECK_NOTNULL(cluster.tablet_server(i));
     HostPort ts_rpc = ts->bound_rpc_hostport();
     string expected_prefix = Substitute("$0:", cluster.GetBindIpForTabletServer(i));
-    EXPECT_NE(expected_prefix, "127.0.0.1") << "Should bind to unique per-server hosts";
+    if (cluster.bind_mode() == MiniCluster::UNIQUE_LOOPBACK) {
+      EXPECT_NE(expected_prefix, "127.0.0.1:") << "Should bind to unique per-server hosts";
+    }
     EXPECT_TRUE(HasPrefixString(ts_rpc.ToString(), expected_prefix)) << ts_rpc.ToString();
 
     HostPort ts_http = ts->bound_http_hostport();

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.cc b/src/kudu/integration-tests/external_mini_cluster.cc
index c0edca4..fa7bed5 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -78,23 +78,13 @@ namespace kudu {
 
 static const char* const kMasterBinaryName = "kudu-master";
 static const char* const kTabletServerBinaryName = "kudu-tserver";
-static const char* const kWildcardIpAddr = "0.0.0.0";
-static const char* const kLoopbackIpAddr = "127.0.0.1";
 static double kTabletServerRegistrationTimeoutSeconds = 15.0;
 static double kMasterCatalogManagerTimeoutSeconds = 60.0;
 
-#if defined(__APPLE__)
-static ExternalMiniClusterOptions::BindMode kBindMode =
-    ExternalMiniClusterOptions::BindMode::LOOPBACK;
-#else
-static ExternalMiniClusterOptions::BindMode kBindMode =
-    ExternalMiniClusterOptions::BindMode::UNIQUE_LOOPBACK;
-#endif
-
 ExternalMiniClusterOptions::ExternalMiniClusterOptions()
     : num_masters(1),
       num_tablet_servers(1),
-      bind_mode(kBindMode),
+      bind_mode(MiniCluster::kDefaultBindMode),
       num_data_dirs(1),
       enable_kerberos(false),
       logtostderr(true),
@@ -303,9 +293,14 @@ Status ExternalMiniCluster::StartSingleMaster() {
   }
   opts.extra_flags = SubstituteInFlags(opts_.extra_master_flags, 0);
   opts.start_process_timeout = opts_.start_process_timeout;
+
+  opts.rpc_bind_address = HostPort(GetBindIpForMaster(0), 0);
   scoped_refptr<ExternalMaster> master = new ExternalMaster(opts);
   if (opts_.enable_kerberos) {
-    RETURN_NOT_OK_PREPEND(master->EnableKerberos(kdc_.get(), Substitute("$0", kLoopbackIpAddr)),
+    // The bind host here is the hostname that will be used to generate the
+    // Kerberos principal, so it has to match the bind address for the master
+    // rpc endpoint.
+    RETURN_NOT_OK_PREPEND(master->EnableKerberos(kdc_.get(), opts.rpc_bind_address.host()),
                           "could not enable Kerberos");
   }
 
@@ -322,13 +317,10 @@ Status ExternalMiniCluster::StartDistributedMasters() {
         opts_.master_rpc_ports.size() << " ports specified in 'master_rpc_ports'";
   }
 
-  vector<string> peer_addrs;
-  for (int i = 0; i < num_masters; i++) {
-    string addr = Substitute("$0:$1", kLoopbackIpAddr, opts_.master_rpc_ports[i]);
-    peer_addrs.push_back(addr);
-  }
+  vector<HostPort> peer_hostports = master_rpc_addrs();
   vector<string> flags = opts_.extra_master_flags;
-  flags.push_back("--master_addresses=" + JoinStrings(peer_addrs, ","));
+  flags.push_back(Substitute("--master_addresses=$0",
+                             HostPort::ToCommaSeparatedString(peer_hostports)));
   string exe = GetBinaryPath(kMasterBinaryName);
 
   // Start the masters.
@@ -347,10 +339,11 @@ Status ExternalMiniCluster::StartDistributedMasters() {
     }
     opts.extra_flags = SubstituteInFlags(flags, i);
     opts.start_process_timeout = opts_.start_process_timeout;
+    opts.rpc_bind_address = peer_hostports[i];
 
-    scoped_refptr<ExternalMaster> peer = new ExternalMaster(opts, peer_addrs[i]);
+    scoped_refptr<ExternalMaster> peer = new ExternalMaster(opts);
     if (opts_.enable_kerberos) {
-      RETURN_NOT_OK_PREPEND(peer->EnableKerberos(kdc_.get(), Substitute("$0", kLoopbackIpAddr)),
+      RETURN_NOT_OK_PREPEND(peer->EnableKerberos(kdc_.get(), peer_hostports[i].host()),
                             "could not enable Kerberos");
     }
     RETURN_NOT_OK_PREPEND(peer->Start(),
@@ -362,17 +355,11 @@ Status ExternalMiniCluster::StartDistributedMasters() {
 }
 
 string ExternalMiniCluster::GetBindIpForTabletServer(int index) const {
-  string bind_ip;
-  if (opts_.bind_mode == ExternalMiniClusterOptions::UNIQUE_LOOPBACK) {
-    pid_t p = getpid();
-    CHECK_LE(p, MathLimits<uint16_t>::kMax) << "Cannot run on systems with >16-bit pid";
-    bind_ip = Substitute("127.$0.$1.$2", p >> 8, p & 0xff, index);
-  } else if (opts_.bind_mode == ExternalMiniClusterOptions::WILDCARD) {
-    bind_ip = Substitute("$0", kWildcardIpAddr);
-  } else {
-    bind_ip = Substitute("$0", kLoopbackIpAddr);
-  }
-  return bind_ip;
+  return MiniCluster::GetBindIpForDaemon(MiniCluster::TSERVER, index, opts_.bind_mode);
+}
+
+string ExternalMiniCluster::GetBindIpForMaster(int index) const {
+  return MiniCluster::GetBindIpForDaemon(MiniCluster::MASTER, index, opts_.bind_mode);
 }
 
 Status ExternalMiniCluster::AddTabletServer() {
@@ -400,9 +387,10 @@ Status ExternalMiniCluster::AddTabletServer() {
   }
   opts.extra_flags = SubstituteInFlags(opts_.extra_tserver_flags, idx);
   opts.start_process_timeout = opts_.start_process_timeout;
+  opts.rpc_bind_address = HostPort(bind_host, 0);
 
   scoped_refptr<ExternalTabletServer> ts =
-      new ExternalTabletServer(opts, bind_host, master_hostports);
+      new ExternalTabletServer(opts, master_hostports);
   if (opts_.enable_kerberos) {
     RETURN_NOT_OK_PREPEND(ts->EnableKerberos(kdc_.get(), bind_host),
                           "could not enable Kerberos");
@@ -587,6 +575,16 @@ vector<ExternalDaemon*> ExternalMiniCluster::daemons() const {
   return results;
 }
 
+vector<HostPort> ExternalMiniCluster::master_rpc_addrs() const {
+  vector<HostPort> master_rpc_addrs;
+  for (int i = 0; i < opts_.master_rpc_ports.size(); i++) {
+    master_rpc_addrs.emplace_back(
+        GetBindIpForDaemon(MiniCluster::MASTER, i, opts_.bind_mode),
+        opts_.master_rpc_ports[i]);
+  }
+  return master_rpc_addrs;
+}
+
 std::shared_ptr<rpc::Messenger> ExternalMiniCluster::messenger() const {
   return messenger_;
 }
@@ -650,8 +648,11 @@ ExternalDaemon::ExternalDaemon(ExternalDaemonOptions opts)
       perf_record_filename_(std::move(opts.perf_record_filename)),
       start_process_timeout_(opts.start_process_timeout),
       logtostderr_(opts.logtostderr),
+      rpc_bind_address_(std::move(opts.rpc_bind_address)),
       exe_(std::move(opts.exe)),
-      extra_flags_(std::move(opts.extra_flags)) {}
+      extra_flags_(std::move(opts.extra_flags)) {
+  CHECK(rpc_bind_address_.Initialized());
+}
 
 ExternalDaemon::~ExternalDaemon() {
 }
@@ -917,14 +918,13 @@ void ExternalDaemon::Shutdown() {
   // Before we kill the process, store the addresses. If we're told to
   // start again we'll reuse these. Store only the port if the
   // daemons were using wildcard address for binding.
-  const string& wildcard_ip = Substitute("$0", kWildcardIpAddr);
-  if (get_rpc_bind_address() != wildcard_ip) {
+  if (rpc_bind_address().host() != MiniCluster::kWildcardIpAddr) {
     bound_rpc_ = bound_rpc_hostport();
     bound_http_ = bound_http_hostport();
   } else {
-    bound_rpc_.set_host(wildcard_ip);
+    bound_rpc_.set_host(MiniCluster::kWildcardIpAddr);
     bound_rpc_.set_port(bound_rpc_hostport().port());
-    bound_http_.set_host(wildcard_ip);
+    bound_http_.set_host(MiniCluster::kWildcardIpAddr);
     bound_http_.set_port(bound_http_hostport().port());
   }
 
@@ -1112,13 +1112,6 @@ ScopedResumeExternalDaemon::~ScopedResumeExternalDaemon() {
 
 ExternalMaster::ExternalMaster(ExternalDaemonOptions opts)
     : ExternalDaemon(std::move(opts)) {
-  set_rpc_bind_address(Substitute("$0:0", kLoopbackIpAddr));
-}
-
-ExternalMaster::ExternalMaster(ExternalDaemonOptions opts,
-                               string rpc_bind_address)
-    : ExternalDaemon(std::move(opts)) {
-  set_rpc_bind_address(std::move(rpc_bind_address));
 }
 
 ExternalMaster::~ExternalMaster() {
@@ -1126,8 +1119,9 @@ ExternalMaster::~ExternalMaster() {
 
 Status ExternalMaster::Start() {
   vector<string> flags(GetCommonFlags());
+  flags.push_back(Substitute("--rpc_bind_addresses=$0", rpc_bind_address().ToString()));
+  flags.push_back(Substitute("--webserver_interface=$0", rpc_bind_address().host()));
   flags.emplace_back("--webserver_port=0");
-  flags.push_back("--rpc_bind_addresses=" + get_rpc_bind_address());
   return StartProcess(flags);
 }
 
@@ -1138,10 +1132,15 @@ Status ExternalMaster::Restart() {
   }
 
   vector<string> flags(GetCommonFlags());
+  flags.push_back(Substitute("--rpc_bind_addresses=$0", bound_rpc_.ToString()));
+
   if (bound_http_.Initialized()) {
+    flags.push_back(Substitute("--webserver_interface=$0", bound_http_.host()));
     flags.push_back(Substitute("--webserver_port=$0", bound_http_.port()));
+  } else {
+    flags.push_back(Substitute("--webserver_interface=$0", bound_rpc_.host()));
+    flags.emplace_back("--webserver_port=0");
   }
-  flags.push_back("--rpc_bind_addresses=" + bound_rpc_.ToString());
 
   return StartProcess(flags);
 }
@@ -1210,11 +1209,10 @@ vector<string> ExternalMaster::GetCommonFlags() const {
 //------------------------------------------------------------
 
 ExternalTabletServer::ExternalTabletServer(ExternalDaemonOptions opts,
-                                           string bind_host,
                                            vector<HostPort> master_addrs)
     : ExternalDaemon(std::move(opts)),
-      master_addrs_(HostPort::ToCommaSeparatedString(master_addrs)) {
-  set_rpc_bind_address(std::move(bind_host));
+      master_addrs_(std::move(master_addrs)) {
+  DCHECK(!master_addrs_.empty());
 }
 
 ExternalTabletServer::~ExternalTabletServer() {
@@ -1224,14 +1222,15 @@ Status ExternalTabletServer::Start() {
   vector<string> flags;
   flags.push_back("--fs_wal_dir=" + wal_dir_);
   flags.push_back("--fs_data_dirs=" + JoinStrings(data_dirs_, ","));
-  flags.push_back(Substitute("--rpc_bind_addresses=$0:0",
-                             get_rpc_bind_address()));
+  flags.push_back(Substitute("--rpc_bind_addresses=$0",
+                             rpc_bind_address().ToString()));
   flags.push_back(Substitute("--local_ip_for_outbound_sockets=$0",
-                             get_rpc_bind_address()));
+                             rpc_bind_address().host()));
   flags.push_back(Substitute("--webserver_interface=$0",
-                             get_rpc_bind_address()));
+                             rpc_bind_address().host()));
   flags.emplace_back("--webserver_port=0");
-  flags.push_back("--tserver_master_addrs=" + master_addrs_);
+  flags.push_back(Substitute("--tserver_master_addrs=$0",
+                             HostPort::ToCommaSeparatedString(master_addrs_)));
   RETURN_NOT_OK(StartProcess(flags));
   return Status::OK();
 }
@@ -1244,15 +1243,16 @@ Status ExternalTabletServer::Restart() {
   vector<string> flags;
   flags.push_back("--fs_wal_dir=" + wal_dir_);
   flags.push_back("--fs_data_dirs=" + JoinStrings(data_dirs_, ","));
-  flags.push_back("--rpc_bind_addresses=" + bound_rpc_.ToString());
+  flags.push_back(Substitute("--rpc_bind_addresses=$0", bound_rpc_.ToString()));
   flags.push_back(Substitute("--local_ip_for_outbound_sockets=$0",
-                             get_rpc_bind_address()));
+                             rpc_bind_address().host()));
   if (bound_http_.Initialized()) {
     flags.push_back(Substitute("--webserver_port=$0", bound_http_.port()));
     flags.push_back(Substitute("--webserver_interface=$0",
                                bound_http_.host()));
   }
-  flags.push_back("--tserver_master_addrs=" + master_addrs_);
+  flags.push_back(Substitute("--tserver_master_addrs=$0",
+                             HostPort::ToCommaSeparatedString(master_addrs_)));
   return StartProcess(flags);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/external_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.h b/src/kudu/integration-tests/external_mini_cluster.h
index 29b5edd..c8cbdbf 100644
--- a/src/kudu/integration-tests/external_mini_cluster.h
+++ b/src/kudu/integration-tests/external_mini_cluster.h
@@ -68,43 +68,7 @@ struct ExternalMiniClusterOptions {
   // Default: "", which auto-generates a unique path for this cluster.
   std::string data_root;
 
-  // BindMode lets you specify the socket binding mode for RPC and/or HTTP server.
-  // A) LOOPBACK binds each server to loopback ip address "127.0.0.1".
-  //
-  // B) WILDCARD specifies "0.0.0.0" as the ip to bind to, which means sockets
-  // can be bound to any interface on the local host.
-  // For example, if a host has two interfaces with addresses
-  // 192.168.0.10 and 192.168.0.11, the server process can accept connection
-  // requests addressed to 192.168.0.10 or 192.168.0.11.
-  //
-  // C) UNIQUE_LOOPBACK binds each tablet server to a different loopback address.
-  // This affects the server's RPC server, and also forces the server to
-  // only use this IP address for outgoing socket connections as well.
-  // This allows the use of iptables on the localhost to simulate network
-  // partitions.
-  //
-  // The addresses used are 127.<A>.<B>.<C> where:
-  // - <A,B> are the high and low bytes of the pid of the process running the
-  //   minicluster (not the daemon itself).
-  // - <C> is the index of the server within this minicluster.
-  //
-  // This requires that the system is set up such that processes may bind
-  // to any IP address in the localhost netblock (127.0.0.0/8). This seems
-  // to be the case on common Linux distributions. You can verify by running
-  // 'ip addr | grep 127.0.0.1' and checking that the address is listed as
-  // '127.0.0.1/8'.
-  //
-  // This option is disabled by default on OS X.
-  //
-  // NOTE: this does not currently affect the HTTP server.
-  //
-  // Default: UNIQUE_LOOPBACK on Linux, LOOPBACK on macOS.
-  enum BindMode {
-    UNIQUE_LOOPBACK,
-    WILDCARD,
-    LOOPBACK
-  };
-  BindMode bind_mode;
+  MiniCluster::BindMode bind_mode;
 
   // The path where the kudu daemons should be run from.
   // Default: "", which uses the same path as the currently running executable.
@@ -183,6 +147,9 @@ class ExternalMiniCluster : public MiniCluster {
   // Otherwise, it is another IP in the local netblock.
   std::string GetBindIpForTabletServer(int index) const;
 
+  // Same as above but for a master.
+  std::string GetBindIpForMaster(int index) const;
+
   // Return a pointer to the running leader master. This may be NULL
   // if the cluster is not started.
   //
@@ -243,6 +210,16 @@ class ExternalMiniCluster : public MiniCluster {
     return masters_.size();
   }
 
+  BindMode bind_mode() const override {
+    return opts_.bind_mode;
+  }
+
+  std::vector<uint16_t> master_rpc_ports() const override {
+    return opts_.master_rpc_ports;
+  }
+
+  std::vector<HostPort> master_rpc_addrs() const override;
+
   std::shared_ptr<rpc::Messenger> messenger() const override;
   std::shared_ptr<master::MasterServiceProxy> master_proxy() const override;
   std::shared_ptr<master::MasterServiceProxy> master_proxy(int idx) const override;
@@ -345,6 +322,7 @@ struct ExternalDaemonOptions {
   bool logtostderr;
   std::shared_ptr<rpc::Messenger> messenger;
   std::string exe;
+  HostPort rpc_bind_address;
   std::string wal_dir;
   std::vector<std::string> data_dirs;
   std::string log_dir;
@@ -479,13 +457,10 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
   // LOG(FATAL) if there are any leaks.
   void CheckForLeaks();
 
-  // Get/Set rpc_bind_addresses for daemon.
-  virtual const std::string& get_rpc_bind_address() const {
+  // Get RPC bind address for daemon.
+  const HostPort& rpc_bind_address() const {
     return rpc_bind_address_;
   }
-  virtual void set_rpc_bind_address(std::string host) {
-    rpc_bind_address_ = host;
-  }
 
   const std::shared_ptr<rpc::Messenger> messenger_;
   const std::string wal_dir_;
@@ -494,6 +469,7 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
   const std::string perf_record_filename_;
   const MonoDelta start_process_timeout_;
   const bool logtostderr_;
+  const HostPort rpc_bind_address_;
   std::string exe_;
   std::vector<std::string> extra_flags_;
   std::map<std::string, std::string> extra_env_;
@@ -504,7 +480,6 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
   std::unique_ptr<Subprocess> perf_record_process_;
 
   std::unique_ptr<server::ServerStatusPB> status_;
-  std::string rpc_bind_address_;
 
   // These capture the daemons parameters and running ports and
   // are used to Restart() the daemon with the same parameters.
@@ -535,9 +510,6 @@ class ExternalMaster : public ExternalDaemon {
  public:
   explicit ExternalMaster(ExternalDaemonOptions opts);
 
-  ExternalMaster(ExternalDaemonOptions opts,
-                 std::string rpc_bind_address);
-
   Status Start();
 
   // Restarts the daemon.
@@ -558,7 +530,6 @@ class ExternalMaster : public ExternalDaemon {
 class ExternalTabletServer : public ExternalDaemon {
  public:
   ExternalTabletServer(ExternalDaemonOptions opts,
-                       std::string bind_host,
                        std::vector<HostPort> master_addrs);
 
   Status Start();
@@ -568,7 +539,7 @@ class ExternalTabletServer : public ExternalDaemon {
   Status Restart() WARN_UNUSED_RESULT;
 
  private:
-  const std::string master_addrs_;
+  const std::vector<HostPort> master_addrs_;
 
   friend class RefCountedThreadSafe<ExternalTabletServer>;
   virtual ~ExternalTabletServer();

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/internal_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/internal_mini_cluster.cc b/src/kudu/integration-tests/internal_mini_cluster.cc
index 9d7d3ae..bc53f3f 100644
--- a/src/kudu/integration-tests/internal_mini_cluster.cc
+++ b/src/kudu/integration-tests/internal_mini_cluster.cc
@@ -17,7 +17,6 @@
 
 #include "kudu/integration-tests/internal_mini_cluster.h"
 
-
 #include "kudu/client/client.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -50,19 +49,20 @@ using tserver::MiniTabletServer;
 using tserver::TabletServer;
 
 MiniClusterOptions::MiniClusterOptions()
- :  num_masters(1),
-    num_tablet_servers(1) {
+  : num_masters(1),
+    num_tablet_servers(1),
+    bind_mode(MiniCluster::kDefaultBindMode) {
 }
 
 InternalMiniCluster::InternalMiniCluster(Env* env, const MiniClusterOptions& options)
-  : running_(false),
-    env_(env),
+  : env_(env),
     fs_root_(!options.data_root.empty() ? options.data_root :
                 JoinPathSegments(GetTestDataDirectory(), "minicluster-data")),
     num_masters_initial_(options.num_masters),
     num_ts_initial_(options.num_tablet_servers),
     master_rpc_ports_(options.master_rpc_ports),
-    tserver_rpc_ports_(options.tserver_rpc_ports) {
+    tserver_rpc_ports_(options.tserver_rpc_ports),
+    running_(false) {
 }
 
 InternalMiniCluster::~InternalMiniCluster() {
@@ -111,17 +111,17 @@ Status InternalMiniCluster::StartDistributedMasters() {
   CHECK_GE(master_rpc_ports_.size(), num_masters_initial_);
   CHECK_GT(master_rpc_ports_.size(), 1);
 
-  LOG(INFO) << "Creating distributed mini masters. Ports: "
-            << JoinInts(master_rpc_ports_, ", ");
+  vector<HostPort> master_rpc_addrs = this->master_rpc_addrs();
+  LOG(INFO) << "Creating distributed mini masters. Addrs: "
+            << HostPort::ToCommaSeparatedString(master_rpc_addrs);
 
   for (int i = 0; i < num_masters_initial_; i++) {
-    gscoped_ptr<MiniMaster> mini_master(
-        new MiniMaster(env_, GetMasterFsRoot(i), master_rpc_ports_[i]));
-    RETURN_NOT_OK_PREPEND(mini_master->StartDistributedMaster(master_rpc_ports_),
+    shared_ptr<MiniMaster> mini_master(new MiniMaster(GetMasterFsRoot(i), master_rpc_addrs[i]));
+    RETURN_NOT_OK_PREPEND(mini_master->StartDistributedMaster(master_rpc_addrs),
                           Substitute("Couldn't start follower $0", i));
     VLOG(1) << "Started MiniMaster with UUID " << mini_master->permanent_uuid()
             << " at index " << i;
-    mini_masters_.push_back(shared_ptr<MiniMaster>(mini_master.release()));
+    mini_masters_.push_back(std::move(mini_master));
   }
   int i = 0;
   for (const shared_ptr<MiniMaster>& master : mini_masters_) {
@@ -152,12 +152,13 @@ Status InternalMiniCluster::StartSingleMaster() {
   }
 
   // start the master (we need the port to set on the servers).
-  gscoped_ptr<MiniMaster> mini_master(
-      new MiniMaster(env_, GetMasterFsRoot(0), master_rpc_port));
+  string bind_ip = GetBindIpForDaemon(MiniCluster::MASTER, /*index=*/ 0, opts_.bind_mode);
+  shared_ptr<MiniMaster> mini_master(
+      new MiniMaster(GetMasterFsRoot(0), HostPort(std::move(bind_ip), master_rpc_port)));
   RETURN_NOT_OK_PREPEND(mini_master->Start(), "Couldn't start master");
   RETURN_NOT_OK(mini_master->master()->WaitUntilCatalogManagerIsLeaderAndReadyForTests(
       MonoDelta::FromSeconds(kMasterStartupWaitTimeSeconds)));
-  mini_masters_.push_back(shared_ptr<MiniMaster>(mini_master.release()));
+  mini_masters_.push_back(std::move(mini_master));
   return Status::OK();
 }
 
@@ -171,8 +172,10 @@ Status InternalMiniCluster::AddTabletServer() {
   if (tserver_rpc_ports_.size() > new_idx) {
     ts_rpc_port = tserver_rpc_ports_[new_idx];
   }
+
+  string bind_ip = GetBindIpForDaemon(MiniCluster::TSERVER, new_idx, opts_.bind_mode);
   gscoped_ptr<MiniTabletServer> tablet_server(
-    new MiniTabletServer(GetTabletServerFsRoot(new_idx), ts_rpc_port));
+      new MiniTabletServer(GetTabletServerFsRoot(new_idx), HostPort(bind_ip, ts_rpc_port)));
 
   // set the master addresses
   tablet_server->options()->master_addresses.clear();
@@ -212,6 +215,16 @@ MiniTabletServer* InternalMiniCluster::mini_tablet_server(int idx) const {
   return mini_tablet_servers_[idx].get();
 }
 
+vector<HostPort> InternalMiniCluster::master_rpc_addrs() const {
+  vector<HostPort> master_rpc_addrs;
+  for (int i = 0; i < master_rpc_ports_.size(); i++) {
+    master_rpc_addrs.emplace_back(
+        GetBindIpForDaemon(MiniCluster::MASTER, i, opts_.bind_mode),
+        master_rpc_ports_[i]);
+  }
+  return master_rpc_addrs;
+}
+
 string InternalMiniCluster::GetMasterFsRoot(int idx) const {
   return JoinPathSegments(fs_root_, Substitute("master-$0-root", idx));
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/internal_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/internal_mini_cluster.h b/src/kudu/integration-tests/internal_mini_cluster.h
index 74a167f..e5fc6a1 100644
--- a/src/kudu/integration-tests/internal_mini_cluster.h
+++ b/src/kudu/integration-tests/internal_mini_cluster.h
@@ -59,12 +59,19 @@ struct MiniClusterOptions {
   // The default may only be used from a gtest unit test.
   std::string data_root;
 
+  MiniCluster::BindMode bind_mode;
+
   // List of RPC ports for the master to run on.
-  // Defaults to a list 0 (ephemeral ports).
+  // Defaults to an empty list.
+  // In single-master mode, an empty list implies port 0 (transient port).
+  // In multi-master mode, an empty list is illegal and will result in a CHECK failure.
   std::vector<uint16_t> master_rpc_ports;
 
   // List of RPC ports for the tservers to run on.
-  // Defaults to a list of 0 (ephemeral ports).
+  // Defaults to an empty list.
+  // When adding a tablet server to the cluster via AddTabletServer(), if the
+  // index of that tablet server in the cluster is greater than the number of
+  // elements in this list, a transient port (port 0) will be used.
   std::vector<uint16_t> tserver_rpc_ports;
 };
 
@@ -122,6 +129,16 @@ class InternalMiniCluster : public MiniCluster {
     return mini_tablet_servers_.size();
   }
 
+  BindMode bind_mode() const override {
+    return opts_.bind_mode;
+  }
+
+  std::vector<uint16_t> master_rpc_ports() const override {
+    return master_rpc_ports_;
+  }
+
+  std::vector<HostPort> master_rpc_addrs() const override;
+
   std::string GetMasterFsRoot(int idx) const;
 
   std::string GetTabletServerFsRoot(int idx) const;
@@ -171,7 +188,7 @@ class InternalMiniCluster : public MiniCluster {
     kMasterStartupWaitTimeSeconds = 30,
   };
 
-  bool running_;
+  const MiniClusterOptions opts_;
 
   Env* const env_;
   const std::string fs_root_;
@@ -181,6 +198,8 @@ class InternalMiniCluster : public MiniCluster {
   const std::vector<uint16_t> master_rpc_ports_;
   const std::vector<uint16_t> tserver_rpc_ports_;
 
+  bool running_;
+
   std::vector<std::shared_ptr<master::MiniMaster> > mini_masters_;
   std::vector<std::shared_ptr<tserver::MiniTabletServer> > mini_tablet_servers_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/master_migration-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_migration-itest.cc b/src/kudu/integration-tests/master_migration-itest.cc
index 234f1ce..338c4f7 100644
--- a/src/kudu/integration-tests/master_migration-itest.cc
+++ b/src/kudu/integration-tests/master_migration-itest.cc
@@ -186,12 +186,12 @@ TEST_F(MasterMigrationTest, TestEndToEndMigration) {
   // Bring down the old cluster configuration and bring up the new one.
   cluster_->Shutdown();
   ExternalMiniClusterOptions opts;
+  opts.bind_mode = MiniCluster::LOOPBACK; // Required since we use 127.0.0.1 in the config above.
   opts.master_rpc_ports = kMasterRpcPorts;
   opts.num_masters = kMasterRpcPorts.size();
   ExternalMiniCluster migrated_cluster(std::move(opts));
   ASSERT_OK(migrated_cluster.Start());
 
-
   // Perform an operation that requires an elected leader.
   shared_ptr<KuduClient> client;
   ASSERT_OK(migrated_cluster.CreateClient(nullptr, &client));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/master_replication-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_replication-itest.cc b/src/kudu/integration-tests/master_replication-itest.cc
index 02b7625..e5461d1 100644
--- a/src/kudu/integration-tests/master_replication-itest.cc
+++ b/src/kudu/integration-tests/master_replication-itest.cc
@@ -259,9 +259,9 @@ TEST_F(MasterReplicationTest, TestMasterPeerSetsDontMatch) {
   // Restart one master with an additional entry in --master_addresses. The
   // discrepancy with the on-disk list of masters should trigger a failure.
   cluster_->mini_master(0)->Shutdown();
-  vector<uint16_t> master_rpc_ports = opts_.master_rpc_ports;
-  master_rpc_ports.push_back(55555);
-  ASSERT_OK(cluster_->mini_master(0)->StartDistributedMaster(master_rpc_ports));
+  vector<HostPort> master_rpc_addrs = cluster_->master_rpc_addrs();
+  master_rpc_addrs.emplace_back("127.0.0.1", 55555);
+  ASSERT_OK(cluster_->mini_master(0)->StartDistributedMaster(master_rpc_addrs));
   Status s = cluster_->mini_master(0)->WaitForCatalogManagerInit();
   SCOPED_TRACE(s.ToString());
   ASSERT_TRUE(s.IsInvalidArgument());

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/mini_cluster.cc b/src/kudu/integration-tests/mini_cluster.cc
new file mode 100644
index 0000000..bbf3bc2
--- /dev/null
+++ b/src/kudu/integration-tests/mini_cluster.cc
@@ -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.
+
+#include "kudu/integration-tests/mini_cluster.h"
+
+#include <stdint.h>
+#include <unistd.h>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+
+string MiniCluster::GetBindIpForDaemon(DaemonType type, int index, BindMode bind_mode) {
+  switch (bind_mode) {
+    case UNIQUE_LOOPBACK: {
+      // IP address last octet range: [1 - 254].
+      uint8_t last_octet = (type == TSERVER) ? index + 1 : UINT8_MAX - 1 - index;
+      CHECK_GE(last_octet, 1);
+      CHECK_LE(last_octet, 254);
+      pid_t p = getpid();
+      CHECK_LE(p, UINT16_MAX) << "Cannot run on systems with >16-bit pid";
+      return Substitute("127.$0.$1.$2", p >> 8, p & 0xff, last_octet);
+    }
+    case WILDCARD:
+      return kWildcardIpAddr;
+    case LOOPBACK:
+      return kLoopbackIpAddr;
+    default:
+      LOG(FATAL) << bind_mode;
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/integration-tests/mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/mini_cluster.h b/src/kudu/integration-tests/mini_cluster.h
index 00342c2..854cd0b 100644
--- a/src/kudu/integration-tests/mini_cluster.h
+++ b/src/kudu/integration-tests/mini_cluster.h
@@ -16,7 +16,12 @@
 // under the License.
 #pragma once
 
+#include <memory>
+#include <string>
+#include <vector>
+
 #include "kudu/client/shared_ptr.h"
+#include "kudu/util/net/net_util.h"
 
 namespace kudu {
 class Status;
@@ -46,6 +51,55 @@ enum class ClusterNodes {
 // mini-cluster implementation is in-process or out-of-process.
 class MiniCluster {
  public:
+  // BindMode lets you specify the socket binding mode for RPC and/or HTTP server.
+  // A) LOOPBACK binds each server to loopback ip address "127.0.0.1".
+  //
+  // B) WILDCARD specifies "0.0.0.0" as the ip to bind to, which means sockets
+  // can be bound to any interface on the local host.
+  // For example, if a host has two interfaces with addresses
+  // 192.168.0.10 and 192.168.0.11, the server process can accept connection
+  // requests addressed to 192.168.0.10 or 192.168.0.11.
+  //
+  // C) UNIQUE_LOOPBACK binds each tablet server to a different loopback address.
+  // This affects the server's RPC server, and also forces the server to
+  // only use this IP address for outgoing socket connections as well.
+  // This allows the use of iptables on the localhost to simulate network
+  // partitions.
+  //
+  // The addresses used are 127.<A>.<B>.<C> where:
+  // - <A,B> are the high and low bytes of the pid of the process running the
+  //   minicluster (not the daemon itself).
+  // - <C> is the index of the server within this minicluster.
+  //
+  // This requires that the system is set up such that processes may bind
+  // to any IP address in the localhost netblock (127.0.0.0/8). This seems
+  // to be the case on common Linux distributions. You can verify by running
+  // 'ip addr | grep 127.0.0.1' and checking that the address is listed as
+  // '127.0.0.1/8'.
+  //
+  // Note: UNIQUE_LOOPBACK is not supported on macOS.
+  //
+  // Default: UNIQUE_LOOPBACK on Linux, LOOPBACK on macOS.
+  enum BindMode {
+    UNIQUE_LOOPBACK,
+    WILDCARD,
+    LOOPBACK
+  };
+
+#if defined(__APPLE__)
+  static constexpr const BindMode kDefaultBindMode = BindMode::LOOPBACK;
+#else
+  static constexpr const BindMode kDefaultBindMode = BindMode::UNIQUE_LOOPBACK;
+#endif
+
+  enum DaemonType {
+    MASTER,
+    TSERVER
+  };
+
+  static constexpr const char* const kWildcardIpAddr = "0.0.0.0";
+  static constexpr const char* const kLoopbackIpAddr = "127.0.0.1";
+
   MiniCluster() {}
   virtual ~MiniCluster() {}
 
@@ -65,6 +119,12 @@ class MiniCluster {
 
   virtual int num_tablet_servers() const = 0;
 
+  virtual BindMode bind_mode() const = 0;
+
+  virtual std::vector<uint16_t> master_rpc_ports() const = 0;
+
+  virtual std::vector<HostPort> master_rpc_addrs() const = 0;
+
   // Create a client configured to talk to this cluster. 'builder' may contain
   // override options for the client. The master address will be overridden to
   // talk to the running master(s). If 'builder' is a nullptr, default options
@@ -85,6 +145,12 @@ class MiniCluster {
   // Returns an RPC proxy to the master at 'idx'. Requires that the
   // master at 'idx' is running.
   virtual std::shared_ptr<master::MasterServiceProxy> master_proxy(int idx) const = 0;
+
+ protected:
+  // Return the IP address that the daemon with the given index will bind to.
+  // If bind_mode is LOOPBACK, this will be 127.0.0.1 and if it is WILDCARD it
+  // will be 0.0.0.0. Otherwise, it is another IP in the local netblock.
+  static std::string GetBindIpForDaemon(DaemonType type, int index, BindMode bind_mode);
 };
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/master/master-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 0ba583a..9a6324f 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -70,7 +70,7 @@ class MasterTest : public KuduTest {
     FLAGS_catalog_manager_check_ts_count_for_create_table = false;
 
     // Start master
-    mini_master_.reset(new MiniMaster(Env::Default(), GetTestPath("Master"), 0));
+    mini_master_.reset(new MiniMaster(GetTestPath("Master"), HostPort("127.0.0.1", 0)));
     ASSERT_OK(mini_master_->Start());
     master_ = mini_master_->master();
     ASSERT_OK(master_->WaitUntilCatalogManagerIsLeaderAndReadyForTests(MonoDelta::FromSeconds(5)));
@@ -117,7 +117,7 @@ static void MakeHostPortPB(const string& host, uint32_t port, HostPortPB* pb) {
 // Test that shutting down a MiniMaster without starting it does not
 // SEGV.
 TEST_F(MasterTest, TestShutdownWithoutStart) {
-  MiniMaster m(Env::Default(), "/xxxx", 0);
+  MiniMaster m("/xxxx", HostPort("127.0.0.1", 0));
   m.Shutdown();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/master/mini_master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/mini_master.cc b/src/kudu/master/mini_master.cc
index 0862869..f8d1328 100644
--- a/src/kudu/master/mini_master.cc
+++ b/src/kudu/master/mini_master.cc
@@ -1,4 +1,3 @@
-// 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
@@ -43,10 +42,9 @@ DECLARE_bool(rpc_server_allow_ephemeral_ports);
 namespace kudu {
 namespace master {
 
-MiniMaster::MiniMaster(Env* env, string fs_root, uint16_t rpc_port)
-    : env_(env),
-      fs_root_(std::move(fs_root)),
-      rpc_port_(rpc_port) {
+MiniMaster::MiniMaster(string fs_root, HostPort rpc_bind_addr)
+    : rpc_bind_addr_(std::move(rpc_bind_addr)),
+      fs_root_(std::move(fs_root)) {
   // Disable minidump handler (we allow only one per process).
   FLAGS_enable_minidumps = false;
 }
@@ -57,16 +55,21 @@ MiniMaster::~MiniMaster() {
 
 Status MiniMaster::Start() {
   FLAGS_rpc_server_allow_ephemeral_ports = true;
-  RETURN_NOT_OK(StartOnPorts(rpc_port_, 0));
+  MasterOptions opts;
+  HostPort web_bind_addr(rpc_bind_addr_.host(), /*port=*/ 0);
+  RETURN_NOT_OK(StartOnAddrs(rpc_bind_addr_, web_bind_addr, &opts));
   return master_->WaitForCatalogManagerInit();
 }
 
-Status MiniMaster::StartDistributedMaster(const vector<uint16_t>& peer_ports) {
-  return StartDistributedMasterOnPorts(rpc_port_, 0, peer_ports);
+Status MiniMaster::StartDistributedMaster(vector<HostPort> peer_addrs) {
+  HostPort web_bind_addr(rpc_bind_addr_.host(), /*port=*/ 0);
+  return StartDistributedMasterOnAddrs(rpc_bind_addr_, web_bind_addr,
+                                       std::move(peer_addrs));
 }
 
 Status MiniMaster::Restart() {
-  RETURN_NOT_OK(StartOnPorts(bound_rpc_.port(), bound_http_.port()));
+  MasterOptions opts;
+  RETURN_NOT_OK(StartOnAddrs(HostPort(bound_rpc_), HostPort(bound_http_), &opts));
   return WaitForCatalogManagerInit();
 }
 
@@ -99,16 +102,14 @@ std::string MiniMaster::bound_rpc_addr_str() const {
   return bound_rpc_addr().ToString();
 }
 
-Status MiniMaster::StartOnPorts(uint16_t rpc_port, uint16_t web_port) {
-  MasterOptions opts;
-  return StartOnPorts(rpc_port, web_port, &opts);
-}
-
-Status MiniMaster::StartOnPorts(uint16_t rpc_port, uint16_t web_port,
+Status MiniMaster::StartOnAddrs(const HostPort& rpc_bind_addr,
+                                const HostPort& web_bind_addr,
                                 MasterOptions* opts) {
   CHECK(!master_);
-  opts->rpc_opts.rpc_bind_addresses = Substitute("127.0.0.1:$0", rpc_port);
-  opts->webserver_opts.port = web_port;
+
+  opts->rpc_opts.rpc_bind_addresses = rpc_bind_addr.ToString();
+  opts->webserver_opts.bind_interface = web_bind_addr.host();
+  opts->webserver_opts.port = web_bind_addr.port();
   opts->fs_opts.wal_path = fs_root_;
   opts->fs_opts.data_paths = { fs_root_ };
 
@@ -120,17 +121,12 @@ Status MiniMaster::StartOnPorts(uint16_t rpc_port, uint16_t web_port,
   return Status::OK();
 }
 
-Status MiniMaster::StartDistributedMasterOnPorts(uint16_t rpc_port, uint16_t web_port,
-                                                 const vector<uint16_t>& peer_ports) {
-  vector<HostPort> peer_addresses;
-  for (uint16_t peer_port : peer_ports) {
-    HostPort peer_address("127.0.0.1", peer_port);
-    peer_addresses.push_back(peer_address);
-  }
+Status MiniMaster::StartDistributedMasterOnAddrs(const HostPort& rpc_bind_addr,
+                                                 const HostPort& web_bind_addr,
+                                                 std::vector<HostPort> peer_addrs) {
   MasterOptions opts;
-  opts.master_addresses = peer_addresses;
-
-  return StartOnPorts(rpc_port, web_port, &opts);
+  opts.master_addresses = std::move(peer_addrs);
+  return StartOnAddrs(rpc_bind_addr, web_bind_addr, &opts);
 }
 
 } // namespace master

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/master/mini_master.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/mini_master.h b/src/kudu/master/mini_master.h
index b509b3b..77f80b2 100644
--- a/src/kudu/master/mini_master.h
+++ b/src/kudu/master/mini_master.h
@@ -25,13 +25,12 @@
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/util/env.h"
+#include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
 
-class HostPort;
-
 namespace master {
 
 class Master;
@@ -43,15 +42,15 @@ struct MasterOptions;
 // having multiple Start methods.
 class MiniMaster {
  public:
-  MiniMaster(Env* env, std::string fs_root, uint16_t rpc_port);
+  MiniMaster(std::string fs_root, HostPort rpc_bind_addr);
   ~MiniMaster();
 
-  // Start a master running on the loopback interface and
-  // an ephemeral port. To determine the address that the server
-  // bound to, call MiniMaster::bound_addr()
+  // Start a master running on the address specified to the constructor.
+  // To determine the address that the server bound to, call
+  // MiniMaster::bound_addr()
   Status Start();
 
-  Status StartDistributedMaster(const std::vector<uint16_t>& peer_ports);
+  Status StartDistributedMaster(std::vector<HostPort> peer_addrs);
 
   // Restart the master on the same ports as it was previously bound.
   // Requires that the master is currently started.
@@ -75,17 +74,16 @@ class MiniMaster {
   std::string bound_rpc_addr_str() const;
 
  private:
-  Status StartOnPorts(uint16_t rpc_port, uint16_t web_port);
-
-  Status StartOnPorts(uint16_t rpc_port, uint16_t web_port,
+  Status StartOnAddrs(const HostPort& rpc_bind_addr,
+                      const HostPort& web_bind_addr,
                       MasterOptions* opts);
 
-  Status StartDistributedMasterOnPorts(uint16_t rpc_port, uint16_t web_port,
-                                       const std::vector<uint16_t>& peer_ports);
+  Status StartDistributedMasterOnAddrs(const HostPort& rpc_bind_addr,
+                                       const HostPort& web_bind_addr,
+                                       std::vector<HostPort> peer_addrs);
 
-  ATTRIBUTE_MEMBER_UNUSED Env* const env_;
+  const HostPort rpc_bind_addr_;
   const std::string fs_root_;
-  const uint16_t rpc_port_;
   Sockaddr bound_rpc_;
   Sockaddr bound_http_;
   std::unique_ptr<Master> master_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/master/sys_catalog-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog-test.cc b/src/kudu/master/sys_catalog-test.cc
index 12d9c69..b00339d 100644
--- a/src/kudu/master/sys_catalog-test.cc
+++ b/src/kudu/master/sys_catalog-test.cc
@@ -57,7 +57,7 @@ class SysCatalogTest : public KuduTest {
     KuduTest::SetUp();
 
     // Start master
-    mini_master_.reset(new MiniMaster(Env::Default(), GetTestPath("Master"), 0));
+    mini_master_.reset(new MiniMaster(GetTestPath("Master"), HostPort("127.0.0.1", 0)));
     ASSERT_OK(mini_master_->Start());
     master_ = mini_master_->master();
     ASSERT_OK(master_->WaitUntilCatalogManagerIsLeaderAndReadyForTests(MonoDelta::FromSeconds(5)));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 13b7530..71254cc 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1309,7 +1309,7 @@ TEST_F(ToolTest, TestRemoteReplicaCopy) {
   // on the tablet servers in the cluster giving us the test coverage
   // for KUDU-1776. With this, 'kudu remote_replica copy' can be used to
   // connect to tablet servers bound to wildcard ip addresses.
-  cluster_opts_.bind_mode = ExternalMiniClusterOptions::WILDCARD;
+  cluster_opts_.bind_mode = MiniCluster::WILDCARD;
   NO_FATALS(StartExternalMiniCluster(
       {"--catalog_manager_wait_for_new_tablets_to_elect_leader=false"},
       {"--enable_leader_failure_detection=false"}, kNumTservers));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/tserver/mini_tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/mini_tablet_server.cc b/src/kudu/tserver/mini_tablet_server.cc
index 9ee60b4..b13484d 100644
--- a/src/kudu/tserver/mini_tablet_server.cc
+++ b/src/kudu/tserver/mini_tablet_server.cc
@@ -44,12 +44,13 @@ namespace kudu {
 namespace tserver {
 
 MiniTabletServer::MiniTabletServer(const string& fs_root,
-                                   uint16_t rpc_port) {
+                                   const HostPort& rpc_bind_addr) {
   // Disable minidump handler (we allow only one per process).
   FLAGS_enable_minidumps = false;
   // Start RPC server on loopback.
   FLAGS_rpc_server_allow_ephemeral_ports = true;
-  opts_.rpc_opts.rpc_bind_addresses = Substitute("127.0.0.1:$0", rpc_port);
+  opts_.rpc_opts.rpc_bind_addresses = rpc_bind_addr.ToString();
+  opts_.webserver_opts.bind_interface = rpc_bind_addr.host();
   opts_.webserver_opts.port = 0;
   opts_.fs_opts.wal_path = fs_root;
   opts_.fs_opts.data_paths = { fs_root };
@@ -76,11 +77,11 @@ Status MiniTabletServer::WaitStarted() {
 
 void MiniTabletServer::Shutdown() {
   if (server_) {
-    // Save the bound ports back into the options structure so that, if we restart the
+    // Save the bound addrs back into the options structure so that, if we restart the
     // server, it will come back on the same address. This is necessary since we don't
     // currently support tablet servers re-registering on different ports (KUDU-418).
+    opts_.rpc_opts.rpc_bind_addresses = bound_rpc_addr().ToString();
     opts_.webserver_opts.port = bound_http_addr().port();
-    opts_.rpc_opts.rpc_bind_addresses = Substitute("127.0.0.1:$0", bound_rpc_addr().port());
     server_->Shutdown();
     server_.reset();
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/tserver/mini_tablet_server.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/mini_tablet_server.h b/src/kudu/tserver/mini_tablet_server.h
index ffa8b59..88d1289 100644
--- a/src/kudu/tserver/mini_tablet_server.h
+++ b/src/kudu/tserver/mini_tablet_server.h
@@ -41,7 +41,8 @@ class TabletServer;
 // An in-process tablet server meant for use in test cases.
 class MiniTabletServer {
  public:
-  MiniTabletServer(const std::string& fs_root, uint16_t rpc_port);
+  // Note: The host portion of 'rpc_bind_addr' is also used for the http service.
+  MiniTabletServer(const std::string& fs_root, const HostPort& rpc_bind_addr);
   ~MiniTabletServer();
 
   // Return the options which will be used to start the tablet server.

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/tserver/tablet_server-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test-base.h b/src/kudu/tserver/tablet_server-test-base.h
index 800c5cd..2ea9de0 100644
--- a/src/kudu/tserver/tablet_server-test-base.h
+++ b/src/kudu/tserver/tablet_server-test-base.h
@@ -100,7 +100,8 @@ class TabletServerTestBase : public KuduTest {
   virtual void StartTabletServer() {
     // Start server with an invalid master address, so it never successfully
     // heartbeats, even if there happens to be a master running on this machine.
-    mini_server_.reset(new MiniTabletServer(GetTestPath("TabletServerTest-fsroot"), 0));
+    mini_server_.reset(new MiniTabletServer(GetTestPath("TabletServerTest-fsroot"),
+                                            HostPort("127.0.0.1", 0)));
     mini_server_->options()->master_addresses.clear();
     mini_server_->options()->master_addresses.emplace_back("255.255.255.255", 1);
     ASSERT_OK(mini_server_->Start());
@@ -343,7 +344,8 @@ class TabletServerTestBase : public KuduTest {
     ShutdownTablet();
 
     // Start server.
-    mini_server_.reset(new MiniTabletServer(GetTestPath("TabletServerTest-fsroot"), 0));
+    mini_server_.reset(new MiniTabletServer(GetTestPath("TabletServerTest-fsroot"),
+                                            HostPort("127.0.0.1", 0)));
     mini_server_->options()->master_addresses.clear();
     mini_server_->options()->master_addresses.emplace_back("255.255.255.255", 1);
     // this should open the tablet created on StartTabletServer()

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6758739/src/kudu/tserver/ts_tablet_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager-test.cc b/src/kudu/tserver/ts_tablet_manager-test.cc
index 570d8dc..c65abdc 100644
--- a/src/kudu/tserver/ts_tablet_manager-test.cc
+++ b/src/kudu/tserver/ts_tablet_manager-test.cc
@@ -59,9 +59,8 @@ class TsTabletManagerTest : public KuduTest {
 
   virtual void SetUp() OVERRIDE {
     KuduTest::SetUp();
-
-    mini_server_.reset(
-        new MiniTabletServer(GetTestPath("TsTabletManagerTest-fsroot"), 0));
+    mini_server_.reset(new MiniTabletServer(GetTestPath("TsTabletManagerTest-fsroot"),
+                                            HostPort("127.0.0.1", 0)));
     ASSERT_OK(mini_server_->Start());
     mini_server_->FailHeartbeats();
 
@@ -135,8 +134,8 @@ TEST_F(TsTabletManagerTest, TestCreateTablet) {
   LOG(INFO) << "Shutting down tablet manager";
   mini_server_->Shutdown();
   LOG(INFO) << "Restarting tablet manager";
-  mini_server_.reset(
-      new MiniTabletServer(GetTestPath("TsTabletManagerTest-fsroot"), 0));
+  mini_server_.reset(new MiniTabletServer(GetTestPath("TsTabletManagerTest-fsroot"),
+                                          HostPort("127.0.0.1", 0)));
   ASSERT_OK(mini_server_->Start());
   ASSERT_OK(mini_server_->WaitStarted());
   tablet_manager_ = mini_server_->server()->tablet_manager();