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/15 23:14:08 UTC

[kudu] 02/03: Revert "[common] update on HostPortFromPB()/HostPortToPB()"

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

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

commit afae2bd463f3248e9ef89f858d832abadfbf0600
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Wed Apr 15 14:54:59 2020 -0700

    Revert "[common] update on HostPortFromPB()/HostPortToPB()"
    
    This reverts commit abd31bd4b13e95bea7a84108dbe5a20b837a89bc.
    
    Change-Id: I1b2615a3dd5c3f3d2f639ce8a46d05c3845b95c4
    Reviewed-on: http://gerrit.cloudera.org:8080/15740
    Reviewed-by: Hao Hao <ha...@cloudera.com>
    Tested-by: Alexey Serbin <as...@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, 94 insertions(+), 53 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 4a97e28..d4b607f 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -457,7 +457,8 @@ 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 = HostPortFromPB(e.registration().rpc_addresses(0));
+    HostPort hp;
+    RETURN_NOT_OK(HostPortFromPB(e.registration().rpc_addresses(0), &hp));
     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());
@@ -554,7 +555,8 @@ Status KuduClient::GetTablet(const string& tablet_id, KuduTablet** tablet) {
           "No RPC addresses found for tserver $0",
           ts_info.permanent_uuid()));
     }
-    HostPort hp = HostPortFromPB(ts_info.rpc_addresses(0));
+    HostPort hp;
+    RETURN_NOT_OK(HostPortFromPB(ts_info.rpc_addresses(0), &hp));
     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 c423a02..ef14d8b 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -185,15 +185,16 @@ Status StatusFromPB(const AppStatusPB& pb) {
   }
 }
 
-HostPortPB HostPortToPB(const HostPort& host_port) {
-  HostPortPB ret;
-  ret.set_host(host_port.host());
-  ret.set_port(host_port.port());
-  return ret;
+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();
 }
 
-HostPort HostPortFromPB(const HostPortPB& host_port_pb) {
-  return HostPort(host_port_pb.host(), host_port_pb.port());
+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();
 }
 
 Status AddHostPortPBs(const vector<Sockaddr>& addrs,
@@ -848,8 +849,7 @@ Status FindLeaderHostPort(const RepeatedPtrField<ServerEntryPB>& entries,
                               SecureShortDebugString(entry)));
     }
     if (entry.role() == consensus::RaftPeerPB::LEADER) {
-      *leader_hostport = HostPortFromPB(entry.registration().rpc_addresses(0));
-      return Status::OK();
+      return HostPortFromPB(entry.registration().rpc_addresses(0), leader_hostport);
     }
   }
   return Status::NotFound("No leader found.");
diff --git a/src/kudu/common/wire_protocol.h b/src/kudu/common/wire_protocol.h
index f714fc2..7ac76fd 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.
-HostPortPB HostPortToPB(const HostPort& host_port);
+Status HostPortToPB(const HostPort& host_port, HostPortPB* host_port_pb);
 
 // Returns the HostPort created from the specified protobuf.
-HostPort HostPortFromPB(const HostPortPB& host_port_pb);
+Status HostPortFromPB(const HostPortPB& host_port_pb, HostPort* host_port);
 
 // 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 50616f6..37624c9 100644
--- a/src/kudu/consensus/consensus_meta-test.cc
+++ b/src/kudu/consensus/consensus_meta-test.cc
@@ -28,7 +28,6 @@
 #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"
