You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2020/04/13 19:52:02 UTC

[kudu] branch master updated (e6c742e -> b2e4d95)

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

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


    from e6c742e  KUDU-3078: parameterize ts_sentry-itest to use Ranger
     new 8528ac7  [common] update on HostPortFromPB()/HostPortToPB()
     new b2e4d95  [catalog manager] cache masters' addresses

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/client/client.cc                              |  6 ++----
 src/kudu/common/wire_protocol.cc                       | 18 +++++++++---------
 src/kudu/common/wire_protocol.h                        |  4 ++--
 src/kudu/consensus/consensus_meta-test.cc              |  3 ++-
 src/kudu/consensus/consensus_peers.cc                  |  6 ++----
 src/kudu/integration-tests/client_failover-itest.cc    |  3 +--
 src/kudu/integration-tests/cluster_itest_util.cc       |  5 ++---
 src/kudu/integration-tests/tablet_copy-itest.cc        |  6 ++----
 .../tablet_copy_client_session-itest.cc                | 15 +++++----------
 .../integration-tests/tombstoned_voting-stress-test.cc |  3 +--
 src/kudu/integration-tests/ts_tablet_manager-itest.cc  |  3 +--
 src/kudu/master/auto_rebalancer.cc                     |  3 +--
 src/kudu/master/catalog_manager.cc                     |  8 +++-----
 src/kudu/master/catalog_manager.h                      | 10 ++++++++++
 src/kudu/master/master.cc                              | 16 +++++++---------
 src/kudu/master/master.h                               |  4 ++--
 src/kudu/master/master_service.cc                      | 12 ++++--------
 src/kudu/master/sys_catalog.cc                         |  9 +++------
 src/kudu/master/ts_descriptor.cc                       |  8 ++------
 src/kudu/mini-cluster/external_mini_cluster.cc         |  8 ++------
 src/kudu/server/server_base.cc                         | 12 +++---------
 src/kudu/tools/ksck_remote.cc                          |  3 +--
 src/kudu/tools/kudu-tool-test.cc                       |  3 +--
 src/kudu/tools/tool_action_local_replica.cc            |  3 +--
 src/kudu/tools/tool_action_test.cc                     |  6 ++----
 src/kudu/tools/tool_replica_util.cc                    |  6 +++---
 src/kudu/tserver/tablet_copy_client-test.cc            |  3 +--
 src/kudu/tserver/ts_tablet_manager.cc                  |  5 ++---
 28 files changed, 77 insertions(+), 114 deletions(-)


[kudu] 02/02: [catalog manager] cache masters' addresses

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b2e4d952b6134a47db383fa2a0a88ac47cbaecc1
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Thu Apr 9 15:03:25 2020 -0700

    [catalog manager] cache masters' addresses
    
    While troubleshooting one performance issue if running a big cluster
    with large number of tables and high rate of ConnectToMaster requests,
    in the logs I noticed many reports like the following:
    
      0323 03:59:31.091574 (+114857us) spinlock_profiling.cc:243]
      Waited 114 ms on lock 0x4d0ee8c. stack:
        0000000002398852
        00000000020d45b3
        0000000000a8f8fc
        0000000000aa6300
        000000000221aaa8
        ...
    
    which translates into
    
        (anonymous namespace)::SubmitSpinLockProfileData()
        consensus::RaftConsensus::CommittedConfig()
        master::Master::GetMasterHostPorts()
        master::MasterServiceImpl::ConnectToMaster()
        rpc::GeneratedServiceIf::Handle()
        ...
    
    From the code it became apparent that the lock in question was
      LockGuard l(lock_);
    in RaftConsensus::CommittedConfig() accessor method.
    
    This patch introduces caching of the master host ports, so there is
    no need to fetch the information on the master addresses from the
    consensus metadata every time ConnectToMaster() is called.  Since the
    membership for master Raft consensus is static, it's enough to fetch
    the information on the masters' addresses upon catalog initialization
    and use that information since then.  If we implement dynamic master
    change config, we'll need to refresh the cached state on change config.
    
    This is a follow-up to 14912a1fd78ba7cf4d62bf934ae64d6f6f229ee6.
    
    Change-Id: Ic9afea2d708bd3060b3d9f5b672660e1d2dca910
    Reviewed-on: http://gerrit.cloudera.org:8080/15704
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
    Reviewed-by: Bankim Bhavsar <ba...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/master/catalog_manager.cc |  8 +++-----
 src/kudu/master/catalog_manager.h  | 10 ++++++++++
 src/kudu/master/master.cc          | 10 +++++-----
 src/kudu/master/master.h           |  4 ++--
 src/kudu/master/master_service.cc  | 12 ++++--------
 5 files changed, 24 insertions(+), 20 deletions(-)

diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 598c5f6..0f98119 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -825,13 +825,11 @@ Status CatalogManager::Init(bool is_first_run) {
     auto_rebalancer_ = std::move(task);
   }
 
