You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/10/05 17:42:36 UTC

[1/2] kudu git commit: KUDU-2032 (part 1): pass pre-resolution hostname into RPC proxies

Repository: kudu
Updated Branches:
  refs/heads/branch-1.3.x 557183861 -> 1aa969522


KUDU-2032 (part 1): pass pre-resolution hostname into RPC proxies

This modifies the constructor of RPC proxies (generated and otherwise)
to take the remote hostname in addition to the existing resolved
Sockaddr parameter. The hostname is then passed into the ConnectionId
object, and plumbed through to the SASL client in place of the IP
address that was used previously.

The patch changes all of the construction sites of Proxy to fit the new
interface. In most of the test cases, we don't have real hostnames, so
we just use the dotted-decimal string form of the remote Sockaddr, which
matches the existing behavior.

In the real call sites, we have actual host names typically specified by
the user, and in those cases we'll need to pass those into the proxy. In
a few cases, they were conveniently available in the same function that
creates the proxy. In others, they are relatively far away, so this
patch just uses the dotted-decimal string and leaves TODOs.

In the case that Kerberos is not configured, this change should have no
effect since the hostname is ignored by SASL "plain". In the case that
Kerberos is configured with 'rdns=true', they also have no effect,
because the krb5 library will resolve and reverse the hostname from the
IP as it did before. In the case that 'rdns=false', this moves us one
step closer to fixing KUDU-2032 by getting a hostname into the SASL
library.

I verified that, if I set 'rdns = false' on a Kerberized client, I'm now
able to run  'kudu master status <host>' successfully where it would not
before. This tool uses a direct proxy instantiation where the hostname
was easy to plumb in. 'kudu table list <host>' still does not work because
it uses the client, which wasn't convenient to plumb quite yet.

Given that this makes incremental improvement towards fixing the issue
without any regression, and is already a fairly wide patch, I hope to
commit this and then address the remaining plumbing in a separate patch.

Change-Id: I9d2173de4ff2dfff4c576cf19b14e1a23521788b
Reviewed-on: http://gerrit.cloudera.org:8080/8182
Tested-by: Kudu Jenkins
Reviewed-by: Will Berkeley <wd...@gmail.com>


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