@@ -184,7 +183,7 @@ RaftConfigPB BuildConfig(const vector<string>& uuids) {
     RaftPeerPB* peer = config.add_peers();
     peer->set_permanent_uuid(uuid);
     peer->set_member_type(RaftPeerPB::VOTER);
-    *peer->mutable_last_known_addr() = HostPortToPB(HostPort("255.255.255.255", 0));
+    CHECK_OK(HostPortToPB(HostPort("255.255.255.255", 0), peer->mutable_last_known_addr()));
   }
   return config;
 }
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 2d9c116..81ab29d 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -577,7 +577,8 @@ RpcPeerProxyFactory::RpcPeerProxyFactory(shared_ptr<Messenger> messenger,
 
 Status RpcPeerProxyFactory::NewProxy(const RaftPeerPB& peer_pb,
                                      unique_ptr<PeerProxy>* proxy) {
-  HostPort hostport = HostPortFromPB(peer_pb.last_known_addr());
+  HostPort hostport;
+  RETURN_NOT_OK(HostPortFromPB(peer_pb.last_known_addr(), &hostport));
   unique_ptr<ConsensusServiceProxy> new_proxy;
   RETURN_NOT_OK(CreateConsensusServiceProxyForHost(
       hostport, messenger_, dns_resolver_, &new_proxy));
@@ -590,7 +591,8 @@ Status SetPermanentUuidForRemotePeer(
     DnsResolver* resolver,
     RaftPeerPB* remote_peer) {
   DCHECK(!remote_peer->has_permanent_uuid());
-  HostPort hostport = HostPortFromPB(remote_peer->last_known_addr());
+  HostPort hostport;
+  RETURN_NOT_OK(HostPortFromPB(remote_peer->last_known_addr(), &hostport));
   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 8183b78..9bc0525 100644
--- a/src/kudu/integration-tests/client_failover-itest.cc
+++ b/src/kudu/integration-tests/client_failover-itest.cc
@@ -190,7 +190,8 @@ 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 = HostPortFromPB(leader->registration.rpc_addresses(0));
+  HostPort hp;
+  ASSERT_OK(HostPortFromPB(leader->registration.rpc_addresses(0), &hp));
   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 8975d41..8e53fe7 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -316,7 +316,8 @@ Status CreateTabletServerMap(const shared_ptr<MasterServiceProxy>& master_proxy,
 
   ts_map->clear();
   for (const auto& entry : tservers) {
-    HostPort host_port = HostPortFromPB(entry.registration().rpc_addresses(0));
+    HostPort host_port;
+    RETURN_NOT_OK(HostPortFromPB(entry.registration().rpc_addresses(0), &host_port));
     vector<Sockaddr> addresses;
     host_port.ResolveAddresses(&addresses);
 
@@ -1226,7 +1227,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);
-  *req.mutable_copy_peer_addr() = HostPortToPB(copy_source_addr);
+  RETURN_NOT_OK(HostPortToPB(copy_source_addr, req.mutable_copy_peer_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 746c012..f136f02 100644
--- a/src/kudu/integration-tests/tablet_copy-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy-itest.cc
@@ -425,7 +425,8 @@ TEST_F(TabletCopyITest, TestCopyAfterFailedCopy) {
   ASSERT_OK(DeleteTabletWithRetries(follower_ts, tablet_id,
                                     TabletDataState::TABLET_DATA_TOMBSTONED,
                                     kTimeout));
-  HostPort leader_addr = HostPortFromPB(leader_ts->registration.rpc_addresses(0));
+  HostPort leader_addr;
+  ASSERT_OK(HostPortFromPB(leader_ts->registration.rpc_addresses(0), &leader_addr));
 
   // Inject failures to the metadata and trigger the tablet copy. This will
   // cause the copy to fail.
@@ -1164,7 +1165,8 @@ TEST_F(TabletCopyITest, TestTabletCopyThrottling) {
   // ServiceUnavailable error.
   ASSERT_OK(cluster_->tablet_server(1)->Restart());
 
-  HostPort ts0_hostport = HostPortFromPB(ts0->registration.rpc_addresses(0));
+  HostPort ts0_hostport;
+  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &ts0_hostport));
 
   // 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 119a5bd..b2f7c47 100644
--- a/src/kudu/integration-tests/tablet_copy_client_session-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy_client_session-itest.cc
@@ -135,7 +135,8 @@ 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 = HostPortFromPB(ts0->registration.rpc_addresses(0));
+  HostPort src_addr;
+  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
 
   // Repeat the restart process several times.
   for (int i = 0; i < 3; i++) {
@@ -228,7 +229,8 @@ TEST_F(TabletCopyClientSessionITest, TestStartTabletCopy) {
     }
 
     // Run tablet copy.
-    HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
+    HostPort src_addr;
+    ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
     ASSERT_OK(StartTabletCopy(ts1, tablet_id, ts0->uuid(), src_addr,
                               std::numeric_limits<int64_t>::max(), kDefaultTimeout));
     ASSERT_OK(WaitUntilTabletRunning(ts1, tablet_id, kDefaultTimeout));
@@ -252,7 +254,8 @@ TEST_F(TabletCopyClientSessionITest, TestCopyFromCrashedSource) {
                               "fault_crash_on_handle_tc_fetch_data",
                               "1.0"));
 
-  HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
+  HostPort src_addr;
+  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
   ASSERT_OK(StartTabletCopy(ts1, tablet_id, ts0->uuid(), src_addr,
                             std::numeric_limits<int64_t>::max(), kDefaultTimeout));
 
@@ -309,7 +312,8 @@ TEST_F(TabletCopyClientSessionITest, TestTabletCopyWithBusySource) {
                                   &tablets, tablet::TabletStatePB::RUNNING));
   ASSERT_EQ(kNumTablets, tablets.size());
 
-  HostPort src_addr = HostPortFromPB(ts0->registration.rpc_addresses(0));
+  HostPort src_addr;
+  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
 
   vector<thread> threads;
   for (const auto& tablet : tablets) {
@@ -365,7 +369,8 @@ TEST_F(TabletCopyClientSessionITest, TestStopCopyOnClientDiskFailure) {
   });
 
   // Now kick off the tablet copies.
-  HostPort src_addr = HostPortFromPB(ts_src->registration.rpc_addresses(0));
+  HostPort src_addr;
+  ASSERT_OK(HostPortFromPB(ts_src->registration.rpc_addresses(0), &src_addr));
   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 7f77f03..1018215 100644
--- a/src/kudu/integration-tests/tombstoned_voting-stress-test.cc
+++ b/src/kudu/integration-tests/tombstoned_voting-stress-test.cc
@@ -299,7 +299,8 @@ TEST_F(TombstonedVotingStressTest, TestTombstonedVotingUnderStress) {
 
     // 2. Copy tablet.
     LOG(INFO) << "copying tablet...";
-    HostPort source_hp = HostPortFromPB(ts0_ets->registration.rpc_addresses(0));
+    HostPort source_hp;
+    ASSERT_OK(HostPortFromPB(ts0_ets->registration.rpc_addresses(0), &source_hp));
     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 fd8acbc..b03a011 100644
--- a/src/kudu/integration-tests/ts_tablet_manager-itest.cc
+++ b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
@@ -965,7 +965,8 @@ TEST_F(TsTabletManagerITest, TestDeleteTableDuringTabletCopy) {
                                            tablet::TabletDataState::TABLET_DATA_TOMBSTONED,
                                            kTimeout));
   // Copy tablet from leader_ts to follower_ts.
-  HostPort leader_addr = HostPortFromPB(leader_ts->registration.rpc_addresses(0));
+  HostPort leader_addr;
+  ASSERT_OK(HostPortFromPB(leader_ts->registration.rpc_addresses(0), &leader_addr));
   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 3c48797..122dd14 100644
--- a/src/kudu/master/auto_rebalancer.cc
+++ b/src/kudu/master/auto_rebalancer.cc
@@ -449,7 +449,8 @@ 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);
-      *change->mutable_peer()->mutable_last_known_addr() = HostPortToPB(hp);
+      RETURN_NOT_OK(
+          HostPortToPB(hp, change->mutable_peer()->mutable_last_known_addr()));
     }
 
     // Request movement or replacement of the replica.
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index e64b179..f55cc6a 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -344,9 +344,11 @@ 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;
-    Status s = GetMasterEntryForHost(messenger_, hp, &peer_entry);
+    HostPort hp;
+    Status s = HostPortFromPB(peer.last_known_addr(), &hp).AndThen([&] {
+      return 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 46aa8b9..9011cd0 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -228,7 +228,9 @@ Status SysCatalogTable::Load(FsManager *fs_manager) {
     }
     set<string> peer_addrs_from_disk;
     for (const auto& p : cstate.committed_config().peers()) {
-      peer_addrs_from_disk.insert(HostPortFromPB(p.last_known_addr()).ToString());
+      HostPort hp;
+      RETURN_NOT_OK(HostPortFromPB(p.last_known_addr(), &hp));
+      peer_addrs_from_disk.insert(hp.ToString());
     }
     vector<string> symm_diff;
     std::set_symmetric_difference(peer_addrs_from_opts.begin(),
@@ -305,7 +307,8 @@ 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 = HostPortToPB(host_port);
+    HostPortPB peer_host_port_pb;
+    RETURN_NOT_OK(HostPortToPB(host_port, &peer_host_port_pb));
     peer.mutable_last_known_addr()->CopyFrom(peer_host_port_pb);
     peer.set_member_type(RaftPeerPB::VOTER);
     new_config.add_peers()->CopyFrom(peer);
@@ -1004,7 +1007,7 @@ void SysCatalogTable::InitLocalRaftPeerPB() {
   Sockaddr addr = master_->first_rpc_address();
   HostPort hp;
   CHECK_OK(HostPortFromSockaddrReplaceWildcard(addr, &hp));
-  *local_peer_pb_.mutable_last_known_addr() = HostPortToPB(hp);
+  CHECK_OK(HostPortToPB(hp, local_peer_pb_.mutable_last_known_addr()));
 }
 
 } // namespace master
diff --git a/src/kudu/master/ts_descriptor.cc b/src/kudu/master/ts_descriptor.cc
index 712df27..90f035d 100644
--- a/src/kudu/master/ts_descriptor.cc
+++ b/src/kudu/master/ts_descriptor.cc
@@ -93,8 +93,12 @@ 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++) {
-    hostports1.emplace(HostPortFromPB(pb1.Get(i)));
-    hostports2.emplace(HostPortFromPB(pb2.Get(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);
   }
   return hostports1 == hostports2;
 }
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index ee6529c..c249286 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -1377,7 +1377,9 @@ void ExternalDaemon::CheckForLeaks() {
 HostPort ExternalDaemon::bound_rpc_hostport() const {
   CHECK(status_);
   CHECK_GE(status_->bound_rpc_addresses_size(), 1);
-  return HostPortFromPB(status_->bound_rpc_addresses(0));
+  HostPort ret;
+  CHECK_OK(HostPortFromPB(status_->bound_rpc_addresses(0), &ret));
+  return ret;
 }
 
 Sockaddr ExternalDaemon::bound_rpc_addr() const {
@@ -1393,7 +1395,9 @@ HostPort ExternalDaemon::bound_http_hostport() const {
   if (status_->bound_http_addresses_size() == 0) {
     return HostPort();
   }
-  return HostPortFromPB(status_->bound_http_addresses(0));
+  HostPort ret;
+  CHECK_OK(HostPortFromPB(status_->bound_http_addresses(0), &ret));
+  return ret;
 }
 
 const NodeInstancePB& ExternalDaemon::instance_id() const {
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index d8a12bf..610d51d 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -34,7 +34,6 @@
 #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"
@@ -226,6 +225,9 @@ using std::vector;
 using strings::Substitute;
 
 namespace kudu {
+
+class HostPortPB;
+
 namespace server {
 
 namespace {
@@ -587,7 +589,9 @@ Status ServerBase::GetStatusPB(ServerStatusPB* status) const {
       HostPort hp;
       RETURN_NOT_OK_PREPEND(HostPortFromSockaddrReplaceWildcard(addr, &hp),
                             "could not get RPC hostport");
-      *status->add_bound_rpc_addresses() = HostPortToPB(hp);
+      HostPortPB* pb = status->add_bound_rpc_addresses();
+      RETURN_NOT_OK_PREPEND(HostPortToPB(hp, pb),
+                            "could not convert RPC hostport");
     }
   }
 
@@ -600,7 +604,9 @@ Status ServerBase::GetStatusPB(ServerStatusPB* status) const {
       HostPort hp;
       RETURN_NOT_OK_PREPEND(HostPortFromSockaddrReplaceWildcard(addr, &hp),
                             "could not get web hostport");
-      *status->add_bound_http_addresses() = HostPortToPB(hp);
+      HostPortPB* pb = status->add_bound_http_addresses();
+      RETURN_NOT_OK_PREPEND(HostPortToPB(hp, pb),
+                            "could not convert web hostport");
     }
   }
 
diff --git a/src/kudu/tools/ksck_remote.cc b/src/kudu/tools/ksck_remote.cc
index 7627aca..6414e84 100644
--- a/src/kudu/tools/ksck_remote.cc
+++ b/src/kudu/tools/ksck_remote.cc
@@ -569,7 +569,8 @@ Status RemoteKsckCluster::RetrieveTabletServers() {
     if (!ts_pb.has_registration()) {
       continue;
     }
-    HostPort hp = HostPortFromPB(ts_pb.registration().rpc_addresses(0));
+    HostPort hp;
+    RETURN_NOT_OK(HostPortFromPB(ts_pb.registration().rpc_addresses(0), &hp));
     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 09bfd31..4bc44c5 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -4704,7 +4704,8 @@ TEST_P(ControlShellToolTest, TestControlShell) {
   {
     KuduClientBuilder client_builder;
     for (const auto& e : masters) {
-      HostPort hp = HostPortFromPB(e.bound_rpc_address());
+      HostPort hp;
+      ASSERT_OK(HostPortFromPB(e.bound_rpc_address(), &hp));
       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 31b3118..67a2d1a 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -304,7 +304,8 @@ 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 = HostPortToPB(p.second);
+    HostPortPB new_peer_host_port_pb;
+    RETURN_NOT_OK(HostPortToPB(p.second, &new_peer_host_port_pb));
     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 2c3202a..3a56c0b 100644
--- a/src/kudu/tools/tool_action_test.cc
+++ b/src/kudu/tools/tool_action_test.cc
@@ -241,7 +241,8 @@ Status ProcessRequest(const ControlShellRequestPB& req,
     {
       RETURN_NOT_OK(CheckClusterExists(*cluster));
       for (int i = 0; i < (*cluster)->num_masters(); i++) {
-        HostPortPB pb = HostPortToPB((*cluster)->master(i)->bound_rpc_hostport());
+        HostPortPB pb;
+        RETURN_NOT_OK(HostPortToPB((*cluster)->master(i)->bound_rpc_hostport(), &pb));
         DaemonInfoPB* info = resp->mutable_get_masters()->mutable_masters()->Add();
         info->mutable_id()->set_type(MASTER);
         info->mutable_id()->set_index(i);
@@ -253,7 +254,8 @@ Status ProcessRequest(const ControlShellRequestPB& req,
     {
       RETURN_NOT_OK(CheckClusterExists(*cluster));
       for (int i = 0; i < (*cluster)->num_tablet_servers(); i++) {
-        HostPortPB pb = HostPortToPB((*cluster)->tablet_server(i)->bound_rpc_hostport());
+        HostPortPB pb;
+        RETURN_NOT_OK(HostPortToPB((*cluster)->tablet_server(i)->bound_rpc_hostport(), &pb));
         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 0a23cc8..97cad58 100644
--- a/src/kudu/tools/tool_replica_util.cc
+++ b/src/kudu/tools/tool_replica_util.cc
@@ -28,7 +28,6 @@
 
 #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"
@@ -592,7 +591,8 @@ 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));
-    *change->mutable_peer()->mutable_last_known_addr() = HostPortToPB(hp);
+    RETURN_NOT_OK(
+        HostPortToPB(hp, change->mutable_peer()->mutable_last_known_addr()));
   }
   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));
-    *peer_pb.mutable_last_known_addr() = HostPortToPB(hp);
+    RETURN_NOT_OK(HostPortToPB(hp, peer_pb.mutable_last_known_addr()));
   }
 
   // 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 2f10c1b..943966b 100644
--- a/src/kudu/tserver/tablet_copy_client-test.cc
+++ b/src/kudu/tserver/tablet_copy_client-test.cc
@@ -137,7 +137,8 @@ class TabletCopyClientTest : public TabletCopyTest {
 
   // Starts the tablet copy.
   Status StartCopy() {
-    HostPort host_port = HostPortFromPB(leader_.last_known_addr());
+    HostPort host_port;
+    RETURN_NOT_OK(HostPortFromPB(leader_.last_known_addr(), &host_port));
     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 2a6ff2b..706c2e5 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -626,7 +626,8 @@ 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 = HostPortFromPB(req->copy_peer_addr());
+  HostPort copy_source_addr;
+  CALLBACK_RETURN_NOT_OK(HostPortFromPB(req->copy_peer_addr(), &copy_source_addr));
   int64_t leader_term = req->caller_term();
 
   scoped_refptr<TabletReplica> old_replica;
@@ -1335,7 +1336,7 @@ void TSTabletManager::InitLocalRaftPeerPB() {
   Sockaddr addr = server_->first_rpc_address();
   HostPort hp;
   CHECK_OK(HostPortFromSockaddrReplaceWildcard(addr, &hp));
-  *local_peer_pb_.mutable_last_known_addr() = HostPortToPB(hp);
+  CHECK_OK(HostPortToPB(hp, local_peer_pb_.mutable_last_known_addr()));
 }
 
 void TSTabletManager::CreateReportedTabletPB(const scoped_refptr<TabletReplica>& replica,