+  RETURN_NOT_OK(master_->GetMasterHostPorts(&master_addresses_));
   if (hms::HmsCatalog::IsEnabled()) {
-    vector<HostPortPB> master_addrs_pb;
-    RETURN_NOT_OK(master_->GetMasterHostPorts(&master_addrs_pb));
-
     string master_addresses = JoinMapped(
-      master_addrs_pb,
-      [] (const HostPortPB& hostport) {
+      master_addresses_,
+      [] (const HostPort& hostport) {
         return Substitute("$0:$1", hostport.host(), hostport.port());
       },
       ",");
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index d6d5ff6..6650a76 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -48,6 +48,7 @@
 #include "kudu/util/cow_object.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/net/net_util.h"
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/random.h"
 #include "kudu/util/rw_mutex.h"
@@ -783,6 +784,10 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   // name is returned.
   static std::string NormalizeTableName(const std::string& table_name);
 
+  const std::vector<HostPort>& master_addresses() const {
+    return master_addresses_;
+  }
+
  private:
   // These tests call ElectedAsLeaderCb() directly.
   FRIEND_TEST(MasterTest, TestShutdownDuringTableVisit);
@@ -1158,6 +1163,11 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   // Always acquire this lock before state_lock_.
   RWMutex leader_lock_;
 
+  // Cached information on master addresses. It's populated in Init() since the
+  // membership of masters' Raft consensus is static (i.e. no new members are
+  // added or any existing removed).
+  std::vector<HostPort> master_addresses_;
+
   // Async operations are accessing some private methods
   // (TODO: this stuff should be deferred and done in the background thread)
   friend class AsyncAlterTable;
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index e64b179..e8f81c7 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -28,7 +28,6 @@
 #include <glog/logging.h>
 
 #include "kudu/cfile/block_cache.h"
-#include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/metadata.pb.h"
@@ -367,7 +366,7 @@ Status Master::ListMasters(vector<ServerEntryPB>* masters) const {
   return Status::OK();
 }
 
-Status Master::GetMasterHostPorts(vector<HostPortPB>* hostports) const {
+Status Master::GetMasterHostPorts(vector<HostPort>* hostports) const {
   auto consensus = catalog_manager_->master_consensus();
   if (!consensus) {
     return Status::IllegalState("consensus not running");
@@ -375,7 +374,7 @@ Status Master::GetMasterHostPorts(vector<HostPortPB>* hostports) const {
 
   hostports->clear();
   consensus::RaftConfigPB config = consensus->CommittedConfig();
-  for (auto& peer : *config.mutable_peers()) {
+  for (const auto& peer : *config.mutable_peers()) {
     if (peer.member_type() == consensus::RaftPeerPB::VOTER) {
       // In non-distributed master configurations, we don't store our own
       // last known address in the Raft config. So, we'll fill it in from
@@ -383,9 +382,10 @@ Status Master::GetMasterHostPorts(vector<HostPortPB>* hostports) const {
       if (!peer.has_last_known_addr()) {
         DCHECK_EQ(config.peers_size(), 1);
         DCHECK(registration_initialized_.load());
-        hostports->emplace_back(registration_.rpc_addresses(0));
+        DCHECK_GT(registration_.rpc_addresses_size(), 0);
+        hostports->emplace_back(HostPortFromPB(registration_.rpc_addresses(0)));
       } else {
-        hostports->emplace_back(std::move(*peer.mutable_last_known_addr()));
+        hostports->emplace_back(HostPortFromPB(peer.last_known_addr()));
       }
     }
   }
diff --git a/src/kudu/master/master.h b/src/kudu/master/master.h
index 3f77074..28670fe 100644
--- a/src/kudu/master/master.h
+++ b/src/kudu/master/master.h
@@ -32,7 +32,7 @@
 
 namespace kudu {
 
-class HostPortPB;
+class HostPort;
 class MaintenanceManager;
 class MonoDelta;
 class ThreadPool;
@@ -105,7 +105,7 @@ class Master : public kserver::KuduServer {
   // Gets the HostPorts for all of the masters in the cluster.
   // This is not as complete as ListMasters() above, but operates just
   // based on local state.
-  Status GetMasterHostPorts(std::vector<HostPortPB>* hostports) const;
+  Status GetMasterHostPorts(std::vector<HostPort>* hostports) const;
 
   // Crash the master on disk error.
   static void CrashMasterOnDiskError(const std::string& uuid);
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 50ed51d..b67cd02 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -629,14 +629,10 @@ void MasterServiceImpl::ConnectToMaster(const ConnectToMasterRequestPB* /*req*/,
 
   // Set the info about the other masters, so that the client can verify
   // it has the full set of info.
-  {
-    vector<HostPortPB> hostports;
-    WARN_NOT_OK(server_->GetMasterHostPorts(&hostports),
-                "unable to get HostPorts for masters");
-    resp->mutable_master_addrs()->Reserve(hostports.size());
-    for (auto& hp : hostports) {
-      *resp->add_master_addrs() = std::move(hp);
-    }
+  const auto& addresses = server_->catalog_manager()->master_addresses();
+  resp->mutable_master_addrs()->Reserve(addresses.size());
+  for (const auto& hp : addresses) {
+    *resp->add_master_addrs() = HostPortToPB(hp);
   }
 
   const bool is_leader = l.leader_status().ok();


[kudu] 01/02: [common] update on HostPortFromPB()/HostPortToPB()

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8528ac70245efb6d7fec2d18dfa17e45b80421a9
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Fri Apr 10 11:06:02 2020 -0700

    [common] update on HostPortFromPB()/HostPortToPB()
    
    HostPortFromPB() and HostPortToPB() would never return non-OK status,
    so it's possible to simplify the signature of these two methods.
    
    This changelist doesn't contain any functional modifications.
    
    Change-Id: I6dfcf747ee411f89c85e4c2d92ddbc756b551f43
    Reviewed-on: http://gerrit.cloudera.org:8080/15713
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/client/client.cc                              |  6 ++----
 src/kudu/common/wire_protocol.cc                       | 18 +++++++++---------
 src/kudu/common/wire_protocol.h                        |  4 ++--
 src/kudu/consensus/consensus_meta-test.cc              |  3 ++-
 src/kudu/consensus/consensus_peers.cc                  |  6 ++----
 src/kudu/integration-tests/client_failover-itest.cc    |  3 +--
 src/kudu/integration-tests/cluster_itest_util.cc       |  5 ++---
 src/kudu/integration-tests/tablet_copy-itest.cc        |  6 ++----
 .../tablet_copy_client_session-itest.cc                | 15 +++++----------
 .../integration-tests/tombstoned_voting-stress-test.cc |  3 +--
 src/kudu/integration-tests/ts_tablet_manager-itest.cc  |  3 +--
 src/kudu/master/auto_rebalancer.cc                     |  3 +--
 src/kudu/master/master.cc                              |  6 ++----
 src/kudu/master/sys_catalog.cc                         |  9 +++------
 src/kudu/master/ts_descriptor.cc                       |  8 ++------
 src/kudu/mini-cluster/external_mini_cluster.cc         |  8 ++------
 src/kudu/server/server_base.cc                         | 12 +++---------
 src/kudu/tools/ksck_remote.cc                          |  3 +--
 src/kudu/tools/kudu-tool-test.cc                       |  3 +--
 src/kudu/tools/tool_action_local_replica.cc            |  3 +--
 src/kudu/tools/tool_action_test.cc                     |  6 ++----
 src/kudu/tools/tool_replica_util.cc                    |  6 +++---
 src/kudu/tserver/tablet_copy_client-test.cc            |  3 +--
 src/kudu/tserver/ts_tablet_manager.cc                  |  5 ++---
 24 files changed, 53 insertions(+), 94 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index d4b607f..4a97e28 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -457,8 +457,7 @@ Status KuduClient::ListTabletServers(vector<KuduTabletServer*>* tablet_servers)
   RETURN_NOT_OK(data_->ListTabletServers(this, deadline, req, &resp));
   for (int i = 0; i < resp.servers_size(); i++) {
     const ListTabletServersResponsePB_Entry& e = resp.servers(i);
-    HostPort hp;
-    RETURN_NOT_OK(HostPortFromPB(e.registration().rpc_addresses(0), &hp));
+    HostPort hp = HostPortFromPB(e.registration().rpc_addresses(0));
     unique_ptr<KuduTabletServer> ts(new KuduTabletServer);
     ts->data_ = new KuduTabletServer::Data(e.instance_id().permanent_uuid(), hp, e.location());
     tablet_servers->push_back(ts.release());
@@ -555,8 +554,7 @@ Status KuduClient::GetTablet(const string& tablet_id, KuduTablet** tablet) {
           "No RPC addresses found for tserver $0",
           ts_info.permanent_uuid()));
     }
-    HostPort hp;
-    RETURN_NOT_OK(HostPortFromPB(ts_info.rpc_addresses(0), &hp));
+    HostPort hp = HostPortFromPB(ts_info.rpc_addresses(0));
     unique_ptr<KuduTabletServer> ts(new KuduTabletServer);
     ts->data_ = new KuduTabletServer::Data(ts_info.permanent_uuid(), hp, ts_info.location());
 
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index ef14d8b..c423a02 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -185,16 +185,15 @@ Status StatusFromPB(const AppStatusPB& pb) {
   }
 }
 
-Status HostPortToPB(const HostPort& host_port, HostPortPB* host_port_pb) {
-  host_port_pb->set_host(host_port.host());
-  host_port_pb->set_port(host_port.port());
-  return Status::OK();
+HostPortPB HostPortToPB(const HostPort& host_port) {
+  HostPortPB ret;
+  ret.set_host(host_port.host());
+  ret.set_port(host_port.port());
+  return ret;
 }
 
-Status HostPortFromPB(const HostPortPB& host_port_pb, HostPort* host_port) {
-  host_port->set_host(host_port_pb.host());
-  host_port->set_port(host_port_pb.port());
-  return Status::OK();
+HostPort HostPortFromPB(const HostPortPB& host_port_pb) {
+  return HostPort(host_port_pb.host(), host_port_pb.port());
 }
 
 Status AddHostPortPBs(const vector<Sockaddr>& addrs,
@@ -849,7 +848,8 @@ Status FindLeaderHostPort(const RepeatedPtrField<ServerEntryPB>& entries,
                               SecureShortDebugString(entry)));
     }
     if (entry.role() == consensus::RaftPeerPB::LEADER) {
-      return HostPortFromPB(entry.registration().rpc_addresses(0), leader_hostport);
+      *leader_hostport = HostPortFromPB(entry.registration().rpc_addresses(0));
+      return Status::OK();
     }
   }
   return Status::NotFound("No leader found.");
diff --git a/src/kudu/common/wire_protocol.h b/src/kudu/common/wire_protocol.h
index 7ac76fd..f714fc2 100644
--- a/src/kudu/common/wire_protocol.h
+++ b/src/kudu/common/wire_protocol.h
@@ -69,10 +69,10 @@ void StatusToPB(const Status& status, AppStatusPB* pb);
 Status StatusFromPB(const AppStatusPB& pb);
 
 // Convert the specified HostPort to protobuf.
-Status HostPortToPB(const HostPort& host_port, HostPortPB* host_port_pb);
+HostPortPB HostPortToPB(const HostPort& host_port);
 
 // Returns the HostPort created from the specified protobuf.
-Status HostPortFromPB(const HostPortPB& host_port_pb, HostPort* host_port);
+HostPort HostPortFromPB(const HostPortPB& host_port_pb);
 
 // Convert the column schema delta `col_delta` to protobuf.
 void ColumnSchemaDeltaToPB(const ColumnSchemaDelta& col_delta, ColumnSchemaDeltaPB *pb);
diff --git a/src/kudu/consensus/consensus_meta-test.cc b/src/kudu/consensus/consensus_meta-test.cc
index 37624c9..50616f6 100644
--- a/src/kudu/consensus/consensus_meta-test.cc
+++ b/src/kudu/consensus/consensus_meta-test.cc
@@ -28,6 +28,7 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
+#include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/consensus/opid_util.h"
@@ -183,7 +184,7 @@ RaftConfigPB BuildConfig(const vector<string>& uuids) {
     RaftPeerPB* peer = config.add_peers();
     peer->set_permanent_uuid(uuid);
     peer->set_member_type(RaftPeerPB::VOTER);
-    CHECK_OK(HostPortToPB(HostPort("255.255.255.255", 0), peer->mutable_last_known_addr()));
+    *peer->mutable_last_known_addr() = HostPortToPB(HostPort("255.255.255.255", 0));
   }
   return config;
 }
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 81ab29d..2d9c116 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -577,8 +577,7 @@ RpcPeerProxyFactory::RpcPeerProxyFactory(shared_ptr<Messenger> messenger,
 
 Status RpcPeerProxyFactory::NewProxy(const RaftPeerPB& peer_pb,
                                      unique_ptr<PeerProxy>* proxy) {
-  HostPort hostport;
-  RETURN_NOT_OK(HostPortFromPB(peer_pb.last_known_addr(), &hostport));
+  HostPort hostport = HostPortFromPB(peer_pb.last_known_addr());
   unique_ptr<ConsensusServiceProxy> new_proxy;
   RETURN_NOT_OK(CreateConsensusServiceProxyForHost(
       hostport, messenger_, dns_resolver_, &new_proxy));
@@ -591,8 +590,7 @@ Status SetPermanentUuidForRemotePeer(
     DnsResolver* resolver,
     RaftPeerPB* remote_peer) {
   DCHECK(!remote_peer->has_permanent_uuid());
-  HostPort hostport;
-  RETURN_NOT_OK(HostPortFromPB(remote_peer->last_known_addr(), &hostport));
+  HostPort hostport = HostPortFromPB(remote_peer->last_known_addr());
   unique_ptr<ConsensusServiceProxy> proxy;
   RETURN_NOT_OK(CreateConsensusServiceProxyForHost(
       hostport, messenger, resolver, &proxy));
diff --git a/src/kudu/integration-tests/client_failover-itest.cc b/src/kudu/integration-tests/client_failover-itest.cc
index 9bc0525..8183b78 100644
--- a/src/kudu/integration-tests/client_failover-itest.cc
+++ b/src/kudu/integration-tests/client_failover-itest.cc
@@ -190,8 +190,7 @@ TEST_P(ClientFailoverParamITest, TestDeleteLeaderWhileScanning) {
 
   TServerDetails* to_add = ts_map_[cluster_->tablet_server(missing_replica_index)->uuid()];
   ASSERT_OK(AddServer(leader, tablet_id, to_add, consensus::RaftPeerPB::VOTER, kTimeout));
-  HostPort hp;
-  ASSERT_OK(HostPortFromPB(leader->registration.rpc_addresses(0), &hp));
+  HostPort hp = HostPortFromPB(leader->registration.rpc_addresses(0));
   ASSERT_OK(StartTabletCopy(to_add, tablet_id, leader->uuid(), hp, 1, kTimeout));
 
   const string& new_ts_uuid = cluster_->tablet_server(missing_replica_index)->uuid();
diff --git a/src/kudu/integration-tests/cluster_itest_util.cc b/src/kudu/integration-tests/cluster_itest_util.cc
index 8e53fe7..8975d41 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -316,8 +316,7 @@ Status CreateTabletServerMap(const shared_ptr<MasterServiceProxy>& master_proxy,
 
   ts_map->clear();
   for (const auto& entry : tservers) {
-    HostPort host_port;
-    RETURN_NOT_OK(HostPortFromPB(entry.registration().rpc_addresses(0), &host_port));
+    HostPort host_port = HostPortFromPB(entry.registration().rpc_addresses(0));
     vector<Sockaddr> addresses;
     host_port.ResolveAddresses(&addresses);
 
@@ -1227,7 +1226,7 @@ Status StartTabletCopy(const TServerDetails* ts,
   req.set_dest_uuid(ts->uuid());
   req.set_tablet_id(tablet_id);
   req.set_copy_peer_uuid(copy_source_uuid);
-  RETURN_NOT_OK(HostPortToPB(copy_source_addr, req.mutable_copy_peer_addr()));
+  *req.mutable_copy_peer_addr() = HostPortToPB(copy_source_addr);
   req.set_caller_term(caller_term);
 
   RETURN_NOT_OK(ts->consensus_proxy->StartTabletCopy(req, &resp, &rpc));
diff --git a/src/kudu/integration-tests/tablet_copy-itest.cc b/src/kudu/integration-tests/tablet_copy-itest.cc
index f136f02..746c012 100644
--- a/src/kudu/integration-tests/tablet_copy-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy-itest.cc
@@ -425,8 +425,7 @@ TEST_F(TabletCopyITest, TestCopyAfterFailedCopy) {
   ASSERT_OK(DeleteTabletWithRetries(follower_ts, tablet_id,
                                     TabletDataState::TABLET_DATA_TOMBSTONED,
                                     kTimeout));
-  HostPort leader_addr;
-  ASSERT_OK(HostPortFromPB(leader_ts->registration.rpc_addresses(0), &leader_addr));
+  HostPort leader_addr = HostPortFromPB(leader_ts->registration.rpc_addresses(0));
 
   // Inject failures to the metadata and trigger the tablet copy. This will
   // cause the copy to fail.
@@ -1165,8 +1164,7 @@ TEST_F(TabletCopyITest, TestTabletCopyThrottling) {
   // ServiceUnavailable error.
   ASSERT_OK(cluster_->tablet_server(1)->Restart());
 
-  HostPort ts0_hostport;
-  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &ts0_hostport));
+  HostPort ts0_hostport = HostPortFromPB(ts0->registration.rpc_addresses(0));
 
   // Attempt to copy all of the tablets from TS0 to TS1 in parallel. Tablet
   // copies are repeated periodically until complete.
diff --git a/src/kudu/integration-tests/tablet_copy_client_session-itest.cc b/src/kudu/integration-tests/tablet_copy_client_session-itest.cc
index b2f7c47..119a5bd 100644
--- a/src/kudu/integration-tests/tablet_copy_client_session-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy_client_session-itest.cc
@@ -135,8 +135,7 @@ TEST_F(TabletCopyClientSessionITest, TestStartTabletCopyWhileSourceBootstrapping
   ASSERT_OK(WaitForNumTabletsOnTS(ts0, kDefaultNumTablets, kTimeout, &tablets));
   ASSERT_EQ(kDefaultNumTablets, tablets.size());
   const string& tablet_id = tablets[0].tablet_status().tablet_id();
-  HostPort src_addr;
-  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
+  HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
 
   // Repeat the restart process several times.
   for (int i = 0; i < 3; i++) {
@@ -229,8 +228,7 @@ TEST_F(TabletCopyClientSessionITest, TestStartTabletCopy) {
     }
 
     // Run tablet copy.
-    HostPort src_addr;
-    ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
+    HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
     ASSERT_OK(StartTabletCopy(ts1, tablet_id, ts0->uuid(), src_addr,
                               std::numeric_limits<int64_t>::max(), kDefaultTimeout));
     ASSERT_OK(WaitUntilTabletRunning(ts1, tablet_id, kDefaultTimeout));
@@ -254,8 +252,7 @@ TEST_F(TabletCopyClientSessionITest, TestCopyFromCrashedSource) {
                               "fault_crash_on_handle_tc_fetch_data",
                               "1.0"));
 
-  HostPort src_addr;
-  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
+  HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
   ASSERT_OK(StartTabletCopy(ts1, tablet_id, ts0->uuid(), src_addr,
                             std::numeric_limits<int64_t>::max(), kDefaultTimeout));
 
@@ -312,8 +309,7 @@ TEST_F(TabletCopyClientSessionITest, TestTabletCopyWithBusySource) {
                                   &tablets, tablet::TabletStatePB::RUNNING));
   ASSERT_EQ(kNumTablets, tablets.size());
 
-  HostPort src_addr;
-  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
+  HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
 
   vector<thread> threads;
   for (const auto& tablet : tablets) {
@@ -369,8 +365,7 @@ TEST_F(TabletCopyClientSessionITest, TestStopCopyOnClientDiskFailure) {
   });
 
   // Now kick off the tablet copies.
-  HostPort src_addr;
-  ASSERT_OK(HostPortFromPB(ts_src->registration.rpc_addresses(0), &src_addr));
+  HostPort src_addr = HostPortFromPB(ts_src->registration.rpc_addresses(0));
   vector<thread> threads;
   auto copy_tablet_with_index = [&] (int i) {
     LOG(INFO) << Substitute("Copying tablet $0 / $1", i + 1, kNumTablets);
diff --git a/src/kudu/integration-tests/tombstoned_voting-stress-test.cc b/src/kudu/integration-tests/tombstoned_voting-stress-test.cc
index 1018215..7f77f03 100644
--- a/src/kudu/integration-tests/tombstoned_voting-stress-test.cc
+++ b/src/kudu/integration-tests/tombstoned_voting-stress-test.cc
@@ -299,8 +299,7 @@ TEST_F(TombstonedVotingStressTest, TestTombstonedVotingUnderStress) {
 
     // 2. Copy tablet.
     LOG(INFO) << "copying tablet...";
-    HostPort source_hp;
-    ASSERT_OK(HostPortFromPB(ts0_ets->registration.rpc_addresses(0), &source_hp));
+    HostPort source_hp = HostPortFromPB(ts0_ets->registration.rpc_addresses(0));
     SetState(kCopying);
     ASSERT_OK(itest::StartTabletCopy(ts1_ets, tablet_id_, ts0_ets->uuid(), source_hp, current_term_,
                                      kTimeout));
diff --git a/src/kudu/integration-tests/ts_tablet_manager-itest.cc b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
index b03a011..fd8acbc 100644
--- a/src/kudu/integration-tests/ts_tablet_manager-itest.cc
+++ b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
@@ -965,8 +965,7 @@ TEST_F(TsTabletManagerITest, TestDeleteTableDuringTabletCopy) {
                                            tablet::TabletDataState::TABLET_DATA_TOMBSTONED,
                                            kTimeout));
   // Copy tablet from leader_ts to follower_ts.
-  HostPort leader_addr;
-  ASSERT_OK(HostPortFromPB(leader_ts->registration.rpc_addresses(0), &leader_addr));
+  HostPort leader_addr = HostPortFromPB(leader_ts->registration.rpc_addresses(0));
   ASSERT_OK(itest::StartTabletCopy(follower_ts, tablet_id, leader_ts->uuid(),
                                    leader_addr, std::numeric_limits<int64_t>::max(), kTimeout));
 
diff --git a/src/kudu/master/auto_rebalancer.cc b/src/kudu/master/auto_rebalancer.cc
index 122dd14..3c48797 100644
--- a/src/kudu/master/auto_rebalancer.cc
+++ b/src/kudu/master/auto_rebalancer.cc
@@ -449,8 +449,7 @@ Status AutoRebalancerTask::ExecuteMoves(
       *change->mutable_peer()->mutable_permanent_uuid() = dst_ts_uuid;
       change->mutable_peer()->set_member_type(RaftPeerPB::NON_VOTER);
       change->mutable_peer()->mutable_attrs()->set_promote(true);
-      RETURN_NOT_OK(
-          HostPortToPB(hp, change->mutable_peer()->mutable_last_known_addr()));
+      *change->mutable_peer()->mutable_last_known_addr() = HostPortToPB(hp);
     }
 
     // Request movement or replacement of the replica.
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index f55cc6a..e64b179 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -344,11 +344,9 @@ Status Master::ListMasters(vector<ServerEntryPB>* masters) const {
 
   masters->clear();
   for (const auto& peer : config.peers()) {
+    HostPort hp = HostPortFromPB(peer.last_known_addr());
     ServerEntryPB peer_entry;
-    HostPort hp;
-    Status s = HostPortFromPB(peer.last_known_addr(), &hp).AndThen([&] {
-      return GetMasterEntryForHost(messenger_, hp, &peer_entry);
-    });
+    Status s = GetMasterEntryForHost(messenger_, hp, &peer_entry);
     if (!s.ok()) {
       s = s.CloneAndPrepend(
           Substitute("Unable to get registration information for peer $0 ($1)",
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 9011cd0..46aa8b9 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -228,9 +228,7 @@ Status SysCatalogTable::Load(FsManager *fs_manager) {
     }
     set<string> peer_addrs_from_disk;
     for (const auto& p : cstate.committed_config().peers()) {
-      HostPort hp;
-      RETURN_NOT_OK(HostPortFromPB(p.last_known_addr(), &hp));
-      peer_addrs_from_disk.insert(hp.ToString());
+      peer_addrs_from_disk.insert(HostPortFromPB(p.last_known_addr()).ToString());
     }
     vector<string> symm_diff;
     std::set_symmetric_difference(peer_addrs_from_opts.begin(),
@@ -307,8 +305,7 @@ Status SysCatalogTable::CreateDistributedConfig(const MasterOptions& options,
   // Build the set of followers from our server options.
   for (const HostPort& host_port : options.master_addresses) {
     RaftPeerPB peer;
-    HostPortPB peer_host_port_pb;
-    RETURN_NOT_OK(HostPortToPB(host_port, &peer_host_port_pb));
+    HostPortPB peer_host_port_pb = HostPortToPB(host_port);
     peer.mutable_last_known_addr()->CopyFrom(peer_host_port_pb);
     peer.set_member_type(RaftPeerPB::VOTER);
     new_config.add_peers()->CopyFrom(peer);
@@ -1007,7 +1004,7 @@ void SysCatalogTable::InitLocalRaftPeerPB() {
   Sockaddr addr = master_->first_rpc_address();
   HostPort hp;
   CHECK_OK(HostPortFromSockaddrReplaceWildcard(addr, &hp));
-  CHECK_OK(HostPortToPB(hp, local_peer_pb_.mutable_last_known_addr()));
+  *local_peer_pb_.mutable_last_known_addr() = HostPortToPB(hp);
 }
 
 } // namespace master
diff --git a/src/kudu/master/ts_descriptor.cc b/src/kudu/master/ts_descriptor.cc
index 90f035d..712df27 100644
--- a/src/kudu/master/ts_descriptor.cc
+++ b/src/kudu/master/ts_descriptor.cc
@@ -93,12 +93,8 @@ static bool HostPortPBsEqual(const google::protobuf::RepeatedPtrField<HostPortPB
   std::unordered_set<HostPort, HostPortHasher, HostPortEqualityPredicate> hostports1;
   std::unordered_set<HostPort, HostPortHasher, HostPortEqualityPredicate> hostports2;
   for (int i = 0; i < pb1.size(); i++) {
-    HostPort hp1;
-    HostPort hp2;
-    if (!HostPortFromPB(pb1.Get(i), &hp1).ok()) return false;
-    if (!HostPortFromPB(pb2.Get(i), &hp2).ok()) return false;
-    hostports1.insert(hp1);
-    hostports2.insert(hp2);
+    hostports1.emplace(HostPortFromPB(pb1.Get(i)));
+    hostports2.emplace(HostPortFromPB(pb2.Get(i)));
   }
   return hostports1 == hostports2;
 }
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index c249286..ee6529c 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -1377,9 +1377,7 @@ void ExternalDaemon::CheckForLeaks() {
 HostPort ExternalDaemon::bound_rpc_hostport() const {
   CHECK(status_);
   CHECK_GE(status_->bound_rpc_addresses_size(), 1);
-  HostPort ret;
-  CHECK_OK(HostPortFromPB(status_->bound_rpc_addresses(0), &ret));
-  return ret;
+  return HostPortFromPB(status_->bound_rpc_addresses(0));
 }
 
 Sockaddr ExternalDaemon::bound_rpc_addr() const {
@@ -1395,9 +1393,7 @@ HostPort ExternalDaemon::bound_http_hostport() const {
   if (status_->bound_http_addresses_size() == 0) {
     return HostPort();
   }
-  HostPort ret;
-  CHECK_OK(HostPortFromPB(status_->bound_http_addresses(0), &ret));
-  return ret;
+  return HostPortFromPB(status_->bound_http_addresses(0));
 }
 
 const NodeInstancePB& ExternalDaemon::instance_id() const {
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 610d51d..d8a12bf 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -34,6 +34,7 @@
 #include "kudu/clock/hybrid_clock.h"
 #include "kudu/clock/logical_clock.h"
 #include "kudu/codegen/compilation_manager.h"
+#include "kudu/common/common.pb.h"
 #include "kudu/common/timestamp.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
@@ -225,9 +226,6 @@ using std::vector;
 using strings::Substitute;
 
 namespace kudu {
-
-class HostPortPB;
-
 namespace server {
 
 namespace {
@@ -589,9 +587,7 @@ Status ServerBase::GetStatusPB(ServerStatusPB* status) const {
       HostPort hp;
       RETURN_NOT_OK_PREPEND(HostPortFromSockaddrReplaceWildcard(addr, &hp),
                             "could not get RPC hostport");
-      HostPortPB* pb = status->add_bound_rpc_addresses();
-      RETURN_NOT_OK_PREPEND(HostPortToPB(hp, pb),
-                            "could not convert RPC hostport");
+      *status->add_bound_rpc_addresses() = HostPortToPB(hp);
     }
   }
 
@@ -604,9 +600,7 @@ Status ServerBase::GetStatusPB(ServerStatusPB* status) const {
       HostPort hp;
       RETURN_NOT_OK_PREPEND(HostPortFromSockaddrReplaceWildcard(addr, &hp),
                             "could not get web hostport");
-      HostPortPB* pb = status->add_bound_http_addresses();
-      RETURN_NOT_OK_PREPEND(HostPortToPB(hp, pb),
-                            "could not convert web hostport");
+      *status->add_bound_http_addresses() = HostPortToPB(hp);
     }
   }
 
diff --git a/src/kudu/tools/ksck_remote.cc b/src/kudu/tools/ksck_remote.cc
index 6414e84..7627aca 100644
--- a/src/kudu/tools/ksck_remote.cc
+++ b/src/kudu/tools/ksck_remote.cc
@@ -569,8 +569,7 @@ Status RemoteKsckCluster::RetrieveTabletServers() {
     if (!ts_pb.has_registration()) {
       continue;
     }
-    HostPort hp;
-    RETURN_NOT_OK(HostPortFromPB(ts_pb.registration().rpc_addresses(0), &hp));
+    HostPort hp = HostPortFromPB(ts_pb.registration().rpc_addresses(0));
     auto ts = std::make_shared<RemoteKsckTabletServer>(uuid, hp, messenger_, ts_pb.location());
     RETURN_NOT_OK(ts->Init());
     EmplaceOrUpdate(&tablet_servers, uuid, std::move(ts));
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 4bc44c5..09bfd31 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -4704,8 +4704,7 @@ TEST_P(ControlShellToolTest, TestControlShell) {
   {
     KuduClientBuilder client_builder;
     for (const auto& e : masters) {
-      HostPort hp;
-      ASSERT_OK(HostPortFromPB(e.bound_rpc_address(), &hp));
+      HostPort hp = HostPortFromPB(e.bound_rpc_address());
       client_builder.add_master_server_addr(hp.ToString());
     }
     shared_ptr<KuduClient> client;
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index 67a2d1a..31b3118 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -304,8 +304,7 @@ Status RewriteRaftConfig(const RunnerContext& context) {
     RaftPeerPB new_peer;
     new_peer.set_member_type(RaftPeerPB::VOTER);
     new_peer.set_permanent_uuid(p.first);
-    HostPortPB new_peer_host_port_pb;
-    RETURN_NOT_OK(HostPortToPB(p.second, &new_peer_host_port_pb));
+    HostPortPB new_peer_host_port_pb = HostPortToPB(p.second);
     new_peer.mutable_last_known_addr()->CopyFrom(new_peer_host_port_pb);
     new_config.add_peers()->CopyFrom(new_peer);
   }
diff --git a/src/kudu/tools/tool_action_test.cc b/src/kudu/tools/tool_action_test.cc
index 3a56c0b..2c3202a 100644
--- a/src/kudu/tools/tool_action_test.cc
+++ b/src/kudu/tools/tool_action_test.cc
@@ -241,8 +241,7 @@ Status ProcessRequest(const ControlShellRequestPB& req,
     {
       RETURN_NOT_OK(CheckClusterExists(*cluster));
       for (int i = 0; i < (*cluster)->num_masters(); i++) {
-        HostPortPB pb;
-        RETURN_NOT_OK(HostPortToPB((*cluster)->master(i)->bound_rpc_hostport(), &pb));
+        HostPortPB pb = HostPortToPB((*cluster)->master(i)->bound_rpc_hostport());
         DaemonInfoPB* info = resp->mutable_get_masters()->mutable_masters()->Add();
         info->mutable_id()->set_type(MASTER);
         info->mutable_id()->set_index(i);
@@ -254,8 +253,7 @@ Status ProcessRequest(const ControlShellRequestPB& req,
     {
       RETURN_NOT_OK(CheckClusterExists(*cluster));
       for (int i = 0; i < (*cluster)->num_tablet_servers(); i++) {
-        HostPortPB pb;
-        RETURN_NOT_OK(HostPortToPB((*cluster)->tablet_server(i)->bound_rpc_hostport(), &pb));
+        HostPortPB pb = HostPortToPB((*cluster)->tablet_server(i)->bound_rpc_hostport());
         DaemonInfoPB* info = resp->mutable_get_tservers()->mutable_tservers()->Add();
         info->mutable_id()->set_type(TSERVER);
         info->mutable_id()->set_index(i);
diff --git a/src/kudu/tools/tool_replica_util.cc b/src/kudu/tools/tool_replica_util.cc
index 97cad58..0a23cc8 100644
--- a/src/kudu/tools/tool_replica_util.cc
+++ b/src/kudu/tools/tool_replica_util.cc
@@ -28,6 +28,7 @@
 
 #include "kudu/client/client.h"
 #include "kudu/client/shared_ptr.h" // IWYU pragma: keep
+#include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/consensus.proxy.h"
@@ -591,8 +592,7 @@ Status ScheduleReplicaMove(const vector<string>& master_addresses,
     change->mutable_peer()->mutable_attrs()->set_promote(true);
     HostPort hp;
     RETURN_NOT_OK(GetRpcAddressForTS(client, to_ts_uuid, &hp));
-    RETURN_NOT_OK(
-        HostPortToPB(hp, change->mutable_peer()->mutable_last_known_addr()));
+    *change->mutable_peer()->mutable_last_known_addr() = HostPortToPB(hp);
   }
   req.set_cas_config_opid_index(cas_opid_idx);
 
@@ -682,7 +682,7 @@ Status DoChangeConfig(const vector<string>& master_addresses,
   if (cc_type == consensus::ADD_PEER) {
     HostPort hp;
     RETURN_NOT_OK(GetRpcAddressForTS(client, replica_uuid, &hp));
-    RETURN_NOT_OK(HostPortToPB(hp, peer_pb.mutable_last_known_addr()));
+    *peer_pb.mutable_last_known_addr() = HostPortToPB(hp);
   }
 
   // Find this tablet's leader replica. We need its UUID and RPC address.
diff --git a/src/kudu/tserver/tablet_copy_client-test.cc b/src/kudu/tserver/tablet_copy_client-test.cc
index 943966b..2f10c1b 100644
--- a/src/kudu/tserver/tablet_copy_client-test.cc
+++ b/src/kudu/tserver/tablet_copy_client-test.cc
@@ -137,8 +137,7 @@ class TabletCopyClientTest : public TabletCopyTest {
 
   // Starts the tablet copy.
   Status StartCopy() {
-    HostPort host_port;
-    RETURN_NOT_OK(HostPortFromPB(leader_.last_known_addr(), &host_port));
+    HostPort host_port = HostPortFromPB(leader_.last_known_addr());
     return client_->Start(host_port, &meta_);
   }
 
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index 706c2e5..2a6ff2b 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -626,8 +626,7 @@ void TSTabletManager::RunTabletCopy(
   // Copy these strings so they stay valid even after responding to the request.
   string tablet_id = req->tablet_id(); // NOLINT(*)
   string copy_source_uuid = req->copy_peer_uuid(); // NOLINT(*)
-  HostPort copy_source_addr;
-  CALLBACK_RETURN_NOT_OK(HostPortFromPB(req->copy_peer_addr(), &copy_source_addr));
+  HostPort copy_source_addr = HostPortFromPB(req->copy_peer_addr());
   int64_t leader_term = req->caller_term();
 
   scoped_refptr<TabletReplica> old_replica;
@@ -1336,7 +1335,7 @@ void TSTabletManager::InitLocalRaftPeerPB() {
   Sockaddr addr = server_->first_rpc_address();
   HostPort hp;
   CHECK_OK(HostPortFromSockaddrReplaceWildcard(addr, &hp));
-  CHECK_OK(HostPortToPB(hp, local_peer_pb_.mutable_last_known_addr()));
+  *local_peer_pb_.mutable_last_known_addr() = HostPortToPB(hp);
 }
 
 void TSTabletManager::CreateReportedTabletPB(const scoped_refptr<TabletReplica>& replica,