Branch: refs/heads/branch-1.3.x
Commit: de204012f464e7902d53f16440cd6055c070be58
Parents: 5571838
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Aug 15 18:20:48 2017 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Thu Oct 5 04:51:02 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client-internal.cc              |  3 +-
 src/kudu/client/master_rpc.cc                   | 11 ++---
 src/kudu/client/meta_cache.cc                   |  2 +-
 src/kudu/consensus/consensus_peers.cc           |  2 +-
 .../create-table-stress-test.cc                 |  4 +-
 .../exactly_once_writes-itest.cc                |  4 +-
 .../integration-tests/external_mini_cluster.cc  | 21 +++++----
 .../integration-tests/master-stress-test.cc     |  4 +-
 .../master_cert_authority-itest.cc              |  4 +-
 .../master_replication-itest.cc                 |  4 +-
 src/kudu/integration-tests/mini_cluster.cc      |  4 +-
 src/kudu/integration-tests/security-itest.cc    |  8 ++--
 .../integration-tests/table_locations-itest.cc  |  4 +-
 .../ts_tablet_manager-itest.cc                  |  6 ++-
 src/kudu/master/master-test.cc                  |  3 +-
 src/kudu/master/master.cc                       |  2 +-
 src/kudu/master/sys_catalog-test.cc             |  4 +-
 src/kudu/master/ts_descriptor.cc                | 17 ++++---
 src/kudu/master/ts_descriptor.h                 |  4 +-
 src/kudu/rpc/connection.h                       | 15 +++---
 src/kudu/rpc/connection_id.cc                   | 44 +++++++++---------
 src/kudu/rpc/connection_id.h                    | 20 ++++----
 src/kudu/rpc/exactly_once_rpc-test.cc           |  3 +-
 src/kudu/rpc/mt-rpc-test.cc                     |  6 ++-
 src/kudu/rpc/negotiation.cc                     |  9 ++--
 src/kudu/rpc/protoc-gen-krpc.cc                 | 11 +++--
 src/kudu/rpc/proxy.cc                           |  9 ++--
 src/kudu/rpc/proxy.h                            |  5 +-
 src/kudu/rpc/reactor.cc                         | 19 +++++---
 src/kudu/rpc/rpc-bench.cc                       |  4 +-
 src/kudu/rpc/rpc-test.cc                        | 49 +++++++++++++-------
 src/kudu/rpc/rpc_stub-test.cc                   | 47 ++++++++++---------
 src/kudu/tools/ksck_remote.cc                   |  6 ++-
 src/kudu/tools/tool_action_common.cc            |  2 +-
 src/kudu/tserver/heartbeater.cc                 |  2 +-
 src/kudu/tserver/tablet_copy_client.cc          |  2 +-
 src/kudu/tserver/tablet_copy_service-test.cc    |  4 +-
 src/kudu/tserver/tablet_server-test.cc          |  3 +-
 src/kudu/tserver/tablet_server_test_util.cc     | 11 +++--
 src/kudu/util/net/net_util-test.cc              |  1 +
 src/kudu/util/net/sockaddr.cc                   |  4 +-
 src/kudu/util/net/sockaddr.h                    |  3 ++
 42 files changed, 225 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/client/client-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index 9a2d81d..7e3f49b 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -629,7 +629,8 @@ void KuduClient::Data::ConnectedToClusterCb(
 
     if (status.ok()) {
       leader_master_hostport_ = HostPort(leader_addr);
-      master_proxy_.reset(new MasterServiceProxy(messenger_, leader_addr));
+      // TODO(KUDU-2032): retain the original hostname passed by caller.
+      master_proxy_.reset(new MasterServiceProxy(messenger_, leader_addr, leader_addr.host()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/client/master_rpc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/master_rpc.cc b/src/kudu/client/master_rpc.cc
index 3d97f3d..89b2979 100644
--- a/src/kudu/client/master_rpc.cc
+++ b/src/kudu/client/master_rpc.cc
@@ -106,21 +106,20 @@ ConnectToMasterRpc::~ConnectToMasterRpc() {
 }
 
 void ConnectToMasterRpc::SendRpc() {
-  MasterServiceProxy proxy(retrier().messenger(),
-                           addr_);
-  rpc::RpcController* rpc = mutable_retrier()->mutable_controller();
+  MasterServiceProxy proxy(retrier().messenger(), addr_, addr_.host());
+  rpc::RpcController* controller = mutable_retrier()->mutable_controller();
   // TODO(todd): should this be setting an RPC call deadline based on 'deadline'?
   // it doesn't seem to be.
   if (!trying_old_rpc_) {
     ConnectToMasterRequestPB req;
-    rpc->RequireServerFeature(master::MasterFeatures::CONNECT_TO_MASTER);
-    proxy.ConnectToMasterAsync(req, out_, rpc,
+    controller->RequireServerFeature(master::MasterFeatures::CONNECT_TO_MASTER);
+    proxy.ConnectToMasterAsync(req, out_, controller,
                                boost::bind(&ConnectToMasterRpc::SendRpcCb,
                                            this,
                                            Status::OK()));
   } else {
     GetMasterRegistrationRequestPB req;
-    proxy.GetMasterRegistrationAsync(req, &old_rpc_resp_, rpc,
+    proxy.GetMasterRegistrationAsync(req, &old_rpc_resp_, controller,
                                      boost::bind(&ConnectToMasterRpc::SendRpcCb,
                                                  this,
                                                  Status::OK()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/client/meta_cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/meta_cache.cc b/src/kudu/client/meta_cache.cc
index 9be8d64..89f8cff 100644
--- a/src/kudu/client/meta_cache.cc
+++ b/src/kudu/client/meta_cache.cc
@@ -97,7 +97,7 @@ void RemoteTabletServer::DnsResolutionFinished(const HostPort& hp,
 
   {
     std::lock_guard<simple_spinlock> l(lock_);
-    proxy_.reset(new TabletServerServiceProxy(client->data_->messenger_, (*addrs)[0]));
+    proxy_.reset(new TabletServerServiceProxy(client->data_->messenger_, (*addrs)[0], hp.host()));
   }
   user_callback.Run(s);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 62e8d84..e07898a 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -435,7 +435,7 @@ Status CreateConsensusServiceProxyForHost(const shared_ptr<Messenger>& messenger
     << "resolves to " << addrs.size() << " different addresses. Using "
     << addrs[0].ToString();
   }
-  new_proxy->reset(new ConsensusServiceProxy(messenger, addrs[0]));
+  new_proxy->reset(new ConsensusServiceProxy(messenger, addrs[0], hostport.host()));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/create-table-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/create-table-stress-test.cc b/src/kudu/integration-tests/create-table-stress-test.cc
index 9e1abed..4f6c6d2 100644
--- a/src/kudu/integration-tests/create-table-stress-test.cc
+++ b/src/kudu/integration-tests/create-table-stress-test.cc
@@ -98,8 +98,8 @@ class CreateTableStressTest : public KuduTest {
               .set_num_reactors(1)
               .set_max_negotiation_threads(1)
               .Build(&messenger_));
-    master_proxy_.reset(new MasterServiceProxy(messenger_,
-                                               cluster_->mini_master()->bound_rpc_addr()));
+    const auto& addr = cluster_->mini_master()->bound_rpc_addr();
+    master_proxy_.reset(new MasterServiceProxy(messenger_, addr, addr.host()));
     ASSERT_OK(CreateTabletServerMap(master_proxy_.get(), messenger_, &ts_map_));
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/exactly_once_writes-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/exactly_once_writes-itest.cc b/src/kudu/integration-tests/exactly_once_writes-itest.cc
index a89a129..1cf664f 100644
--- a/src/kudu/integration-tests/exactly_once_writes-itest.cc
+++ b/src/kudu/integration-tests/exactly_once_writes-itest.cc
@@ -77,8 +77,8 @@ void ExactlyOnceSemanticsITest::WriteRowsAndCollectResponses(int thread_idx,
   rpc::MessengerBuilder bld("Client");
   ASSERT_OK(bld.Build(&client_messenger));
 
-  std::unique_ptr<TabletServerServiceProxy> proxy(new TabletServerServiceProxy(client_messenger,
-                                                                               address));
+  std::unique_ptr<TabletServerServiceProxy> proxy(new TabletServerServiceProxy(
+      client_messenger, address, address.host()));
   for (int i = 0; i < num_batches; i++) {
     // Wait for all of the other writer threads to finish their attempts of the prior
     // batch before continuing on to the next one. This has two important effects:

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/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 c5eb77f..5c877cd 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -433,7 +433,7 @@ void ExternalMiniCluster::AssertNoCrashes() {
 Status ExternalMiniCluster::WaitForTabletsRunning(ExternalTabletServer* ts,
                                                   int min_tablet_count,
                                                   const MonoDelta& timeout) {
-  TabletServerServiceProxy proxy(messenger_, ts->bound_rpc_addr());
+  TabletServerServiceProxy proxy(messenger_, ts->bound_rpc_addr(), ts->bound_rpc_addr().host());
   ListTabletsRequestPB req;
   ListTabletsResponsePB resp;
 
@@ -548,8 +548,8 @@ std::shared_ptr<MasterServiceProxy> ExternalMiniCluster::master_proxy() const {
 
 std::shared_ptr<MasterServiceProxy> ExternalMiniCluster::master_proxy(int idx) const {
   CHECK_LT(idx, masters_.size());
-  return std::shared_ptr<MasterServiceProxy>(
-      new MasterServiceProxy(messenger_, CHECK_NOTNULL(master(idx))->bound_rpc_addr()));
+  const auto& addr = CHECK_NOTNULL(master(idx))->bound_rpc_addr();
+  return std::make_shared<MasterServiceProxy>(messenger_, addr, addr.host());
 }
 
 Status ExternalMiniCluster::CreateClient(client::KuduClientBuilder* builder,
@@ -570,7 +570,8 @@ Status ExternalMiniCluster::CreateClient(client::KuduClientBuilder* builder,
 Status ExternalMiniCluster::SetFlag(ExternalDaemon* daemon,
                                     const string& flag,
                                     const string& value) {
-  server::GenericServiceProxy proxy(messenger_, daemon->bound_rpc_addr());
+  const auto& addr = daemon->bound_rpc_addr();
+  server::GenericServiceProxy proxy(messenger_, addr, addr.host());
 
   rpc::RpcController controller;
   controller.set_timeout(MonoDelta::FromSeconds(30));
@@ -856,10 +857,11 @@ void ExternalDaemon::Shutdown() {
 
 void ExternalDaemon::FlushCoverage() {
 #ifndef COVERAGE_BUILD
-  return;
+  return; // NOLINT(*)
 #else
   LOG(INFO) << "Attempting to flush coverage for " << exe_ << " pid " << process_->pid();
-  server::GenericServiceProxy proxy(messenger_, bound_rpc_addr());
+  server::GenericServiceProxy proxy(
+      messenger_, bound_rpc_addr(), bound_rpc_addr().host());
 
   server::FlushCoverageRequestPB req;
   server::FlushCoverageResponsePB resp;
@@ -878,7 +880,8 @@ void ExternalDaemon::CheckForLeaks() {
 #if defined(__has_feature)
 #  if __has_feature(address_sanitizer)
   LOG(INFO) << "Attempting to check leaks for " << exe_ << " pid " << process_->pid();
-  server::GenericServiceProxy proxy(messenger_, bound_rpc_addr());
+  server::GenericServiceProxy proxy(messenger_, bound_rpc_addr(), bound_rpc_addr().host());
+
   server::CheckLeaksRequestPB req;
   server::CheckLeaksResponsePB resp;
   rpc::RpcController rpc;
@@ -1046,8 +1049,8 @@ Status ExternalMaster::Restart() {
 }
 
 Status ExternalMaster::WaitForCatalogManager() {
-  unique_ptr<MasterServiceProxy> proxy(
-      new MasterServiceProxy(messenger_, bound_rpc_addr()));
+  unique_ptr<MasterServiceProxy> proxy(new MasterServiceProxy(
+      messenger_, bound_rpc_addr(), bound_rpc_addr().host()));
   Stopwatch sw;
   sw.start();
   while (sw.elapsed().wall_seconds() < kMasterCatalogManagerTimeoutSeconds) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/master-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master-stress-test.cc b/src/kudu/integration-tests/master-stress-test.cc
index b99ea5e..ed41e6d 100644
--- a/src/kudu/integration-tests/master-stress-test.cc
+++ b/src/kudu/integration-tests/master-stress-test.cc
@@ -241,8 +241,8 @@ class MasterStressTest : public KuduTest {
 
   Status WaitForMasterUpAndRunning(const shared_ptr<Messenger>& messenger,
                                    ExternalMaster* master) {
-    unique_ptr<MasterServiceProxy> proxy(
-        new MasterServiceProxy(messenger, master->bound_rpc_addr()));
+    const auto& addr = master->bound_rpc_addr();
+    unique_ptr<MasterServiceProxy> proxy(new MasterServiceProxy(messenger, addr, addr.host()));
     while (true) {
       ListTablesRequestPB req;
       ListTablesResponsePB resp;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/master_cert_authority-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_cert_authority-itest.cc b/src/kudu/integration-tests/master_cert_authority-itest.cc
index 6d6016a..b69c3ff 100644
--- a/src/kudu/integration-tests/master_cert_authority-itest.cc
+++ b/src/kudu/integration-tests/master_cert_authority-itest.cc
@@ -118,7 +118,7 @@ class MasterCertAuthorityTest : public KuduTest {
       if (!m->is_running()) {
         continue;
       }
-      MasterServiceProxy proxy(messenger_, m->bound_rpc_addr());
+      MasterServiceProxy proxy(messenger_, m->bound_rpc_addr(), m->bound_rpc_addr().host());
 
       // All masters (including followers) should accept the heartbeat.
       ASSERT_OK(proxy.TSHeartbeat(req, &resp, &rpc));
@@ -148,7 +148,7 @@ class MasterCertAuthorityTest : public KuduTest {
       if (!m->is_running()) {
         continue;
       }
-      MasterServiceProxy proxy(messenger_, m->bound_rpc_addr());
+      MasterServiceProxy proxy(messenger_, m->bound_rpc_addr(), m->bound_rpc_addr().host());
 
       // All masters (including followers) should accept the heartbeat.
       RETURN_NOT_OK(proxy.TSHeartbeat(req, &resp, &rpc));

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/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 8a3f38a..c45b230 100644
--- a/src/kudu/integration-tests/master_replication-itest.cc
+++ b/src/kudu/integration-tests/master_replication-itest.cc
@@ -239,8 +239,8 @@ TEST_F(MasterReplicationTest, TestHeartbeatAcceptedByAnyMaster) {
     req.mutable_common()->CopyFrom(common);
     req.mutable_registration()->CopyFrom(fake_reg);
 
-    MasterServiceProxy proxy(messenger,
-                             cluster_->mini_master(i)->bound_rpc_addr());
+    const auto& addr = cluster_->mini_master(i)->bound_rpc_addr();
+    MasterServiceProxy proxy(messenger, addr, addr.host());
 
     // All masters (including followers) should accept the heartbeat.
     ASSERT_OK(proxy.TSHeartbeat(req, &resp, &rpc));

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/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
index c910502..4883e6c 100644
--- a/src/kudu/integration-tests/mini_cluster.cc
+++ b/src/kudu/integration-tests/mini_cluster.cc
@@ -341,8 +341,8 @@ std::shared_ptr<MasterServiceProxy> MiniCluster::master_proxy() const {
 }
 
 std::shared_ptr<MasterServiceProxy> MiniCluster::master_proxy(int idx) const {
-  return std::shared_ptr<MasterServiceProxy>(
-      new MasterServiceProxy(messenger_, CHECK_NOTNULL(mini_master(idx))->bound_rpc_addr()));
+  const auto& addr = CHECK_NOTNULL(mini_master(idx))->bound_rpc_addr();
+  return std::make_shared<MasterServiceProxy>(messenger_, addr, addr.host());
 }
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/security-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 4dec15f..9911081 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -52,8 +52,8 @@ class SecurityITest : public KuduTest {
     // Make a new messenger so that we don't reuse any cached connections from
     // the minicluster startup sequence.
     auto messenger = NewMessengerOrDie();
-    server::GenericServiceProxy proxy(
-        messenger, cluster_->tablet_server(0)->bound_rpc_addr());
+    const auto& addr = cluster_->tablet_server(0)->bound_rpc_addr();
+    server::GenericServiceProxy proxy(messenger, addr, addr.host());
 
     rpc::RpcController controller;
     controller.set_timeout(MonoDelta::FromSeconds(30));
@@ -71,8 +71,8 @@ class SecurityITest : public KuduTest {
     // Make a new messenger so that we don't reuse any cached connections from
     // the minicluster startup sequence.
     auto messenger = NewMessengerOrDie();
-    master::MasterServiceProxy proxy(
-        messenger, cluster_->master(0)->bound_rpc_addr());
+    const auto& addr = cluster_->master(0)->bound_rpc_addr();
+    master::MasterServiceProxy proxy(messenger, addr, addr.host());
 
     rpc::RpcController rpc;
     master::TSHeartbeatRequestPB req;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/table_locations-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/table_locations-itest.cc b/src/kudu/integration-tests/table_locations-itest.cc
index 775ba25..e7bb1eb 100644
--- a/src/kudu/integration-tests/table_locations-itest.cc
+++ b/src/kudu/integration-tests/table_locations-itest.cc
@@ -66,8 +66,8 @@ class TableLocationsTest : public KuduTest {
     // Create a client proxy to the master.
     MessengerBuilder bld("Client");
     ASSERT_OK(bld.Build(&client_messenger_));
-    proxy_.reset(new MasterServiceProxy(client_messenger_,
-                                        cluster_->mini_master()->bound_rpc_addr()));
+    const auto& addr = cluster_->mini_master()->bound_rpc_addr();
+    proxy_.reset(new MasterServiceProxy(client_messenger_, addr, addr.host()));
   }
 
   void TearDown() override {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/integration-tests/ts_tablet_manager-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_tablet_manager-itest.cc b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
index 2a3f0d3..f5dfbad 100644
--- a/src/kudu/integration-tests/ts_tablet_manager-itest.cc
+++ b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
@@ -65,6 +65,7 @@ using master::ReportedTabletPB;
 using master::TabletReportPB;
 using rpc::Messenger;
 using rpc::MessengerBuilder;
+using std::shared_ptr;
 using strings::Substitute;
 using tablet::TabletPeer;
 using tserver::MiniTabletServer;
@@ -126,8 +127,9 @@ TEST_F(TsTabletManagerITest, TestReportNewLeaderOnLeaderChange) {
   ASSERT_OK(client_->OpenTable(kTableName, &table));
 
   // Build a TServerDetails map so we can check for convergence.
-  gscoped_ptr<MasterServiceProxy> master_proxy(
-      new MasterServiceProxy(client_messenger_, cluster_->mini_master()->bound_rpc_addr()));
+  const auto& addr = cluster_->mini_master()->bound_rpc_addr();
+  shared_ptr<MasterServiceProxy> master_proxy(
+      new MasterServiceProxy(client_messenger_, addr, addr.host()));
 
   itest::TabletServerMap ts_map;
   ASSERT_OK(CreateTabletServerMap(master_proxy.get(), client_messenger_, &ts_map));

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/master/master-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 3777513..42402bd 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -78,7 +78,8 @@ class MasterTest : public KuduTest {
     // Create a client proxy to it.
     MessengerBuilder bld("Client");
     ASSERT_OK(bld.Build(&client_messenger_));
-    proxy_.reset(new MasterServiceProxy(client_messenger_, mini_master_->bound_rpc_addr()));
+    proxy_.reset(new MasterServiceProxy(client_messenger_, mini_master_->bound_rpc_addr(),
+                                        mini_master_->bound_rpc_addr().host()));
   }
 
   virtual void TearDown() OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/master/master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index 1d1d78a..a406482 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -260,7 +260,7 @@ Status GetMasterEntryForHost(const shared_ptr<rpc::Messenger>& messenger,
                              ServerEntryPB* e) {
   Sockaddr sockaddr;
   RETURN_NOT_OK(SockaddrFromHostPort(hostport, &sockaddr));
-  MasterServiceProxy proxy(messenger, sockaddr);
+  MasterServiceProxy proxy(messenger, sockaddr, hostport.host());
   GetMasterRegistrationRequestPB req;
   GetMasterRegistrationResponsePB resp;
   rpc::RpcController controller;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/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 7d8f361..dc1726e 100644
--- a/src/kudu/master/sys_catalog-test.cc
+++ b/src/kudu/master/sys_catalog-test.cc
@@ -65,7 +65,9 @@ class SysCatalogTest : public KuduTest {
     // Create a client proxy to it.
     MessengerBuilder bld("Client");
     ASSERT_OK(bld.Build(&client_messenger_));
-    proxy_.reset(new MasterServiceProxy(client_messenger_, mini_master_->bound_rpc_addr()));
+    proxy_.reset(new MasterServiceProxy(
+        client_messenger_, mini_master_->bound_rpc_addr(),
+        mini_master_->bound_rpc_addr().host()));
   }
 
   virtual void TearDown() OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/master/ts_descriptor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_descriptor.cc b/src/kudu/master/ts_descriptor.cc
index 6f93bb3..8093b94 100644
--- a/src/kudu/master/ts_descriptor.cc
+++ b/src/kudu/master/ts_descriptor.cc
@@ -192,7 +192,7 @@ void TSDescriptor::GetNodeInstancePB(NodeInstancePB* instance_pb) const {
   instance_pb->set_instance_seqno(latest_seqno_);
 }
 
-Status TSDescriptor::ResolveSockaddr(Sockaddr* addr) const {
+Status TSDescriptor::ResolveSockaddr(Sockaddr* addr, string* host) const {
   vector<HostPort> hostports;
   {
     std::lock_guard<simple_spinlock> l(lock_);
@@ -212,7 +212,7 @@ Status TSDescriptor::ResolveSockaddr(Sockaddr* addr) const {
     }
   }
 
-  if (addrs.size() == 0) {
+  if (addrs.empty()) {
     return Status::NetworkError("Unable to find the TS address: ",
                                 SecureDebugString(*registration_));
   }
@@ -223,6 +223,7 @@ Status TSDescriptor::ResolveSockaddr(Sockaddr* addr) const {
                   << addrs[0].ToString();
   }
   *addr = addrs[0];
+  *host = last_hostport.host();
   return Status::OK();
 }
 
@@ -237,11 +238,13 @@ Status TSDescriptor::GetTSAdminProxy(const shared_ptr<rpc::Messenger>& messenger
   }
 
   Sockaddr addr;
-  RETURN_NOT_OK(ResolveSockaddr(&addr));
+  string host;
+  RETURN_NOT_OK(ResolveSockaddr(&addr, &host));
 
   std::lock_guard<simple_spinlock> l(lock_);
   if (!ts_admin_proxy_) {
-    ts_admin_proxy_.reset(new tserver::TabletServerAdminServiceProxy(messenger, addr));
+    ts_admin_proxy_.reset(new tserver::TabletServerAdminServiceProxy(
+        messenger, addr, std::move(host)));
   }
   *proxy = ts_admin_proxy_;
   return Status::OK();
@@ -258,11 +261,13 @@ Status TSDescriptor::GetConsensusProxy(const shared_ptr<rpc::Messenger>& messeng
   }
 
   Sockaddr addr;
-  RETURN_NOT_OK(ResolveSockaddr(&addr));
+  string host;
+  RETURN_NOT_OK(ResolveSockaddr(&addr, &host));
 
   std::lock_guard<simple_spinlock> l(lock_);
   if (!consensus_proxy_) {
-    consensus_proxy_.reset(new consensus::ConsensusServiceProxy(messenger, addr));
+    consensus_proxy_.reset(new consensus::ConsensusServiceProxy(
+        messenger, addr, std::move(host)));
   }
   *proxy = consensus_proxy_;
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/master/ts_descriptor.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_descriptor.h b/src/kudu/master/ts_descriptor.h
index 701acd1..1e2b776 100644
--- a/src/kudu/master/ts_descriptor.h
+++ b/src/kudu/master/ts_descriptor.h
@@ -124,7 +124,9 @@ class TSDescriptor {
   explicit TSDescriptor(std::string perm_id);
 
   // Uses DNS to resolve registered hosts to a single Sockaddr.
-  Status ResolveSockaddr(Sockaddr* addr) const;
+  // Returns the resolved address as well as the hostname associated with it
+  // in 'addr' and 'host'.
+  Status ResolveSockaddr(Sockaddr* addr, std::string* host) const;
 
   void DecayRecentReplicaCreationsUnlocked();
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/connection.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.h b/src/kudu/rpc/connection.h
index e475cc5..2cc1292 100644
--- a/src/kudu/rpc/connection.h
+++ b/src/kudu/rpc/connection.h
@@ -123,15 +123,17 @@ class Connection : public RefCountedThreadSafe<Connection> {
   const Sockaddr &remote() const { return remote_; }
 
   // Set the user credentials for an outbound connection.
-  void set_local_user_credentials(UserCredentials creds) {
+  void set_outbound_connection_id(ConnectionId conn_id) {
     DCHECK_EQ(direction_, CLIENT);
-    local_user_credentials_ = std::move(creds);
+    DCHECK(!outbound_connection_id_);
+    outbound_connection_id_ = std::move(conn_id);
   }
 
   // Get the user credentials which will be used to log in.
-  const UserCredentials& local_user_credentials() const {
+  const ConnectionId& outbound_connection_id() const {
     DCHECK_EQ(direction_, CLIENT);
-    return local_user_credentials_;
+    DCHECK(outbound_connection_id_);
+    return *outbound_connection_id_;
   }
 
   RpczStore* rpcz_store();
@@ -247,8 +249,9 @@ class Connection : public RefCountedThreadSafe<Connection> {
   // The socket we're communicating on.
   std::unique_ptr<Socket> socket_;
 
-  // The credentials of the user operating on this connection (if a client user).
-  UserCredentials local_user_credentials_;
+  // The ConnectionId that serves as a key into the client connection map
+  // within this reactor. Only set in the case of outbound connections.
+  boost::optional<ConnectionId> outbound_connection_id_;
 
   // The authenticated remote user (if this is an inbound connection on the server).
   RemoteUser remote_user_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/connection_id.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection_id.cc b/src/kudu/rpc/connection_id.cc
index e4a4dba..d5bb0ba 100644
--- a/src/kudu/rpc/connection_id.cc
+++ b/src/kudu/rpc/connection_id.cc
@@ -20,6 +20,7 @@
 #include <boost/functional/hash.hpp>
 
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
 
 using std::string;
 
@@ -28,13 +29,13 @@ namespace rpc {
 
 ConnectionId::ConnectionId() {}
 
-ConnectionId::ConnectionId(const ConnectionId& other) {
-  DoCopyFrom(other);
-}
-
-ConnectionId::ConnectionId(const Sockaddr& remote, UserCredentials user_credentials) {
-  remote_ = remote;
-  user_credentials_ = std::move(user_credentials);
+ConnectionId::ConnectionId(const Sockaddr& remote,
+                           std::string hostname,
+                           UserCredentials user_credentials)
+    : remote_(remote),
+      hostname_(std::move(hostname)),
+      user_credentials_(std::move(user_credentials)) {
+  CHECK(!hostname_.empty());
 }
 
 void ConnectionId::set_remote(const Sockaddr& remote) {
@@ -45,32 +46,31 @@ void ConnectionId::set_user_credentials(UserCredentials user_credentials) {
   user_credentials_ = std::move(user_credentials);
 }
 
-void ConnectionId::CopyFrom(const ConnectionId& other) {
-  DoCopyFrom(other);
-}
-
 string ConnectionId::ToString() const {
-  // Does not print the password.
-  return StringPrintf("{remote=%s, user_credentials=%s}",
-      remote_.ToString().c_str(),
-      user_credentials_.ToString().c_str());
-}
-
-void ConnectionId::DoCopyFrom(const ConnectionId& other) {
-  remote_ = other.remote_;
-  user_credentials_ = other.user_credentials_;
+  string remote;
+  if (hostname_ != remote_.host()) {
+    remote = strings::Substitute("$0 ($1)", remote_.ToString(), hostname_);
+  } else {
+    remote = remote_.ToString();
+  }
+
+  return strings::Substitute("{remote=$0, user_credentials=$1}",
+                             remote,
+                             user_credentials_.ToString());
 }
 
 size_t ConnectionId::HashCode() const {
   size_t seed = 0;
   boost::hash_combine(seed, remote_.HashCode());
+  boost::hash_combine(seed, hostname_);
   boost::hash_combine(seed, user_credentials_.HashCode());
   return seed;
 }
 
 bool ConnectionId::Equals(const ConnectionId& other) const {
-  return (remote() == other.remote()
-       && user_credentials().Equals(other.user_credentials()));
+  return remote() == other.remote() &&
+      hostname_ == other.hostname_ &&
+      user_credentials().Equals(other.user_credentials());
 }
 
 size_t ConnectionIdHash::operator() (const ConnectionId& conn_id) const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/connection_id.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection_id.h b/src/kudu/rpc/connection_id.h
index ae34b29..d954a2d 100644
--- a/src/kudu/rpc/connection_id.h
+++ b/src/kudu/rpc/connection_id.h
@@ -32,23 +32,24 @@ class ConnectionId {
   ConnectionId();
 
   // Copy constructor required for use with STL unordered_map.
-  ConnectionId(const ConnectionId& other);
+  ConnectionId(const ConnectionId& other) = default;
 
   // Convenience constructor.
-  ConnectionId(const Sockaddr& remote, UserCredentials user_credentials);
+  ConnectionId(const Sockaddr& remote,
+               std::string hostname,
+               UserCredentials user_credentials);
 
   // The remote address.
   void set_remote(const Sockaddr& remote);
   const Sockaddr& remote() const { return remote_; }
 
+  const std::string& hostname() const { return hostname_; }
+
   // The credentials of the user associated with this connection, if any.
   void set_user_credentials(UserCredentials user_credentials);
   const UserCredentials& user_credentials() const { return user_credentials_; }
   UserCredentials* mutable_user_credentials() { return &user_credentials_; }
 
-  // Copy state from another object to this one.
-  void CopyFrom(const ConnectionId& other);
-
   // Returns a string representation of the object, not including the password field.
   std::string ToString() const;
 
@@ -58,13 +59,12 @@ class ConnectionId {
  private:
   // Remember to update HashCode() and Equals() when new fields are added.
   Sockaddr remote_;
-  UserCredentials user_credentials_;
 
-  // Implementation of CopyFrom that can be shared with copy constructor.
-  void DoCopyFrom(const ConnectionId& other);
+  // The original host name before it was resolved to 'remote_'.
+  // This must be retained since it is used to compute Kerberos Service Principal Names (SPNs).
+  std::string hostname_;
 
-  // Disable assignment operator.
-  void operator=(const ConnectionId&);
+  UserCredentials user_credentials_;
 };
 
 class ConnectionIdHash {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/exactly_once_rpc-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/exactly_once_rpc-test.cc b/src/kudu/rpc/exactly_once_rpc-test.cc
index 6941f38..d7c9fbd 100644
--- a/src/kudu/rpc/exactly_once_rpc-test.cc
+++ b/src/kudu/rpc/exactly_once_rpc-test.cc
@@ -159,7 +159,8 @@ class ExactlyOnceRpcTest : public RpcTestBase {
     // Set up server.
     StartTestServerWithGeneratedCode(&server_addr_);
     client_messenger_ = CreateMessenger("Client");
-    proxy_.reset(new CalculatorServiceProxy(client_messenger_, server_addr_));
+    proxy_.reset(new CalculatorServiceProxy(
+        client_messenger_, server_addr_, server_addr_.host()));
     test_picker_.reset(new TestServerPicker(proxy_.get()));
     request_tracker_.reset(new RequestTracker(kClientId));
     attempt_nos_ = 0;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/mt-rpc-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/mt-rpc-test.cc b/src/kudu/rpc/mt-rpc-test.cc
index 73e3a13..10056ae 100644
--- a/src/kudu/rpc/mt-rpc-test.cc
+++ b/src/kudu/rpc/mt-rpc-test.cc
@@ -45,7 +45,8 @@ class MultiThreadedRpcTest : public RpcTestBase {
                   Status* result, CountDownLatch* latch) {
     LOG(INFO) << "Connecting to " << server_addr.ToString();
     shared_ptr<Messenger> client_messenger(CreateMessenger("ClientSC"));
-    Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+    Proxy p(client_messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
     *result = DoTestSyncCall(p, method_name);
     latch->CountDown();
   }
@@ -61,7 +62,8 @@ class MultiThreadedRpcTest : public RpcTestBase {
       Sockaddr server_addr, const char* method_name, Status* last_result,
       const shared_ptr<Messenger>& messenger) {
     LOG(INFO) << "Connecting to " << server_addr.ToString();
-    Proxy p(messenger, server_addr, GenericCalculatorService::static_service_name());
+    Proxy p(messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
 
     int i = 0;
     while (true) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index db742ca..b0ade7c 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -163,11 +163,7 @@ static Status DoClientNegotiation(Connection* conn,
                                        messenger->authn_token(),
                                        encryption);
 
-  // Note that the fqdn is an IP address here: we've already lost whatever DNS
-  // name the client was attempting to use. Unless krb5 is configured with 'rdns
-  // = false', it will automatically take care of reversing this address to its
-  // canonical hostname to determine the expected server principal.
-  client_negotiation.set_server_fqdn(conn->remote().host());
+  client_negotiation.set_server_fqdn(conn->outbound_connection_id().hostname());
 
   if (authentication != RpcAuthentication::DISABLED) {
     Status s = client_negotiation.EnableGSSAPI();
@@ -196,7 +192,8 @@ static Status DoClientNegotiation(Connection* conn,
   }
 
   if (authentication != RpcAuthentication::REQUIRED) {
-    RETURN_NOT_OK(client_negotiation.EnablePlain(conn->local_user_credentials().real_user(), ""));
+    const auto& creds = conn->outbound_connection_id().user_credentials();
+    RETURN_NOT_OK(client_negotiation.EnablePlain(creds.real_user(), ""));
   }
 
   client_negotiation.set_deadline(deadline);

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/protoc-gen-krpc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/protoc-gen-krpc.cc b/src/kudu/rpc/protoc-gen-krpc.cc
index de41aa9..950ae46 100644
--- a/src/kudu/rpc/protoc-gen-krpc.cc
+++ b/src/kudu/rpc/protoc-gen-krpc.cc
@@ -576,8 +576,9 @@ class CodeGenerator : public ::google::protobuf::compiler::CodeGenerator {
       Print(printer, *subs,
         "class $service_name$Proxy : public ::kudu::rpc::Proxy {\n"
         " public:\n"
-        "  $service_name$Proxy(const std::shared_ptr< ::kudu::rpc::Messenger>\n"
-        "                &messenger, const ::kudu::Sockaddr &sockaddr);\n"
+        "  $service_name$Proxy(std::shared_ptr<::kudu::rpc::Messenger>\n"
+        "                messenger, const ::kudu::Sockaddr &sockaddr,"
+        "                std::string hostname);\n"
         "  ~$service_name$Proxy();\n"
         "\n"
         );
@@ -631,9 +632,9 @@ class CodeGenerator : public ::google::protobuf::compiler::CodeGenerator {
       subs->PushService(service);
       Print(printer, *subs,
         "$service_name$Proxy::$service_name$Proxy(\n"
-        "   const std::shared_ptr< ::kudu::rpc::Messenger> &messenger,\n"
-        "   const ::kudu::Sockaddr &remote)\n"
-        "  : Proxy(messenger, remote, \"$full_service_name$\") {\n"
+        "   std::shared_ptr< ::kudu::rpc::Messenger> messenger,\n"
+        "   const ::kudu::Sockaddr &remote, std::string hostname)\n"
+        "  : Proxy(std::move(messenger), remote, std::move(hostname), \"$full_service_name$\") {\n"
         "}\n"
         "\n"
         "$service_name$Proxy::~$service_name$Proxy() {\n"

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/proxy.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/proxy.cc b/src/kudu/rpc/proxy.cc
index 206aac3..9189f84 100644
--- a/src/kudu/rpc/proxy.cc
+++ b/src/kudu/rpc/proxy.cc
@@ -48,7 +48,9 @@ namespace kudu {
 namespace rpc {
 
 Proxy::Proxy(std::shared_ptr<Messenger> messenger,
-             const Sockaddr& remote, string service_name)
+             const Sockaddr& remote,
+             string hostname,
+             string service_name)
     : service_name_(std::move(service_name)),
       messenger_(std::move(messenger)),
       is_started_(false) {
@@ -64,8 +66,9 @@ Proxy::Proxy(std::shared_ptr<Messenger> messenger,
         << s.ToString() << " before connecting to remote: " << remote.ToString();
   }
 
-  conn_id_.set_remote(remote);
-  conn_id_.mutable_user_credentials()->set_real_user(real_user);
+  UserCredentials creds;
+  creds.set_real_user(std::move(real_user));
+  conn_id_ = ConnectionId(remote, std::move(hostname), std::move(creds));
 }
 
 Proxy::~Proxy() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/proxy.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/proxy.h b/src/kudu/rpc/proxy.h
index 6e43e93..a3d852e 100644
--- a/src/kudu/rpc/proxy.h
+++ b/src/kudu/rpc/proxy.h
@@ -55,8 +55,11 @@ class Messenger;
 // After initialization, multiple threads may make calls using the same proxy object.
 class Proxy {
  public:
-  Proxy(std::shared_ptr<Messenger> messenger, const Sockaddr& remote,
+  Proxy(std::shared_ptr<Messenger> messenger,
+        const Sockaddr& remote,
+        std::string hostname,
         std::string service_name);
+
   ~Proxy();
 
   // Call a remote method asynchronously.

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/reactor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/reactor.cc b/src/kudu/rpc/reactor.cc
index 87056d1..520d232 100644
--- a/src/kudu/rpc/reactor.cc
+++ b/src/kudu/rpc/reactor.cc
@@ -343,8 +343,9 @@ Status ReactorThread::FindOrStartConnection(const ConnectionId& conn_id,
   std::unique_ptr<Socket> new_socket(new Socket(sock.Release()));
 
   // Register the new connection in our map.
-  *conn = new Connection(this, conn_id.remote(), std::move(new_socket), Connection::CLIENT);
-  (*conn)->set_local_user_credentials(conn_id.user_credentials());
+  *conn = new Connection(
+      this, conn_id.remote(), std::move(new_socket), Connection::CLIENT);
+  (*conn)->set_outbound_connection_id(conn_id);
 
   // Kick off blocking client connection negotiation.
   Status s = StartConnectionNegotiation(*conn);
@@ -437,10 +438,16 @@ void ReactorThread::DestroyConnection(Connection *conn,
 
   // Unlink connection from lists.
   if (conn->direction() == Connection::CLIENT) {
-    ConnectionId conn_id(conn->remote(), conn->local_user_credentials());
-    auto it = client_conns_.find(conn_id);
-    CHECK(it != client_conns_.end()) << "Couldn't find connection " << conn->ToString();
-    client_conns_.erase(it);
+    const auto range = client_conns_.equal_range(conn->outbound_connection_id());
+    CHECK(range.first != range.second) << "Couldn't find connection " << conn->ToString();
+    // The client_conns_ container is a multi-map.
+    for (auto it = range.first; it != range.second;) {
+      if (it->second.get() == conn) {
+        it = client_conns_.erase(it);
+        break;
+      }
+      ++it;
+    }
   } else if (conn->direction() == Connection::SERVER) {
     auto it = server_conns_.begin();
     while (it != server_conns_.end()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/rpc-bench.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc-bench.cc b/src/kudu/rpc/rpc-bench.cc
index 809697a..224d3f7 100644
--- a/src/kudu/rpc/rpc-bench.cc
+++ b/src/kudu/rpc/rpc-bench.cc
@@ -124,7 +124,7 @@ class ClientThread {
   void Run() {
     shared_ptr<Messenger> client_messenger = bench_->CreateMessenger("Client");
 
-    CalculatorServiceProxy p(client_messenger, bench_->server_addr_);
+    CalculatorServiceProxy p(client_messenger, bench_->server_addr_, "localhost");
 
     AddRequestPB req;
     AddResponsePB resp;
@@ -177,7 +177,7 @@ class ClientAsyncWorkload {
       messenger_(std::move(messenger)),
       request_count_(0) {
     controller_.set_timeout(MonoDelta::FromSeconds(10));
-    proxy_.reset(new CalculatorServiceProxy(messenger_, bench_->server_addr_));
+    proxy_.reset(new CalculatorServiceProxy(messenger_, bench_->server_addr_, "localhost"));
   }
 
   void CallOneRpc() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/rpc-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc-test.cc b/src/kudu/rpc/rpc-test.cc
index e18d07c..8f9df0e 100644
--- a/src/kudu/rpc/rpc-test.cc
+++ b/src/kudu/rpc/rpc-test.cc
@@ -112,7 +112,8 @@ TEST_P(TestRpc, TestCall) {
   // Set up client.
   LOG(INFO) << "Connecting to " << server_addr.ToString();
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
   ASSERT_STR_CONTAINS(p.ToString(), strings::Substitute("kudu.rpc.GenericCalculatorService@"
                                                             "{remote=$0, user_credentials=",
                                                         server_addr.ToString()));
@@ -127,7 +128,8 @@ TEST_P(TestRpc, TestCallToBadServer) {
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, GetParam()));
   Sockaddr addr;
   addr.set_port(0);
-  Proxy p(client_messenger, addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, addr, addr.host(),
+          GenericCalculatorService::static_service_name());
 
   // Loop a few calls to make sure that we properly set up and tear down
   // the connections.
@@ -148,7 +150,8 @@ TEST_P(TestRpc, TestInvalidMethodCall) {
   // Set up client.
   LOG(INFO) << "Connecting to " << server_addr.ToString();
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   // Call the method which fails.
   Status s = DoTestSyncCall(p, "ThisMethodDoesNotExist");
@@ -166,7 +169,7 @@ TEST_P(TestRpc, TestWrongService) {
 
   // Set up client with the wrong service name.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, "WrongServiceName");
+  Proxy p(client_messenger, server_addr, "localhost", "WrongServiceName");
 
   // Call the method which fails.
   Status s = DoTestSyncCall(p, "ThisMethodDoesNotExist");
@@ -201,7 +204,8 @@ TEST_P(TestRpc, TestHighFDs) {
   bool enable_ssl = GetParam();
   StartTestServer(&server_addr, enable_ssl);
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
   ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
 }
 
@@ -220,7 +224,8 @@ TEST_P(TestRpc, TestConnectionKeepalive) {
   // Set up client.
   LOG(INFO) << "Connecting to " << server_addr.ToString();
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
 
@@ -262,7 +267,8 @@ TEST_P(TestRpc, TestCallLongerThanKeepalive) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   // Make a call which sleeps longer than the keepalive.
   RpcController controller;
@@ -283,7 +289,8 @@ TEST_P(TestRpc, TestRpcSidecar) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, GetParam()));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   // Test a zero-length sidecar
   DoTestSidecar(p, 0, 0);
@@ -302,7 +309,8 @@ TEST_P(TestRpc, TestCallTimeout) {
   bool enable_ssl = GetParam();
   StartTestServer(&server_addr, enable_ssl);
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   // Test a very short timeout - we expect this will time out while the
   // call is still trying to connect, or in the send queue. This was triggering ASAN failures
@@ -330,7 +338,8 @@ TEST_P(TestRpc, TestCallTimeoutDoesntAffectNegotiation) {
   bool enable_ssl = GetParam();
   StartTestServer(&server_addr, enable_ssl);
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   FLAGS_rpc_negotiation_inject_delay_ms = 500;
   ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(p, MonoDelta::FromMilliseconds(50)));
@@ -373,7 +382,8 @@ TEST_F(TestRpc, TestNegotiationTimeout) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client"));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(p, MonoDelta::FromMilliseconds(100)));
 
@@ -391,7 +401,8 @@ TEST_F(TestRpc, TestServerShutsDown) {
   // Set up client.
   LOG(INFO) << "Connecting to " << server_addr.ToString();
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client"));
-  Proxy p(client_messenger, server_addr, GenericCalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
 
   // Send a call.
   AddRequestPB req;
@@ -475,7 +486,8 @@ TEST_P(TestRpc, TestRpcHandlerLatencyMetric) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, CalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
 
   RpcController controller;
   SleepRequestPB req;
@@ -523,7 +535,7 @@ TEST_P(TestRpc, TestRpcCallbackDestroysMessenger) {
   RpcController controller;
   controller.set_timeout(MonoDelta::FromMilliseconds(1));
   {
-    Proxy p(client_messenger, bad_addr, "xxx");
+    Proxy p(client_messenger, bad_addr, "xxx-host", "xxx-service");
     p.AsyncRequest("my-fake-method", req, &resp, &controller,
                    boost::bind(&DestroyMessengerCallback, &client_messenger, &latch));
   }
@@ -542,7 +554,8 @@ TEST_P(TestRpc, TestRpcContextClientDeadline) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, CalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
 
   SleepRequestPB req;
   req.set_sleep_micros(sleep_micros);
@@ -568,7 +581,8 @@ TEST_P(TestRpc, TestApplicationFeatureFlag) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, CalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
 
   { // Supported flag
     AddRequestPB req;
@@ -609,7 +623,8 @@ TEST_P(TestRpc, TestApplicationFeatureFlagUnsupportedServer) {
 
   // Set up client.
   shared_ptr<Messenger> client_messenger(CreateMessenger("Client", 1, enable_ssl));
-  Proxy p(client_messenger, server_addr, CalculatorService::static_service_name());
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
 
   { // Required flag
     AddRequestPB req;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/rpc/rpc_stub-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_stub-test.cc b/src/kudu/rpc/rpc_stub-test.cc
index 645dde3..3c20a35 100644
--- a/src/kudu/rpc/rpc_stub-test.cc
+++ b/src/kudu/rpc/rpc_stub-test.cc
@@ -60,7 +60,7 @@ class RpcStubTest : public RpcTestBase {
   }
  protected:
   void SendSimpleCall() {
-    CalculatorServiceProxy p(client_messenger_, server_addr_);
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
     RpcController controller;
     AddRequestPB req;
@@ -84,7 +84,7 @@ TEST_F(RpcStubTest, TestSimpleCall) {
 // reads and then makes a number of calls.
 TEST_F(RpcStubTest, TestShortRecvs) {
   FLAGS_socket_inject_short_recvs = true;
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   for (int i = 0; i < 100; i++) {
     NO_FATALS(SendSimpleCall());
@@ -102,7 +102,7 @@ TEST_F(RpcStubTest, TestBigCallData) {
   string data;
   data.resize(kMessageSize);
 
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   EchoRequestPB req;
   req.set_data(data);
@@ -127,7 +127,7 @@ TEST_F(RpcStubTest, TestBigCallData) {
 }
 
 TEST_F(RpcStubTest, TestRespondDeferred) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   RpcController controller;
   SleepRequestPB req;
@@ -139,7 +139,7 @@ TEST_F(RpcStubTest, TestRespondDeferred) {
 
 // Test that the default user credentials are propagated to the server.
 TEST_F(RpcStubTest, TestDefaultCredentialsPropagated) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   string expected;
   ASSERT_OK(GetLoggedInUser(&expected));
@@ -155,7 +155,7 @@ TEST_F(RpcStubTest, TestDefaultCredentialsPropagated) {
 // Test that the user can specify other credentials.
 TEST_F(RpcStubTest, TestCustomCredentialsPropagated) {
   const char* const kFakeUserName = "some fake user";
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   UserCredentials creds;
   creds.set_real_user(kFakeUserName);
@@ -172,7 +172,7 @@ TEST_F(RpcStubTest, TestCustomCredentialsPropagated) {
 TEST_F(RpcStubTest, TestAuthorization) {
   // First test calling WhoAmI() as user "alice", who is disallowed.
   {
-    CalculatorServiceProxy p(client_messenger_, server_addr_);
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
     UserCredentials creds;
     creds.set_real_user("alice");
     p.set_user_credentials(creds);
@@ -189,7 +189,7 @@ TEST_F(RpcStubTest, TestAuthorization) {
 
   // Try some calls as "bob".
   {
-    CalculatorServiceProxy p(client_messenger_, server_addr_);
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
     UserCredentials creds;
     creds.set_real_user("bob");
     p.set_user_credentials(creds);
@@ -217,7 +217,7 @@ TEST_F(RpcStubTest, TestAuthorization) {
 
 // Test that the user's remote address is accessible to the server.
 TEST_F(RpcStubTest, TestRemoteAddress) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   RpcController controller;
   WhoAmIRequestPB req;
@@ -233,7 +233,8 @@ TEST_F(RpcStubTest, TestRemoteAddress) {
 // Test sending a PB parameter with a missing field, where the client
 // thinks it has sent a full PB. (eg due to version mismatch)
 TEST_F(RpcStubTest, TestCallWithInvalidParam) {
-  Proxy p(client_messenger_, server_addr_, CalculatorService::static_service_name());
+  Proxy p(client_messenger_, server_addr_, server_addr_.host(),
+          CalculatorService::static_service_name());
 
   AddRequestPartialPB req;
   req.set_x(rand());
@@ -258,7 +259,7 @@ static void DoIncrement(Atomic32* count) {
 // This also ensures that the async callback is only called once
 // (regression test for a previously-encountered bug).
 TEST_F(RpcStubTest, TestCallWithMissingPBFieldClientSide) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   RpcController controller;
   AddRequestPB req;
@@ -278,7 +279,7 @@ TEST_F(RpcStubTest, TestCallWithMissingPBFieldClientSide) {
 }
 
 TEST_F(RpcStubTest, TestResponseWithMissingField) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   RpcController rpc;
   TestInvalidResponseRequestPB req;
@@ -293,7 +294,7 @@ TEST_F(RpcStubTest, TestResponseWithMissingField) {
 // configured RPC message size. The server should send the response, but the client
 // will reject it.
 TEST_F(RpcStubTest, TestResponseLargerThanFrameSize) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   RpcController rpc;
   TestInvalidResponseRequestPB req;
@@ -305,7 +306,8 @@ TEST_F(RpcStubTest, TestResponseLargerThanFrameSize) {
 
 // Test sending a call which isn't implemented by the server.
 TEST_F(RpcStubTest, TestCallMissingMethod) {
-  Proxy p(client_messenger_, server_addr_, CalculatorService::static_service_name());
+  Proxy p(client_messenger_, server_addr_, server_addr_.host(),
+          CalculatorService::static_service_name());
 
   Status s = DoTestSyncCall(p, "DoesNotExist");
   ASSERT_TRUE(s.IsRemoteError()) << "Bad status: " << s.ToString();
@@ -313,7 +315,7 @@ TEST_F(RpcStubTest, TestCallMissingMethod) {
 }
 
 TEST_F(RpcStubTest, TestApplicationError) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   RpcController controller;
   SleepRequestPB req;
@@ -378,7 +380,7 @@ TEST_F(RpcStubTest, TestRpcPanic) {
     CHECK_OK(env_->DeleteRecursively(test_dir_));
 
     // Make an RPC which causes the server to abort.
-    CalculatorServiceProxy p(client_messenger_, server_addr_);
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
     RpcController controller;
     PanicRequestPB req;
     PanicResponsePB resp;
@@ -396,7 +398,7 @@ struct AsyncSleep {
 };
 
 TEST_F(RpcStubTest, TestDontHandleTimedOutCalls) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
   vector<AsyncSleep*> sleeps;
   ElementDeleter d(&sleeps);
 
@@ -460,7 +462,8 @@ TEST_F(RpcStubTest, TestEarliestDeadlineFirstQueue) {
   for (int thread_id = 0; thread_id < num_client_threads; thread_id++) {
     threads.emplace_back([&, thread_id] {
         Random rng(thread_id);
-        CalculatorServiceProxy p(client_messenger_, server_addr_);
+        CalculatorServiceProxy p(
+            client_messenger_, server_addr_, server_addr_.host());
         while (!done.load()) {
           // Set a deadline in the future. We'll keep using this same deadline
           // on each of our retries.
@@ -518,7 +521,7 @@ TEST_F(RpcStubTest, TestEarliestDeadlineFirstQueue) {
 }
 
 TEST_F(RpcStubTest, TestDumpCallsInFlight) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
   AsyncSleep sleep;
   sleep.req.set_sleep_micros(100 * 1000); // 100ms
   p.SleepAsync(sleep.req, &sleep.resp, &sleep.rpc,
@@ -562,7 +565,7 @@ TEST_F(RpcStubTest, TestDumpCallsInFlight) {
 }
 
 TEST_F(RpcStubTest, TestDumpSampledCalls) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   // Issue two calls that fall into different latency buckets.
   AsyncSleep sleeps[2];
@@ -619,7 +622,7 @@ void MyTestCallback(CountDownLatch* latch, scoped_refptr<RefCountedTest> my_refp
 // is held. This is important when the callback holds a refcounted ptr,
 // since we expect to be able to release that pointer when the call is done.
 TEST_F(RpcStubTest, TestCallbackClearedAfterRunning) {
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   CountDownLatch latch(1);
   scoped_refptr<RefCountedTest> my_refptr(new RefCountedTest);
@@ -648,7 +651,7 @@ TEST_F(RpcStubTest, DontTimeOutWhenReactorIsBlocked) {
       << "This test requires only a single reactor. Otherwise the injected sleep might "
       << "be scheduled on a different reactor than the RPC call.";
 
-  CalculatorServiceProxy p(client_messenger_, server_addr_);
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
 
   // Schedule a 1-second sleep on the reactor thread.
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tools/ksck_remote.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/ksck_remote.cc b/src/kudu/tools/ksck_remote.cc
index a17c5ea..2ce7388 100644
--- a/src/kudu/tools/ksck_remote.cc
+++ b/src/kudu/tools/ksck_remote.cc
@@ -59,8 +59,10 @@ Status RemoteKsckTabletServer::Init() {
   RETURN_NOT_OK(ParseAddressList(
       host_port_.ToString(),
       tserver::TabletServer::kDefaultPort, &addresses));
-  generic_proxy_.reset(new server::GenericServiceProxy(messenger_, addresses[0]));
-  ts_proxy_.reset(new tserver::TabletServerServiceProxy(messenger_, addresses[0]));
+  const auto& addr = addresses[0];
+  const auto& host = host_port_.host();
+  generic_proxy_.reset(new server::GenericServiceProxy(messenger_, addr, host));
+  ts_proxy_.reset(new tserver::TabletServerServiceProxy(messenger_, addr, host));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tools/tool_action_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_common.cc b/src/kudu/tools/tool_action_common.cc
index 5f12072..4b04e81 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -219,7 +219,7 @@ Status BuildProxy(const string& address,
   vector<Sockaddr> resolved;
   RETURN_NOT_OK(hp.ResolveAddresses(&resolved));
 
-  proxy->reset(new ProxyClass(messenger, resolved[0]));
+  proxy->reset(new ProxyClass(messenger, resolved[0], hp.host()));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tserver/heartbeater.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/heartbeater.cc b/src/kudu/tserver/heartbeater.cc
index 093399e..46d7aa7 100644
--- a/src/kudu/tserver/heartbeater.cc
+++ b/src/kudu/tserver/heartbeater.cc
@@ -81,7 +81,7 @@ Status MasterServiceProxyForHostPort(const HostPort& hostport,
                  << "resolves to " << addrs.size() << " different addresses. Using "
                  << addrs[0].ToString();
   }
-  proxy->reset(new MasterServiceProxy(messenger, addrs[0]));
+  proxy->reset(new MasterServiceProxy(messenger, addrs[0], hostport.host()));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tserver/tablet_copy_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client.cc b/src/kudu/tserver/tablet_copy_client.cc
index e0e0e97..269e326 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -165,7 +165,7 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
                         << " from remote peer at address " << copy_source_addr.ToString();
 
   // Set up an RPC proxy for the TabletCopyService.
-  proxy_.reset(new TabletCopyServiceProxy(messenger_, addr));
+  proxy_.reset(new TabletCopyServiceProxy(messenger_, addr, copy_source_addr.host()));
 
   BeginTabletCopySessionRequestPB req;
   req.set_requestor_uuid(fs_manager_->uuid());

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tserver/tablet_copy_service-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service-test.cc b/src/kudu/tserver/tablet_copy_service-test.cc
index fc88155..8f59a84 100644
--- a/src/kudu/tserver/tablet_copy_service-test.cc
+++ b/src/kudu/tserver/tablet_copy_service-test.cc
@@ -68,7 +68,9 @@ class TabletCopyServiceTest : public TabletCopyTest {
   void SetUp() OVERRIDE {
     TabletCopyTest::SetUp();
     tablet_copy_proxy_.reset(
-        new TabletCopyServiceProxy(client_messenger_, mini_server_->bound_rpc_addr()));
+        new TabletCopyServiceProxy(
+            client_messenger_, mini_server_->bound_rpc_addr(),
+            mini_server_->bound_rpc_addr().host()));
   }
 
   Status DoBeginTabletCopySession(const string& tablet_id,

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tserver/tablet_server-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index 971a76c..ecf79f2 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -102,7 +102,8 @@ TEST_F(TabletServerTest, TestServerClock) {
 
 TEST_F(TabletServerTest, TestSetFlags) {
   server::GenericServiceProxy proxy(
-      client_messenger_, mini_server_->bound_rpc_addr());
+      client_messenger_, mini_server_->bound_rpc_addr(),
+      mini_server_->bound_rpc_addr().host());
 
   server::SetFlagRequestPB req;
   server::SetFlagResponsePB resp;

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/tserver/tablet_server_test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server_test_util.cc b/src/kudu/tserver/tablet_server_test_util.cc
index 5caf507..796ad07 100644
--- a/src/kudu/tserver/tablet_server_test_util.cc
+++ b/src/kudu/tserver/tablet_server_test_util.cc
@@ -17,6 +17,8 @@
 
 #include "kudu/tserver/tablet_server_test_util.h"
 
+#include <string>
+
 #include "kudu/consensus/consensus.proxy.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/server/server_base.proxy.h"
@@ -36,10 +38,11 @@ void CreateTsClientProxies(const Sockaddr& addr,
                            gscoped_ptr<TabletServerAdminServiceProxy>* admin_proxy,
                            gscoped_ptr<ConsensusServiceProxy>* consensus_proxy,
                            gscoped_ptr<server::GenericServiceProxy>* generic_proxy) {
-  proxy->reset(new TabletServerServiceProxy(messenger, addr));
-  admin_proxy->reset(new TabletServerAdminServiceProxy(messenger, addr));
-  consensus_proxy->reset(new ConsensusServiceProxy(messenger, addr));
-  generic_proxy->reset(new server::GenericServiceProxy(messenger, addr));
+  const auto& host = addr.host();
+  proxy->reset(new TabletServerServiceProxy(messenger, addr, host));
+  admin_proxy->reset(new TabletServerAdminServiceProxy(messenger, addr, host));
+  consensus_proxy->reset(new ConsensusServiceProxy(messenger, addr, host));
+  generic_proxy->reset(new server::GenericServiceProxy(messenger, addr, host));
 }
 
 } // namespace tserver

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/util/net/net_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/net_util-test.cc b/src/kudu/util/net/net_util-test.cc
index b1c33ef..2058333 100644
--- a/src/kudu/util/net/net_util-test.cc
+++ b/src/kudu/util/net/net_util-test.cc
@@ -53,6 +53,7 @@ TEST(SockaddrTest, Test) {
   Sockaddr addr;
   ASSERT_OK(addr.ParseString("1.1.1.1:12345", 12345));
   ASSERT_EQ(12345, addr.port());
+  ASSERT_EQ("1.1.1.1", addr.host());
 }
 
 TEST_F(NetUtilTest, TestParseAddresses) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/util/net/sockaddr.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/sockaddr.cc b/src/kudu/util/net/sockaddr.cc
index ed249c7..a462c05 100644
--- a/src/kudu/util/net/sockaddr.cc
+++ b/src/kudu/util/net/sockaddr.cc
@@ -98,9 +98,7 @@ const struct sockaddr_in& Sockaddr::addr() const {
 }
 
 std::string Sockaddr::ToString() const {
-  char str[INET_ADDRSTRLEN];
-  ::inet_ntop(AF_INET, &addr_.sin_addr, str, INET_ADDRSTRLEN);
-  return StringPrintf("%s:%d", str, port());
+  return Substitute("$0:$1", host(), port());
 }
 
 bool Sockaddr::IsWildcard() const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/de204012/src/kudu/util/net/sockaddr.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/sockaddr.h b/src/kudu/util/net/sockaddr.h
index 09777f3..01506e7 100644
--- a/src/kudu/util/net/sockaddr.h
+++ b/src/kudu/util/net/sockaddr.h
@@ -54,11 +54,14 @@ class Sockaddr {
 
   uint32_t HashCode() const;
 
+  // Returns the dotted-decimal string '1.2.3.4' of the host component of this address.
   std::string host() const;
 
   void set_port(int port);
   int port() const;
   const struct sockaddr_in& addr() const;
+
+  // Returns the stringified address in '1.2.3.4:<port>' format.
   std::string ToString() const;
 
   // Returns true if the address is 0.0.0.0


[2/2] kudu git commit: KUDU-2032 (part 2): propagate master hostnames into client

Posted by to...@apache.org.
KUDU-2032 (part 2): propagate master hostnames into client

This changes the client code to remember the user-specified master
addresses and propagate them into the creation of master proxies.

It's not possible to reproduce the necessary DNS configurations in a
minicluster test, but with this patch I am now able to use 'kudu perf
loadgen' against a Kerberized cluster even when my local krb5.conf has
rdns=false.

Change-Id: I0c3e6c5f6543a86173e242b04d3515f5ec69200d
Reviewed-on: http://gerrit.cloudera.org:8080/8183
Tested-by: Kudu Jenkins
Reviewed-by: Will Berkeley <wd...@gmail.com>


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

Branch: refs/heads/branch-1.3.x
Commit: 1aa9695229341efbabcdd55dc3b107de1d05a265
Parents: de20401
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Aug 16 17:44:39 2017 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Thu Oct 5 05:34:19 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client-internal.cc              | 26 +++++----
 src/kudu/client/client-internal.h               |  3 +-
 src/kudu/client/master_rpc.cc                   | 60 ++++++++++++--------
 src/kudu/client/master_rpc.h                    | 12 ++--
 .../integration-tests/external_mini_cluster.cc  | 12 ++--
 5 files changed, 68 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1aa96952/src/kudu/client/client-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index 7e3f49b..6ab46a8 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -23,6 +23,7 @@
 #include <memory>
 #include <mutex>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include "kudu/client/master_rpc.h"
@@ -50,6 +51,7 @@
 #include "kudu/util/pb_util.h"
 #include "kudu/util/thread_restrictions.h"
 
+using std::pair;
 using std::set;
 using std::shared_ptr;
 using std::string;
@@ -591,8 +593,10 @@ Status KuduClient::Data::GetTableSchema(KuduClient* client,
 
 void KuduClient::Data::ConnectedToClusterCb(
     const Status& status,
-    const Sockaddr& leader_addr,
+    const pair<Sockaddr, string>& leader_addr_and_name,
     const master::ConnectToMasterResponsePB& connect_response) {
+  const auto& leader_addr = leader_addr_and_name.first;
+  const auto& leader_hostname = leader_addr_and_name.second;
 
   // Ensure that all of the CAs reported by the master are trusted
   // in our local TLS configuration.
@@ -628,9 +632,8 @@ void KuduClient::Data::ConnectedToClusterCb(
     leader_master_rpc_.reset();
 
     if (status.ok()) {
-      leader_master_hostport_ = HostPort(leader_addr);
-      // TODO(KUDU-2032): retain the original hostname passed by caller.
-      master_proxy_.reset(new MasterServiceProxy(messenger_, leader_addr, leader_addr.host()));
+      leader_master_hostport_ = HostPort(leader_hostname, leader_addr.port());
+      master_proxy_.reset(new MasterServiceProxy(messenger_, leader_addr, leader_hostname));
     }
   }
 
@@ -651,12 +654,15 @@ void KuduClient::Data::ConnectToClusterAsync(KuduClient* client,
                                              const StatusCallback& cb) {
   DCHECK(deadline.Initialized());
 
-  vector<Sockaddr> master_sockaddrs;
+  vector<pair<Sockaddr, string>> master_addrs_with_names;
   for (const string& master_server_addr : master_server_addrs_) {
     vector<Sockaddr> addrs;
-    Status s;
-    // TODO: Do address resolution asynchronously as well.
-    s = ParseAddressList(master_server_addr, master::Master::kDefaultPort, &addrs);
+    HostPort hp;
+    Status s = hp.ParseString(master_server_addr, master::Master::kDefaultPort);
+    if (s.ok()) {
+      // TODO(todd): Do address resolution asynchronously as well.
+      s = hp.ResolveAddresses(&addrs);
+    }
     if (!s.ok()) {
       cb.Run(s);
       return;
@@ -671,7 +677,7 @@ void KuduClient::Data::ConnectToClusterAsync(KuduClient* client,
           << "Specified master server address '" << master_server_addr << "' "
           << "resolved to multiple IPs. Using " << addrs[0].ToString();
     }
-    master_sockaddrs.push_back(addrs[0]);
+    master_addrs_with_names.emplace_back(addrs[0], hp.host());
   }
 
   // This ensures that no more than one ConnectToClusterRpc is in
@@ -688,7 +694,7 @@ void KuduClient::Data::ConnectToClusterAsync(KuduClient* client,
                   std::placeholders::_1,
                   std::placeholders::_2,
                   std::placeholders::_3),
-        std::move(master_sockaddrs),
+        std::move(master_addrs_with_names),
         deadline,
         client->default_rpc_timeout(),
         messenger_));

http://git-wip-us.apache.org/repos/asf/kudu/blob/1aa96952/src/kudu/client/client-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.h b/src/kudu/client/client-internal.h
index 864a734..83a861b 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -22,6 +22,7 @@
 #include <set>
 #include <string>
 #include <unordered_set>
+#include <utility>
 #include <vector>
 
 #include <boost/function.hpp>
@@ -140,7 +141,7 @@ class KuduClient::Data {
   //
   // See also: ConnectToClusterAsync.
   void ConnectedToClusterCb(const Status& status,
-                            const Sockaddr& leader_addr,
+                            const std::pair<Sockaddr, std::string>& leader_addr_and_name,
                             const master::ConnectToMasterResponsePB& connect_response);
 
   // Asynchronously sets 'master_proxy_' to the leader master by

http://git-wip-us.apache.org/repos/asf/kudu/blob/1aa96952/src/kudu/client/master_rpc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/master_rpc.cc b/src/kudu/client/master_rpc.cc
index 89b2979..2019bd7 100644
--- a/src/kudu/client/master_rpc.cc
+++ b/src/kudu/client/master_rpc.cc
@@ -19,8 +19,12 @@
 
 #include "kudu/client/master_rpc.h"
 
-#include <boost/bind.hpp>
+#include <algorithm>
 #include <mutex>
+#include <ostream>
+#include <utility>
+
+#include <boost/bind.hpp>
 
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
@@ -32,6 +36,7 @@
 #include "kudu/util/scoped_cleanup.h"
 
 
+using std::pair;
 using std::shared_ptr;
 using std::string;
 using std::vector;
@@ -45,6 +50,7 @@ using kudu::master::MasterErrorPB;
 using kudu::master::MasterServiceProxy;
 using kudu::rpc::Messenger;
 using kudu::rpc::Rpc;
+using strings::Substitute;
 
 namespace kudu {
 namespace client {
@@ -65,7 +71,7 @@ class ConnectToMasterRpc : public rpc::Rpc {
   //
   // Invokes 'user_cb' upon failure or success of the RPC call.
   ConnectToMasterRpc(StatusCallback user_cb,
-                     const Sockaddr& addr,
+                     pair<Sockaddr, string> addr_with_name,
                      const MonoTime& deadline,
                      std::shared_ptr<rpc::Messenger> messenger,
                      ConnectToMasterResponsePB* out);
@@ -80,7 +86,9 @@ class ConnectToMasterRpc : public rpc::Rpc {
   virtual void SendRpcCb(const Status& status) OVERRIDE;
 
   const StatusCallback user_cb_;
-  const Sockaddr addr_;
+
+  // The resolved address to try to connect to, along with its original specified hostname.
+  const pair<Sockaddr, string> addr_with_name_;
 
   // Owned by the caller of this RPC, not this instance.
   ConnectToMasterResponsePB* out_;
@@ -93,20 +101,22 @@ class ConnectToMasterRpc : public rpc::Rpc {
   bool trying_old_rpc_ = false;
 };
 
-
-ConnectToMasterRpc::ConnectToMasterRpc(
-    StatusCallback user_cb, const Sockaddr& addr,const MonoTime& deadline,
-    shared_ptr<Messenger> messenger, ConnectToMasterResponsePB* out)
-    : Rpc(deadline, std::move(messenger)),
-      user_cb_(std::move(user_cb)),
-      addr_(addr),
-      out_(DCHECK_NOTNULL(out)) {}
+ConnectToMasterRpc::ConnectToMasterRpc(StatusCallback user_cb,
+    pair<Sockaddr, string> addr_with_name,
+    const MonoTime& deadline,
+    shared_ptr<Messenger> messenger,
+    ConnectToMasterResponsePB* out)
+      : Rpc(deadline, std::move(messenger)),
+        user_cb_(std::move(user_cb)),
+        addr_with_name_(std::move(addr_with_name)),
+        out_(DCHECK_NOTNULL(out)) {
+}
 
 ConnectToMasterRpc::~ConnectToMasterRpc() {
 }
 
 void ConnectToMasterRpc::SendRpc() {
-  MasterServiceProxy proxy(retrier().messenger(), addr_, addr_.host());
+  MasterServiceProxy proxy(retrier().messenger(), addr_with_name_.first, addr_with_name_.second);
   rpc::RpcController* controller = mutable_retrier()->mutable_controller();
   // TODO(todd): should this be setting an RPC call deadline based on 'deadline'?
   // it doesn't seem to be.
@@ -127,8 +137,9 @@ void ConnectToMasterRpc::SendRpc() {
 }
 
 string ConnectToMasterRpc::ToString() const {
-  return strings::Substitute("ConnectToMasterRpc(address: $0, num_attempts: $1)",
-                             addr_.ToString(), num_attempts());
+  return strings::Substitute("ConnectToMasterRpc(address: $0:$1, num_attempts: $2)",
+                             addr_with_name_.second, addr_with_name_.first.port(),
+                             num_attempts());
 }
 
 void ConnectToMasterRpc::SendRpcCb(const Status& status) {
@@ -189,32 +200,33 @@ void ConnectToMasterRpc::SendRpcCb(const Status& status) {
 ////////////////////////////////////////////////////////////
 
 ConnectToClusterRpc::ConnectToClusterRpc(LeaderCallback user_cb,
-                                         vector<Sockaddr> addrs,
+                                         vector<pair<Sockaddr, string>> addrs_with_names,
                                          MonoTime deadline,
                                          MonoDelta rpc_timeout,
                                          shared_ptr<Messenger> messenger)
     : Rpc(deadline, std::move(messenger)),
       user_cb_(std::move(user_cb)),
-      addrs_(std::move(addrs)),
+      addrs_with_names_(std::move(addrs_with_names)),
       rpc_timeout_(rpc_timeout),
       pending_responses_(0),
       completed_(false) {
   DCHECK(deadline.Initialized());
 
   // Using resize instead of reserve to explicitly initialized the values.
-  responses_.resize(addrs_.size());
+  responses_.resize(addrs_with_names_.size());
 }
 
 ConnectToClusterRpc::~ConnectToClusterRpc() {
 }
 
 string ConnectToClusterRpc::ToString() const {
-  vector<string> sockaddr_str;
-  for (const Sockaddr& addr : addrs_) {
-    sockaddr_str.push_back(addr.ToString());
+  vector<string> addrs_str;
+  for (const auto& addr_with_name : addrs_with_names_) {
+    addrs_str.emplace_back(Substitute(
+        "$0:$1", addr_with_name.second, addr_with_name.first.port()));
   }
   return strings::Substitute("ConnectToClusterRpc(addrs: $0, num_attempts: $1)",
-                             JoinStrings(sockaddr_str, ","),
+                             JoinStrings(addrs_str, ","),
                              num_attempts());
 }
 
@@ -225,10 +237,10 @@ void ConnectToClusterRpc::SendRpc() {
                                                 rpc_deadline);
 
   std::lock_guard<simple_spinlock> l(lock_);
-  for (int i = 0; i < addrs_.size(); i++) {
+  for (int i = 0; i < addrs_with_names_.size(); i++) {
     ConnectToMasterRpc* rpc = new ConnectToMasterRpc(
         Bind(&ConnectToClusterRpc::SingleNodeCallback, this, i),
-        addrs_[i],
+        addrs_with_names_[i],
         actual_deadline,
         retrier().messenger(),
         &responses_[i]);
@@ -267,7 +279,7 @@ void ConnectToClusterRpc::SendRpcCb(const Status& status) {
   // We are not retrying.
   undo_completed.cancel();
   if (leader_idx_ != -1) {
-    user_cb_(status, addrs_[leader_idx_], responses_[leader_idx_]);
+    user_cb_(status, addrs_with_names_[leader_idx_], responses_[leader_idx_]);
   } else {
     user_cb_(status, {}, {});
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1aa96952/src/kudu/client/master_rpc.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/master_rpc.h b/src/kudu/client/master_rpc.h
index 1e55bea..28dbaf7 100644
--- a/src/kudu/client/master_rpc.h
+++ b/src/kudu/client/master_rpc.h
@@ -19,8 +19,10 @@
 #pragma once
 
 #include <functional>
-#include <vector>
+#include <memory>
 #include <string>
+#include <utility>
+#include <vector>
 
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/master/master.pb.h"
@@ -66,7 +68,7 @@ class ConnectToClusterRpc : public rpc::Rpc,
  public:
   typedef std::function<void(
       const Status& status,
-      const Sockaddr& leader_master,
+      const std::pair<Sockaddr, std::string> leader_master,
       const master::ConnectToMasterResponsePB& connect_response)> LeaderCallback;
   // The host and port of the leader master server is stored in
   // 'leader_master', which must remain valid for the lifetime of this
@@ -76,7 +78,7 @@ class ConnectToClusterRpc : public rpc::Rpc,
   // until 'deadline' passes. Each RPC has 'rpc_timeout' time to complete
   // before it times out and may be retried if 'deadline' has not yet passed.
   ConnectToClusterRpc(LeaderCallback user_cb,
-                      std::vector<Sockaddr> addrs,
+                      std::vector<std::pair<Sockaddr, std::string>> addrs_with_names,
                       MonoTime deadline,
                       MonoDelta rpc_timeout,
                       std::shared_ptr<rpc::Messenger> messenger);
@@ -100,8 +102,8 @@ class ConnectToClusterRpc : public rpc::Rpc,
 
   const LeaderCallback user_cb_;
 
-  // The addresses of the masters.
-  const std::vector<Sockaddr> addrs_;
+  // The addresses of the masters, along with their original specified names.
+  const std::vector<std::pair<Sockaddr, std::string>> addrs_with_names_;
 
   // The amount of time alloted to each GetMasterRegistration RPC.
   const MonoDelta rpc_timeout_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1aa96952/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 5c877cd..60ac657 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -24,6 +24,7 @@
 #include <rapidjson/document.h>
 #include <string>
 #include <unordered_set>
+#include <utility>
 
 #include "kudu/client/client.h"
 #include "kudu/client/master_rpc.h"
@@ -61,6 +62,7 @@ using kudu::tserver::ListTabletsRequestPB;
 using kudu::tserver::ListTabletsResponsePB;
 using kudu::tserver::TabletServerServiceProxy;
 using rapidjson::Value;
+using std::pair;
 using std::string;
 using std::unique_ptr;
 using std::unordered_set;
@@ -469,23 +471,23 @@ Status ExternalMiniCluster::WaitForTabletsRunning(ExternalTabletServer* ts,
 Status ExternalMiniCluster::GetLeaderMasterIndex(int* idx) {
   scoped_refptr<ConnectToClusterRpc> rpc;
   Synchronizer sync;
-  vector<Sockaddr> addrs;
+  vector<pair<Sockaddr, string>> addrs_with_names;
   Sockaddr leader_master_addr;
   MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(5);
 
   for (const scoped_refptr<ExternalMaster>& master : masters_) {
-    addrs.push_back(master->bound_rpc_addr());
+    addrs_with_names.emplace_back(master->bound_rpc_addr(), master->bound_rpc_addr().host());
   }
   const auto& cb = [&](const Status& status,
-                       const Sockaddr& leader_master,
+                       const pair<Sockaddr, string>& leader_master,
                        const master::ConnectToMasterResponsePB& resp) {
     if (status.ok()) {
-      leader_master_addr = leader_master;
+      leader_master_addr = leader_master.first;
     }
     sync.StatusCB(status);
   };
   rpc.reset(new ConnectToClusterRpc(cb,
-                                    std::move(addrs),
+                                    std::move(addrs_with_names),
                                     deadline,
                                     MonoDelta::FromSeconds(5),
                                     messenger_));