You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2016/12/23 00:47:33 UTC

[4/6] kudu git commit: KUDU-1812. Replace PB DebugString calls with redactable variants

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index 86639d3..647fecb 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -41,6 +41,7 @@
 #include "kudu/integration-tests/test_workload.h"
 #include "kudu/integration-tests/ts_itest-base.h"
 #include "kudu/server/server_base.pb.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util.h"
 
@@ -122,9 +123,9 @@ class RaftConsensusITest : public TabletServerIntegrationTestBase {
     // Send the call
     {
       req.set_batch_size_bytes(0);
-      SCOPED_TRACE(req.DebugString());
+      SCOPED_TRACE(SecureDebugString(req));
       ASSERT_OK(replica_proxy->Scan(req, &resp, &rpc));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
       if (resp.has_error()) {
         ASSERT_OK(StatusFromPB(resp.error().status()));
       }
@@ -483,7 +484,7 @@ TEST_F(RaftConsensusITest, TestFailedTransaction) {
   controller.set_timeout(MonoDelta::FromSeconds(FLAGS_rpc_timeout));
 
   ASSERT_OK(DCHECK_NOTNULL(leader->tserver_proxy.get())->Write(req, &resp, &controller));
-  SCOPED_TRACE(resp.ShortDebugString());
+  SCOPED_TRACE(SecureShortDebugString(resp));
   ASSERT_FALSE(resp.has_error());
 
   ASSERT_ALL_REPLICAS_AGREE(1);
@@ -552,7 +553,7 @@ TEST_F(RaftConsensusITest, TestInsertOnNonLeader) {
   GetOnlyLiveFollowerReplicas(tablet_id_, &followers);
 
   ASSERT_OK(followers[0]->tserver_proxy->Write(req, &resp, &rpc));
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   ASSERT_TRUE(resp.has_error());
   Status s = StatusFromPB(resp.error().status());
   EXPECT_TRUE(s.IsIllegalState());
@@ -628,7 +629,7 @@ void RaftConsensusITest::Write128KOpsToLeader(int num_writes) {
     key++;
     ASSERT_OK(leader->tserver_proxy->Write(req, &resp, &rpc));
 
-    ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+    ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
   }
 }
 
@@ -1190,7 +1191,7 @@ void RaftConsensusITest::StubbornlyWriteSameRowThread(int replica_idx, const Ato
     rpc.set_timeout(MonoDelta::FromSeconds(10));
     ignore_result(ts->tserver_proxy->Write(req, &resp, &rpc));
     VLOG(1) << "Response from server " << replica_idx << ": "
-            << resp.ShortDebugString();
+            << SecureShortDebugString(resp);
   }
 }
 
@@ -1253,7 +1254,7 @@ void RaftConsensusITest::AddOpWithTypeAndKey(const OpId& id,
   CHECK_OK(SchemaToPB(schema_, write_req->mutable_schema()));
   write_req->set_tablet_id(tablet_id_);
   AddTestRowToPB(op_type, schema_, key, id.term(),
-                 id.ShortDebugString(), write_req->mutable_row_operations());
+                 SecureShortDebugString(id), write_req->mutable_row_operations());
 }
 
 void RaftConsensusITest::SetupSingleReplicaTest(TServerDetails** replica_ts) {
@@ -1303,7 +1304,7 @@ TEST_F(RaftConsensusITest, TestLMPMismatchOnRestartedReplica) {
   req.mutable_preceding_id()->CopyFrom(MakeOpId(1, 1));
 
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Send operations 2.1 through 2.3, committing through 2.2.
   AddOp(MakeOpId(2, 1), &req);
@@ -1312,7 +1313,7 @@ TEST_F(RaftConsensusITest, TestLMPMismatchOnRestartedReplica) {
   req.set_committed_index(2);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // The COMMIT messages end up in the WAL asynchronously, so loop reading the
   // tablet server's WAL until it shows up.
@@ -1339,9 +1340,10 @@ TEST_F(RaftConsensusITest, TestLMPMismatchOnRestartedReplica) {
       rpc.Reset();
       ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
       ASSERT_EQ(resp.status().error().code(),
-                consensus::ConsensusErrorPB::PRECEDING_ENTRY_DIDNT_MATCH) << resp.DebugString();
+                consensus::ConsensusErrorPB::PRECEDING_ENTRY_DIDNT_MATCH)
+          << SecureDebugString(resp);
     });
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   EXPECT_EQ(2, resp.status().last_committed_idx());
   EXPECT_EQ("0.0", OpIdToString(resp.status().last_received_current_leader()));
   // Even though the replica previously received operations through 2.3, the LMP mismatch
@@ -1379,7 +1381,7 @@ TEST_F(RaftConsensusITest, TestReplaceOperationStuckInPrepareQueue) {
   req.set_committed_index(2);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Replace operation 2.4 with 3.4, add 3.5 (upsert of a new key)
   req.set_caller_term(3);
@@ -1390,7 +1392,7 @@ TEST_F(RaftConsensusITest, TestReplaceOperationStuckInPrepareQueue) {
   rpc.Reset();
   rpc.set_timeout(MonoDelta::FromSeconds(5));
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Commit all ops.
   req.clear_ops();
@@ -1398,7 +1400,7 @@ TEST_F(RaftConsensusITest, TestReplaceOperationStuckInPrepareQueue) {
   req.mutable_preceding_id()->CopyFrom(MakeOpId(3, 5));
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Ensure we can read the data.
   // We need to AssertEventually here because otherwise it's possible to read the old value
@@ -1444,7 +1446,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
   req.mutable_preceding_id()->CopyFrom(MakeOpId(1, 1));
 
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Send some operations, but don't advance the commit index.
   // They should not commit.
@@ -1453,7 +1455,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
   AddOp(MakeOpId(2, 4), &req);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // We shouldn't read anything yet, because the ops should be pending.
   {
@@ -1469,7 +1471,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
   AddOp(MakeOpId(2, 6), &req);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_TRUE(resp.has_error()) << resp.DebugString();
+  ASSERT_TRUE(resp.has_error()) << SecureDebugString(resp);
   ASSERT_EQ(resp.error().status().message(),
             "New operation's index does not follow the previous op's index. "
             "Current: 2.6. Previous: 2.4");
@@ -1483,7 +1485,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
   AddOp(MakeOpId(2, 6), &req);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_TRUE(resp.has_error()) << resp.DebugString();
+  ASSERT_TRUE(resp.has_error()) << SecureDebugString(resp);
   ASSERT_EQ(resp.error().status().message(),
             "New operation's term is not >= than the previous op's term."
             " Current: 2.6. Previous: 3.5");
@@ -1502,7 +1504,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
   req.set_committed_index(4);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
   // Verify only 2.2 and 2.3 are committed.
   {
     vector<string> results;
@@ -1520,7 +1522,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
   AddOp(MakeOpId(2, 6), &req);
   rpc.Reset();
   ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Verify they are committed.
   {
@@ -1547,9 +1549,9 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
     // Send the call. We expect to get a timeout passed back from the server side
     // (i.e. not an RPC timeout)
     req.set_batch_size_bytes(0);
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(replica_ts->tserver_proxy->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     string err_str = StatusFromPB(resp.error().status()).ToString();
     ASSERT_STR_CONTAINS(err_str, "Timed out waiting for ts:");
     ASSERT_STR_CONTAINS(err_str, "to be safe");
@@ -1571,8 +1573,8 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
     AddOp(MakeOpId(leader_term, 6), &req);
     rpc.Reset();
     ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-    ASSERT_FALSE(resp.has_error()) << "Req: " << req.ShortDebugString()
-        << " Resp: " << resp.DebugString();
+    ASSERT_FALSE(resp.has_error()) << "Req: " << SecureShortDebugString(req)
+        << " Resp: " << SecureDebugString(resp);
   }
 
   // Send an empty request from the newest term which should commit
@@ -1583,7 +1585,7 @@ TEST_F(RaftConsensusITest, TestReplicaBehaviorViaRPC) {
     req.clear_ops();
     rpc.Reset();
     ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
-    ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+    ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
   }
 
   // Verify the new rows are committed.
@@ -1624,7 +1626,7 @@ TEST_F(RaftConsensusITest, TestLeaderStepDown) {
   TabletServerErrorPB error;
   s = LeaderStepDown(tservers[0], tablet_id_, MonoDelta::FromSeconds(10), &error);
   ASSERT_TRUE(s.IsIllegalState()) << "TS #0 should not be leader anymore: " << s.ToString();
-  ASSERT_EQ(TabletServerErrorPB::NOT_THE_LEADER, error.code()) << error.ShortDebugString();
+  ASSERT_EQ(TabletServerErrorPB::NOT_THE_LEADER, error.code()) << SecureShortDebugString(error);
 
   s = WriteSimpleTestRow(tservers[0], tablet_id_, RowOperationsPB::INSERT,
                          kTestRowKey, kTestRowIntVal, "foo", MonoDelta::FromSeconds(10));
@@ -1748,11 +1750,11 @@ Status RaftConsensusITest::GetTabletLocations(const string& tablet_id, const Mon
     return StatusFromPB(resp.error().status());
   }
   if (resp.errors_size() > 0) {
-    CHECK_EQ(1, resp.errors_size()) << resp.ShortDebugString();
-    CHECK_EQ(tablet_id, resp.errors(0).tablet_id()) << resp.ShortDebugString();
+    CHECK_EQ(1, resp.errors_size()) << SecureShortDebugString(resp);
+    CHECK_EQ(tablet_id, resp.errors(0).tablet_id()) << SecureShortDebugString(resp);
     return StatusFromPB(resp.errors(0).status());
   }
-  CHECK_EQ(1, resp.tablet_locations_size()) << resp.ShortDebugString();
+  CHECK_EQ(1, resp.tablet_locations_size()) << SecureShortDebugString(resp);
   *tablet_locations = resp.tablet_locations(0);
   return Status::OK();
 }
@@ -1782,9 +1784,10 @@ void RaftConsensusITest::WaitForReplicasReportedToMaster(
     if (deadline < MonoTime::Now()) break;
     SleepFor(MonoDelta::FromMilliseconds(20));
   }
-  ASSERT_EQ(num_replicas, tablet_locations->replicas_size()) << tablet_locations->DebugString();
+  ASSERT_EQ(num_replicas, tablet_locations->replicas_size())
+      << SecureDebugString(*tablet_locations);
   if (wait_for_leader == WAIT_FOR_LEADER) {
-    ASSERT_TRUE(*has_leader) << tablet_locations->DebugString();
+    ASSERT_TRUE(*has_leader) << SecureDebugString(*tablet_locations);
   }
 }
 
@@ -2257,7 +2260,7 @@ TEST_F(RaftConsensusITest, TestMasterNotifiedOnConfigChange) {
   bool has_leader;
   NO_FATALS(WaitForReplicasReportedToMaster(2, tablet_id, timeout, WAIT_FOR_LEADER,
                                             &has_leader, &tablet_locations));
-  LOG(INFO) << "Tablet locations:\n" << tablet_locations.DebugString();
+  LOG(INFO) << "Tablet locations:\n" << SecureDebugString(tablet_locations);
 
   // Wait for initial NO_OP to be committed by the leader.
   TServerDetails* leader_ts;
@@ -2276,8 +2279,8 @@ TEST_F(RaftConsensusITest, TestMasterNotifiedOnConfigChange) {
   LOG(INFO) << "Waiting for Master to see config change...";
   NO_FATALS(WaitForReplicasReportedToMaster(3, tablet_id, timeout, NO_WAIT_FOR_LEADER,
                                             &has_leader, &tablet_locations));
-  ASSERT_TRUE(has_leader) << tablet_locations.DebugString();
-  LOG(INFO) << "Tablet locations:\n" << tablet_locations.DebugString();
+  ASSERT_TRUE(has_leader) << SecureDebugString(tablet_locations);
+  LOG(INFO) << "Tablet locations:\n" << SecureDebugString(tablet_locations);
 
   // Change the config again.
   LOG(INFO) << "Removing tserver with uuid " << tserver_to_add->uuid();
@@ -2290,8 +2293,8 @@ TEST_F(RaftConsensusITest, TestMasterNotifiedOnConfigChange) {
   LOG(INFO) << "Waiting for Master to see config change...";
   NO_FATALS(WaitForReplicasReportedToMaster(2, tablet_id, timeout, NO_WAIT_FOR_LEADER,
                                             &has_leader, &tablet_locations));
-  ASSERT_TRUE(has_leader) << tablet_locations.DebugString();
-  LOG(INFO) << "Tablet locations:\n" << tablet_locations.DebugString();
+  ASSERT_TRUE(has_leader) << SecureDebugString(tablet_locations);
+  LOG(INFO) << "Tablet locations:\n" << SecureDebugString(tablet_locations);
 }
 
 // Test that even with memory pressure, a replica will still commit pending
@@ -2347,7 +2350,7 @@ TEST_F(RaftConsensusITest, TestEarlyCommitDespiteMemoryPressure) {
 
   // At the time that the follower received our request it was still under the
   // tiny memory limit defined above, so the request should have succeeded.
-  ASSERT_FALSE(resp.has_error()) << resp.DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
   ASSERT_TRUE(resp.has_status());
   ASSERT_TRUE(resp.status().has_last_committed_idx());
   ASSERT_EQ(last_opid.index(), resp.status().last_received().index());
@@ -2916,10 +2919,10 @@ TEST_F(RaftConsensusITest, TestUpdateConsensusErrorNonePrepared) {
   }
 
   ASSERT_OK(replica_ts->consensus_proxy->UpdateConsensus(req, &resp, &rpc));
-  LOG(INFO) << resp.ShortDebugString();
+  LOG(INFO) << SecureShortDebugString(resp);
   ASSERT_TRUE(resp.status().has_error());
   ASSERT_EQ(consensus::ConsensusErrorPB::CANNOT_PREPARE, resp.status().error().code());
-  ASSERT_STR_CONTAINS(resp.ShortDebugString(), "Could not prepare a single transaction");
+  ASSERT_STR_CONTAINS(SecureShortDebugString(resp), "Could not prepare a single transaction");
 }
 
 // Test that, if the raft metadata on a replica is corrupt, then the server

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/integration-tests/registration-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index 255c152..4c35c0a 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -34,8 +34,9 @@
 #include "kudu/tserver/tablet_server.h"
 #include "kudu/util/curl_util.h"
 #include "kudu/util/faststring.h"
-#include "kudu/util/test_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
+#include "kudu/util/test_util.h"
 #include "kudu/util/version_info.h"
 
 DECLARE_int32(heartbeat_interval_ms);
@@ -123,8 +124,8 @@ TEST_F(RegistrationTest, TestTSRegisters) {
   ServerRegistrationPB reg;
   descs[0]->GetRegistration(&reg);
   {
-    SCOPED_TRACE(reg.ShortDebugString());
-    ASSERT_EQ(reg.ShortDebugString().find("0.0.0.0"), string::npos)
+    SCOPED_TRACE(SecureShortDebugString(reg));
+    ASSERT_EQ(SecureShortDebugString(reg).find("0.0.0.0"), string::npos)
       << "Should not include wildcards in registration";
   }
 
@@ -146,7 +147,7 @@ TEST_F(RegistrationTest, TestMasterSoftwareVersion) {
   ServerRegistrationPB reg;
   cluster_->mini_master()->master()->GetMasterRegistration(&reg);
   {
-    SCOPED_TRACE(reg.ShortDebugString());
+    SCOPED_TRACE(SecureShortDebugString(reg));
     ASSERT_TRUE(reg.has_software_version());
     ASSERT_STR_CONTAINS(reg.software_version(),
                         VersionInfo::GetShortVersionString());

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/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 385b2a5..775ba25 100644
--- a/src/kudu/integration-tests/table_locations-itest.cc
+++ b/src/kudu/integration-tests/table_locations-itest.cc
@@ -31,6 +31,7 @@
 #include "kudu/master/master.proxy.h"
 #include "kudu/master/mini_master.h"
 #include "kudu/rpc/messenger.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_util.h"
 
 using kudu::rpc::Messenger;
@@ -149,7 +150,7 @@ TEST_F(TableLocationsTest, TestGetTableLocations) {
 
       controller.Reset();
       ASSERT_OK(proxy_->GetTableLocations(req, &resp, &controller));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
 
       if (resp.has_error()) {
         ASSERT_EQ(MasterErrorPB::TABLET_NOT_RUNNING, resp.error().code());
@@ -169,7 +170,7 @@ TEST_F(TableLocationsTest, TestGetTableLocations) {
     req.set_partition_key_start("a");
     req.set_max_returned_locations(3);
     ASSERT_OK(proxy_->GetTableLocations(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
 
     ASSERT_TRUE(!resp.has_error());
     ASSERT_EQ(3, resp.tablet_locations().size());
@@ -186,7 +187,7 @@ TEST_F(TableLocationsTest, TestGetTableLocations) {
     req.set_partition_key_start("");
     req.set_max_returned_locations(3);
     ASSERT_OK(proxy_->GetTableLocations(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
 
     ASSERT_TRUE(!resp.has_error());
     ASSERT_EQ(3, resp.tablet_locations().size());
@@ -203,7 +204,7 @@ TEST_F(TableLocationsTest, TestGetTableLocations) {
     req.set_partition_key_start("b");
     req.set_max_returned_locations(3);
     ASSERT_OK(proxy_->GetTableLocations(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
 
     ASSERT_TRUE(!resp.has_error());
     ASSERT_EQ(3, resp.tablet_locations().size());
@@ -220,7 +221,7 @@ TEST_F(TableLocationsTest, TestGetTableLocations) {
     req.set_partition_key_start("z");
     req.set_max_returned_locations(3);
     ASSERT_OK(proxy_->GetTableLocations(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
 
     ASSERT_TRUE(!resp.has_error());
     ASSERT_EQ(1, resp.tablet_locations().size());

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/integration-tests/ts_itest-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_itest-base.h b/src/kudu/integration-tests/ts_itest-base.h
index 35ded6a..844369c 100644
--- a/src/kudu/integration-tests/ts_itest-base.h
+++ b/src/kudu/integration-tests/ts_itest-base.h
@@ -32,6 +32,7 @@
 #include "kudu/integration-tests/external_mini_cluster_fs_inspector.h"
 #include "kudu/master/master.proxy.h"
 #include "kudu/tserver/tablet_server-test-base.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/random.h"
 #include "kudu/util/test_util.h"
 
@@ -151,7 +152,7 @@ class TabletServerIntegrationTestBase : public TabletServerTestBase {
           SleepFor(MonoDelta::FromSeconds(1));
           continue;
         }
-        FAIL() << "Response had a fatal error: " << resp.error().ShortDebugString();
+        FAIL() << "Response had a fatal error: " << SecureShortDebugString(resp.error());
       }
 
       for (const master::TabletLocationsPB& location : resp.tablet_locations()) {
@@ -162,7 +163,7 @@ class TabletServerIntegrationTestBase : public TabletServerTestBase {
 
         if (tablet_replicas.count(location.tablet_id()) < FLAGS_num_replicas) {
           LOG(WARNING)<< "Couldn't find the leader and/or replicas. Location: "
-              << location.ShortDebugString();
+              << SecureShortDebugString(location);
           replicas_missing = true;
           SleepFor(MonoDelta::FromSeconds(1));
           break;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/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 a963eb9..2a3f0d3 100644
--- a/src/kudu/integration-tests/ts_tablet_manager-itest.cc
+++ b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
@@ -40,6 +40,7 @@
 #include "kudu/tserver/tserver_admin.proxy.h"
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/tserver/ts_tablet_manager.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_util.h"
 
 DECLARE_bool(enable_leader_failure_detection);
@@ -177,7 +178,8 @@ TEST_F(TsTabletManagerITest, TestReportNewLeaderOnLeaderChange) {
 
       // Ensure that our tablet reports are consistent.
       TabletReportPB& report = reports[0];
-      ASSERT_EQ(1, report.updated_tablets_size()) << "Wrong report size:\n" << report.DebugString();
+      ASSERT_EQ(1, report.updated_tablets_size())
+          << "Wrong report size:\n" << SecureDebugString(report);
       ReportedTabletPB reported_tablet = report.updated_tablets(0);
       ASSERT_TRUE(reported_tablet.has_committed_consensus_state());
 
@@ -185,10 +187,10 @@ TEST_F(TsTabletManagerITest, TestReportNewLeaderOnLeaderChange) {
       RaftPeerPB::Role role = GetConsensusRole(uuid, reported_tablet.committed_consensus_state());
       if (replica == new_leader_idx) {
         ASSERT_EQ(RaftPeerPB::LEADER, role)
-            << "Tablet report: " << report.ShortDebugString();
+            << "Tablet report: " << SecureShortDebugString(report);
       } else {
         ASSERT_EQ(RaftPeerPB::FOLLOWER, role)
-            << "Tablet report: " << report.ShortDebugString();
+            << "Tablet report: " << SecureShortDebugString(report);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index c360730..d2ea3c6 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -85,6 +85,7 @@
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/random_util.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/stopwatch.h"
@@ -250,7 +251,8 @@ class TableLoader : public TableVisitor {
     l.Commit();
 
     LOG(INFO) << "Loaded metadata for table " << table->ToString();
-    VLOG(1) << "Metadata for table " << table->ToString() << ": " << metadata.ShortDebugString();
+    VLOG(1) << "Metadata for table " << table->ToString()
+            << ": " << SecureShortDebugString(metadata);
     return Status::OK();
   }
 
@@ -280,7 +282,7 @@ class TabletLoader : public TabletVisitor {
       // Tables and tablets are always created/deleted in one operation, so
       // this shouldn't be possible.
       LOG(ERROR) << "Missing Table " << table_id << " required by tablet " << tablet_id;
-      LOG(ERROR) << "Metadata: " << metadata.DebugString();
+      LOG(ERROR) << "Metadata: " << SecureDebugString(metadata);
       return Status::Corruption("Missing table for tablet: ", tablet_id);
     }
 
@@ -301,7 +303,7 @@ class TabletLoader : public TabletVisitor {
 
     LOG(INFO) << "Loaded metadata for tablet " << tablet_id
               << " (table " << table->ToString() << ")";
-    VLOG(2) << "Metadata for tablet " << tablet_id << ": " << metadata.ShortDebugString();
+    VLOG(2) << "Metadata for tablet " << tablet_id << ": " << SecureShortDebugString(metadata);
     return Status::OK();
   }
 
@@ -657,7 +659,7 @@ Status CatalogManager::WaitUntilCaughtUpAsLeader(const MonoDelta& timeout) {
   if (!cstate.has_leader_uuid() || cstate.leader_uuid() != uuid) {
     return Status::IllegalState(
         Substitute("Node $0 not leader. Consensus state: $1",
-                    uuid, cstate.ShortDebugString()));
+                    uuid, SecureShortDebugString(cstate)));
   }
 
   // Wait for all transactions to be committed.
@@ -915,7 +917,7 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   // Copy the request, so we can fill in some defaults.
   CreateTableRequestPB req = *orig_req;
   LOG(INFO) << "CreateTable from " << RequestorString(rpc)
-            << ":\n" << req.DebugString();
+            << ":\n" << SecureDebugString(req);
 
   // Do some fix-up of any defaults specified on columns.
   // Clients are only expected to pass the default value in the 'read_default'
@@ -1152,7 +1154,7 @@ Status CatalogManager::IsCreateTableDone(const IsCreateTableDoneRequestPB* req,
   TRACE("Looking up table");
   RETURN_NOT_OK(FindTable(req->table(), &table));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", req->table().ShortDebugString());
+    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
     SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
     return s;
   }
@@ -1219,14 +1221,14 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB* req,
   RETURN_NOT_OK(CheckOnline());
 
   LOG(INFO) << "Servicing DeleteTable request from " << RequestorString(rpc)
-            << ": " << req->ShortDebugString();
+            << ": " << SecureShortDebugString(*req);
 
   // 1. Look up the table, lock it, and mark it as removed.
   TRACE("Looking up table");
   scoped_refptr<TableInfo> table;
   RETURN_NOT_OK(FindTable(req->table(), &table));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", req->table().ShortDebugString());
+    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
     SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
     return s;
   }
@@ -1327,7 +1329,7 @@ Status CatalogManager::ApplyAlterSchemaSteps(const SysTablesEntryPB& current_pb,
         ColumnSchemaPB new_col_pb = step.add_column().schema();
         if (new_col_pb.has_id()) {
           return Status::InvalidArgument("column $0: client should not specify column ID",
-                                         new_col_pb.ShortDebugString());
+                                         SecureShortDebugString(new_col_pb));
         }
         RETURN_NOT_OK(ProcessColumnPBDefaults(&new_col_pb));
 
@@ -1374,7 +1376,8 @@ Status CatalogManager::ApplyAlterSchemaSteps(const SysTablesEntryPB& current_pb,
       // TODO: EDIT_COLUMN
 
       default: {
-        return Status::InvalidArgument("Invalid alter schema step type", step.ShortDebugString());
+        return Status::InvalidArgument("Invalid alter schema step type",
+                                       SecureShortDebugString(step));
       }
     }
   }
@@ -1414,7 +1417,7 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
 
     if (ops.size() != 2) {
       return Status::InvalidArgument("expected two row operations for alter range partition step",
-                                     step.ShortDebugString());
+                                     SecureShortDebugString(step));
     }
 
     if ((ops[0].type != RowOperationsPB::RANGE_LOWER_BOUND &&
@@ -1536,7 +1539,7 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
       }
       default: {
         return Status::InvalidArgument("Unknown alter table range partitioning step",
-                                       step.ShortDebugString());
+                                       SecureShortDebugString(step));
       }
     }
   }
@@ -1554,7 +1557,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   RETURN_NOT_OK(CheckOnline());
 
   LOG(INFO) << "Servicing AlterTable request from " << RequestorString(rpc)
-            << ": " << req->ShortDebugString();
+            << ": " << SecureShortDebugString(*req);
 
   RETURN_NOT_OK(CheckOnline());
 
@@ -1576,7 +1579,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
       }
       case AlterTableRequestPB::ALTER_COLUMN:
       case AlterTableRequestPB::UNKNOWN: {
-        return Status::InvalidArgument("Invalid alter step type", step.ShortDebugString());
+        return Status::InvalidArgument("Invalid alter step type", SecureShortDebugString(step));
       }
     }
   }
@@ -1586,7 +1589,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   scoped_refptr<TableInfo> table;
   RETURN_NOT_OK(FindTable(req->table(), &table));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", req->table().ShortDebugString());
+    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
     SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
     return s;
   }
@@ -1605,7 +1608,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
     scoped_refptr<TableInfo> table_again;
     CHECK_OK(FindTable(req->table(), &table_again));
     if (table_again == nullptr) {
-      Status s = Status::NotFound("The table does not exist", req->table().ShortDebugString());
+      Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
       SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
       return s;
     }
@@ -1831,7 +1834,7 @@ Status CatalogManager::IsAlterTableDone(const IsAlterTableDoneRequestPB* req,
   TRACE("Looking up table");
   RETURN_NOT_OK(FindTable(req->table(), &table));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", req->table().ShortDebugString());
+    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
     SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
     return s;
   }
@@ -1859,7 +1862,7 @@ Status CatalogManager::GetTableSchema(const GetTableSchemaRequestPB* req,
   TRACE("Looking up table");
   RETURN_NOT_OK(FindTable(req->table(), &table));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", req->table().ShortDebugString());
+    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
     SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
     return s;
   }
@@ -1962,7 +1965,7 @@ Status CatalogManager::ProcessTabletReport(TSDescriptor* ts_desc,
 
   if (VLOG_IS_ON(2)) {
     VLOG(2) << "Received tablet report from " <<
-      RequestorString(rpc) << ": " << report.DebugString();
+      RequestorString(rpc) << ": " << SecureDebugString(report);
   }
 
   // TODO: on a full tablet report, we may want to iterate over the tablets we think
@@ -1973,7 +1976,7 @@ Status CatalogManager::ProcessTabletReport(TSDescriptor* ts_desc,
     ReportedTabletUpdatesPB *tablet_report = report_update->add_tablets();
     tablet_report->set_tablet_id(reported.tablet_id());
     RETURN_NOT_OK_PREPEND(HandleReportedTablet(ts_desc, reported, tablet_report),
-                          Substitute("Error handling $0", reported.ShortDebugString()));
+                          Substitute("Error handling $0", SecureShortDebugString(reported)));
   }
 
   if (report.updated_tablets_size() > 0) {
@@ -2033,7 +2036,7 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   }
   DCHECK(tablet->table()); // guaranteed by TabletLoader
 
-  VLOG(3) << "tablet report: " << report.ShortDebugString();
+  VLOG(3) << "tablet report: " << SecureShortDebugString(report);
 
   // TODO: we don't actually need to do the COW here until we see we're going
   // to change the state. Can we change CowedObject to lazily do the copy?
@@ -2131,7 +2134,7 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
     if (!tablet_lock.data().is_running() && ShouldTransitionTabletToRunning(report)) {
       DCHECK_EQ(SysTabletsEntryPB::CREATING, tablet_lock.data().pb.state())
           << "Tablet in unexpected state: " << tablet->ToString()
-          << ": " << tablet_lock.data().pb.ShortDebugString();
+          << ": " << SecureShortDebugString(tablet_lock.data().pb);
       // Mark the tablet as running
       // TODO: we could batch the IO onto a background thread, or at least
       // across multiple tablets in the same report.
@@ -2143,7 +2146,7 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
     // The Master only accepts committed consensus configurations since it needs the committed index
     // to only cache the most up-to-date config.
     if (PREDICT_FALSE(!cstate.config().has_opid_index())) {
-      LOG(DFATAL) << "Missing opid_index in reported config:\n" << report.DebugString();
+      LOG(DFATAL) << "Missing opid_index in reported config:\n" << SecureDebugString(report);
       return Status::InvalidArgument("Missing opid_index in reported config");
     }
 
@@ -2169,7 +2172,8 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
                                   "a different leader for term $1 than the current cstate. "
                                   "Previous cstate: $2. Current cstate: $3.",
                                   tablet->ToString(), cstate.current_term(),
-                                  prev_cstate.ShortDebugString(), cstate.ShortDebugString());
+                                  SecureShortDebugString(prev_cstate),
+                                  SecureShortDebugString(cstate));
           LOG(DFATAL) << msg;
           return Status::InvalidArgument(msg);
         }
@@ -2179,7 +2183,7 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
       // master's copy of that configuration.
       LOG(INFO) << "T " << tablet->tablet_id() << " reported consensus state change: "
                 << DiffConsensusStates(prev_cstate, cstate)
-                << ". New consensus state: " << cstate.ShortDebugString();
+                << ". New consensus state: " << SecureShortDebugString(cstate);
 
       // If we need to change the report, copy the whole thing on the stack
       // rather than const-casting.
@@ -2213,7 +2217,7 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   Status s = sys_catalog_->Write(actions);
   if (!s.ok()) {
     LOG(WARNING) << "Error updating tablets: " << s.ToString() << ". Tablet report was: "
-                 << report.ShortDebugString();
+                 << SecureShortDebugString(report);
     return s;
   }
   tablet_lock.Commit();
@@ -2660,7 +2664,7 @@ class AsyncCreateReplica : public RetrySpecificTSRpcTask {
     VLOG(1) << "Send create tablet request to "
             << target_ts_desc_->ToString() << ":\n"
             << " (attempt " << attempt << "):\n"
-            << req_.DebugString();
+            << SecureDebugString(req_);
     ts_proxy_->CreateTabletAsync(req_, &resp_, &rpc_,
                                  boost::bind(&AsyncCreateReplica::RpcCallback, this));
     return true;
@@ -2838,7 +2842,7 @@ class AsyncAlterTable : public RetryingTSRpcTask {
 
     VLOG(1) << "Send alter table request to " << target_ts_desc_->ToString()
             << " (attempt " << attempt << "):\n"
-            << req.DebugString();
+            << SecureDebugString(req);
     ts_proxy_->AlterSchemaAsync(req, &resp_, &rpc_,
                                 boost::bind(&AsyncAlterTable::RpcCallback, this));
     return true;
@@ -2951,7 +2955,7 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   peer->set_member_type(RaftPeerPB::VOTER);
   VLOG(1) << "Sending AddServer ChangeConfig request to "
           << target_ts_desc_->ToString() << ":\n"
-          << req_.DebugString();
+          << SecureDebugString(req_);
   consensus_proxy_->ChangeConfigAsync(req_, &resp_, &rpc_,
                                       boost::bind(&AsyncAddServerTask::RpcCallback, this));
   return true;
@@ -3444,7 +3448,7 @@ shared_ptr<TSDescriptor> CatalogManager::SelectReplica(
 void CatalogManager::SelectReplicas(const TSDescriptorVector& ts_descs,
                                     int nreplicas,
                                     consensus::RaftConfigPB *config) {
-  DCHECK_EQ(0, config->peers_size()) << "RaftConfig not empty: " << config->ShortDebugString();
+  DCHECK_EQ(0, config->peers_size()) << "RaftConfig not empty: " << SecureShortDebugString(*config);
   DCHECK_LE(nreplicas, ts_descs.size());
 
   // Keep track of servers we've already selected, so that we don't attempt to
@@ -3620,7 +3624,7 @@ void CatalogManager::DumpState(std::ostream* out) const {
     if (names_copy.erase(name) != 1) {
       *out << "  [not present in by-name map]\n";
     }
-    *out << "  metadata: " << l.data().pb.ShortDebugString() << "\n";
+    *out << "  metadata: " << SecureShortDebugString(l.data().pb) << "\n";
 
     *out << "  tablets:\n";
 
@@ -3629,7 +3633,7 @@ void CatalogManager::DumpState(std::ostream* out) const {
     for (const scoped_refptr<TabletInfo>& tablet : table_tablets) {
       TabletMetadataLock l_tablet(tablet.get(), TabletMetadataLock::READ);
       *out << "    " << tablet->tablet_id() << ": "
-           << l_tablet.data().pb.ShortDebugString() << "\n";
+           << SecureShortDebugString(l_tablet.data().pb) << "\n";
 
       if (tablets_copy.erase(tablet->tablet_id()) != 1) {
         *out << "  [ERROR: not present in CM tablet map!]\n";
@@ -3643,7 +3647,7 @@ void CatalogManager::DumpState(std::ostream* out) const {
       const scoped_refptr<TabletInfo>& tablet = entry.second;
       TabletMetadataLock l_tablet(tablet.get(), TabletMetadataLock::READ);
       *out << "    " << tablet->tablet_id() << ": "
-           << l_tablet.data().pb.ShortDebugString() << "\n";
+           << SecureShortDebugString(l_tablet.data().pb) << "\n";
     }
   }
 
@@ -3697,7 +3701,7 @@ CatalogManager::ScopedLeaderSharedLock::ScopedLeaderSharedLock(
   if (PREDICT_FALSE(!cstate.has_leader_uuid() || cstate.leader_uuid() != uuid)) {
     leader_status_ = Status::IllegalState(
         Substitute("Not the leader. Local UUID: $0, Consensus state: $1",
-                   uuid, cstate.ShortDebugString()));
+                   uuid, SecureShortDebugString(cstate)));
     return;
   }
   if (PREDICT_FALSE(catalog_->leader_ready_term_ != cstate.current_term() ||

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/master-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-path-handlers.cc b/src/kudu/master/master-path-handlers.cc
index f0ce997..efd5183 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -44,6 +44,7 @@
 #include "kudu/master/sys_catalog.h"
 #include "kudu/master/ts_descriptor.h"
 #include "kudu/master/ts_manager.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/string_case.h"
 #include "kudu/util/url-coding.h"
 
@@ -84,7 +85,7 @@ void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& req,
     string row = Substitute("<tr><th>$0</th><td>$1</td><td><pre><code>$2</code></pre></td></tr>\n",
                             RegistrationToHtml(reg, desc->permanent_uuid()),
                             time_since_hb,
-                            EscapeForHtmlToString(reg.ShortDebugString()));
+                            EscapeForHtmlToString(SecureShortDebugString(reg)));
 
     if (desc->PresumedDead()) {
       version_counts[reg.software_version()][1]++;
@@ -369,7 +370,7 @@ void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& req,
         master.registration(),
         master.instance_id().permanent_uuid());
     string reg_str = EscapeForHtmlToString(
-        master.registration().ShortDebugString());
+        SecureShortDebugString(master.registration()));
     *output << Substitute(
         "  <tr><td>$0</td><td>$1</td><td><code>$2</code></td></tr>\n",
         uuid_text,

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/master-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 80b4ecb..0899336 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -36,6 +36,7 @@
 #include "kudu/master/ts_manager.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/server/rpc_server.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_util.h"
 
@@ -167,7 +168,8 @@ TEST_F(MasterTest, TestRegisterAndHeartbeat) {
   ASSERT_EQ(1, descs.size()) << "Should have registered the TS";
   ServerRegistrationPB reg;
   descs[0]->GetRegistration(&reg);
-  ASSERT_EQ(fake_reg.DebugString(), reg.DebugString()) << "Master got different registration";
+  ASSERT_EQ(SecureDebugString(fake_reg), SecureDebugString(reg))
+      << "Master got different registration";
 
   ASSERT_TRUE(master_->ts_manager()->LookupTSByUUID(kTsUUID, &ts_desc));
   ASSERT_EQ(ts_desc, descs[0]);
@@ -272,7 +274,7 @@ TEST_F(MasterTest, TestRegisterAndHeartbeat) {
     ListTabletServersResponsePB resp;
     RpcController rpc;
     ASSERT_OK(proxy_->ListTabletServers(req, &resp, &rpc));
-    LOG(INFO) << resp.DebugString();
+    LOG(INFO) << SecureDebugString(resp);
     ASSERT_EQ(1, resp.servers_size());
     ASSERT_EQ("my-ts-uuid", resp.servers(0).instance_id().permanent_uuid());
     ASSERT_EQ(1, resp.servers(0).instance_id().instance_seqno());
@@ -353,7 +355,7 @@ Status MasterTest::CreateTable(const string& table_name,
 void MasterTest::DoListTables(const ListTablesRequestPB& req, ListTablesResponsePB* resp) {
   RpcController controller;
   ASSERT_OK(proxy_->ListTables(req, resp, &controller));
-  SCOPED_TRACE(resp->DebugString());
+  SCOPED_TRACE(SecureDebugString(*resp));
   ASSERT_FALSE(resp->has_error());
 }
 
@@ -384,7 +386,7 @@ TEST_F(MasterTest, TestCatalog) {
     RpcController controller;
     req.mutable_table()->set_table_name(kTableName);
     ASSERT_OK(proxy_->DeleteTable(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -607,10 +609,10 @@ TEST_F(MasterTest, TestCreateTableInvalidSchema) {
   }
 
   ASSERT_OK(proxy_->CreateTable(req, &resp, &controller));
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   ASSERT_TRUE(resp.has_error());
   ASSERT_EQ("code: INVALID_ARGUMENT message: \"Duplicate column name: col\"",
-            resp.error().status().ShortDebugString());
+            SecureShortDebugString(resp.error().status()));
 }
 
 // Test that, if the client specifies mismatched read and write defaults,
@@ -635,11 +637,11 @@ TEST_F(MasterTest, TestCreateTableMismatchedDefaults) {
   req.mutable_schema()->mutable_columns(1)->set_write_default_value("bye");
 
   ASSERT_OK(proxy_->CreateTable(req, &resp, &controller));
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   ASSERT_TRUE(resp.has_error());
   ASSERT_EQ("code: INVALID_ARGUMENT message: \"column \\'col\\' has "
             "mismatched read/write defaults\"",
-            resp.error().status().ShortDebugString());
+            SecureShortDebugString(resp.error().status()));
 }
 
 // Regression test for KUDU-253/KUDU-592: crash if the GetTableLocations RPC call is
@@ -657,11 +659,11 @@ TEST_F(MasterTest, TestInvalidGetTableLocations) {
     req.set_partition_key_start("zzzz");
     req.set_partition_key_end("aaaa");
     ASSERT_OK(proxy_->GetTableLocations(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ("code: INVALID_ARGUMENT message: "
               "\"start partition key is greater than the end partition key\"",
-              resp.error().status().ShortDebugString());
+              SecureShortDebugString(resp.error().status()));
   }
 }
 
@@ -731,7 +733,7 @@ TEST_F(MasterTest, TestGetTableSchemaIsAtomicWithCreateTable) {
       RpcController controller;
 
       CHECK_OK(proxy_->GetTableSchema(req, &resp, &controller));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
 
       // There are two possible outcomes:
       //
@@ -1084,7 +1086,7 @@ TEST_F(MasterTest, TestConcurrentCreateOfSameTable) {
       req.set_name(kTableName);
       CHECK_OK(SchemaToPB(kTableSchema, req.mutable_schema()));
       CHECK_OK(proxy_->CreateTable(req, &resp, &controller));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
 
       // There are three expected outcomes:
       //
@@ -1096,7 +1098,7 @@ TEST_F(MasterTest, TestConcurrentCreateOfSameTable) {
       if (resp.has_error()) {
         Status s = StatusFromPB(resp.error().status());
         string failure_msg = Substitute("Unexpected response: $0",
-                                        resp.DebugString());
+                                        SecureDebugString(resp));
         switch (resp.error().code()) {
           case MasterErrorPB::TABLE_NOT_FOUND:
             CHECK(s.IsServiceUnavailable()) << failure_msg;
@@ -1136,7 +1138,7 @@ TEST_F(MasterTest, TestConcurrentRenameOfSameTable) {
       req.mutable_table()->set_table_name(kOldName);
       req.set_new_table_name(kNewName);
       CHECK_OK(proxy_->AlterTable(req, &resp, &controller));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
 
       // There are two expected outcomes:
       //
@@ -1146,7 +1148,7 @@ TEST_F(MasterTest, TestConcurrentRenameOfSameTable) {
       if (resp.has_error()) {
         Status s = StatusFromPB(resp.error().status());
         string failure_msg = Substitute("Unexpected response: $0",
-                                        resp.DebugString());
+                                        SecureDebugString(resp));
         CHECK_EQ(MasterErrorPB::TABLE_NOT_FOUND, resp.error().code()) << failure_msg;
         CHECK(s.IsNotFound()) << failure_msg;
       }
@@ -1190,7 +1192,7 @@ TEST_F(MasterTest, TestConcurrentCreateAndRenameOfSameTable) {
 
         CHECK_OK(SchemaToPB(kTableSchema, req.mutable_schema()));
         CHECK_OK(proxy_->CreateTable(req, &resp, &controller));
-        SCOPED_TRACE(resp.DebugString());
+        SCOPED_TRACE(SecureDebugString(resp));
 
         // There are three expected outcomes:
         //
@@ -1202,7 +1204,7 @@ TEST_F(MasterTest, TestConcurrentCreateAndRenameOfSameTable) {
         if (resp.has_error()) {
           Status s = StatusFromPB(resp.error().status());
           string failure_msg = Substitute("Unexpected response: $0",
-                                          resp.DebugString());
+                                          SecureDebugString(resp));
           switch (resp.error().code()) {
             case MasterErrorPB::TABLE_NOT_FOUND:
               CHECK(s.IsServiceUnavailable()) << failure_msg;
@@ -1227,7 +1229,7 @@ TEST_F(MasterTest, TestConcurrentCreateAndRenameOfSameTable) {
         req.mutable_table()->set_table_name(kOldName);
         req.set_new_table_name(kNewName);
         CHECK_OK(proxy_->AlterTable(req, &resp, &controller));
-        SCOPED_TRACE(resp.DebugString());
+        SCOPED_TRACE(SecureDebugString(resp));
 
         // There are three expected outcomes:
         //
@@ -1241,7 +1243,7 @@ TEST_F(MasterTest, TestConcurrentCreateAndRenameOfSameTable) {
         if (resp.has_error()) {
           Status s = StatusFromPB(resp.error().status());
           string failure_msg = Substitute("Unexpected response: $0",
-                                          resp.DebugString());
+                                          SecureDebugString(resp));
           switch (resp.error().code()) {
             case MasterErrorPB::TABLE_NOT_FOUND:
               CHECK(s.IsServiceUnavailable() || s.IsNotFound()) << failure_msg;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 4bacb9b..9a83ff5 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -30,6 +30,7 @@
 #include "kudu/rpc/rpc_context.h"
 #include "kudu/server/webserver.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/pb_util.h"
 
 
 DEFINE_int32(master_inject_latency_on_tablet_lookups_ms, 0,
@@ -107,7 +108,7 @@ void MasterServiceImpl::TSHeartbeat(const TSHeartbeatRequestPB* req,
     if (s.IsNotFound()) {
       LOG(INFO) << Substitute("Got heartbeat from unknown tserver ($0) as $1; "
           "Asking this server to re-register.",
-          req->common().ts_instance().ShortDebugString(), rpc->requestor_string());
+          SecureShortDebugString(req->common().ts_instance()), rpc->requestor_string());
       resp->set_needs_reregister(true);
 
       // Don't bother asking for a full tablet report if we're a follower;
@@ -118,7 +119,7 @@ void MasterServiceImpl::TSHeartbeat(const TSHeartbeatRequestPB* req,
       return;
     } else if (!s.ok()) {
       LOG(WARNING) << Substitute("Unable to look up tserver for heartbeat "
-          "request $0 from $1: $2", req->DebugString(),
+          "request $0 from $1: $2", SecureDebugString(*req),
           rpc->requestor_string(), s.ToString());
       rpc->RespondFailure(s.CloneAndPrepend("Unable to lookup tserver"));
       return;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/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 62729e9..abaef6a 100644
--- a/src/kudu/master/sys_catalog-test.cc
+++ b/src/kudu/master/sys_catalog-test.cc
@@ -29,10 +29,11 @@
 #include "kudu/master/mini_master.h"
 #include "kudu/master/sys_catalog.h"
 #include "kudu/server/rpc_server.h"
+#include "kudu/rpc/messenger.h"
 #include "kudu/util/net/sockaddr.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_util.h"
-#include "kudu/rpc/messenger.h"
 
 using std::string;
 using std::shared_ptr;
@@ -99,7 +100,7 @@ class TableLoader : public TableVisitor {
 };
 
 static bool PbEquals(const google::protobuf::Message& a, const google::protobuf::Message& b) {
-  return a.DebugString() == b.DebugString();
+  return SecureDebugString(a) == SecureDebugString(b);
 }
 
 template<class C>

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/sys_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index e6b01f9..3f2e093 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -226,19 +226,19 @@ Status SysCatalogTable::CreateDistributedConfig(const MasterOptions& options,
     if (peer.has_permanent_uuid()) {
       resolved_config.add_peers()->CopyFrom(peer);
     } else {
-      LOG(INFO) << peer.ShortDebugString()
+      LOG(INFO) << SecureShortDebugString(peer)
                 << " has no permanent_uuid. Determining permanent_uuid...";
       RaftPeerPB new_peer = peer;
       RETURN_NOT_OK_PREPEND(consensus::SetPermanentUuidForRemotePeer(master_->messenger(),
                                                                      &new_peer),
                             Substitute("Unable to resolve UUID for peer $0",
-                                       peer.ShortDebugString()));
+                                       SecureShortDebugString(peer)));
       resolved_config.add_peers()->CopyFrom(new_peer);
     }
   }
 
   RETURN_NOT_OK(consensus::VerifyRaftConfig(resolved_config, consensus::COMMITTED_QUORUM));
-  VLOG(1) << "Distributed Raft configuration: " << resolved_config.ShortDebugString();
+  VLOG(1) << "Distributed Raft configuration: " << SecureShortDebugString(resolved_config);
 
   *committed_config = resolved_config;
   return Status::OK();
@@ -255,7 +255,7 @@ void SysCatalogTable::SysCatalogStateChanged(const string& tablet_id, const stri
   }
   consensus::ConsensusStatePB cstate = consensus->ConsensusState(CONSENSUS_CONFIG_COMMITTED);
   LOG_WITH_PREFIX(INFO) << "SysCatalogTable state changed. Reason: " << reason << ". "
-                        << "Latest consensus state: " << cstate.ShortDebugString();
+                        << "Latest consensus state: " << SecureShortDebugString(cstate);
   RaftPeerPB::Role new_role = GetConsensusRole(tablet_peer_->permanent_uuid(), cstate);
   LOG_WITH_PREFIX(INFO) << "This master's current role is: "
                         << RaftPeerPB::Role_Name(new_role);

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/ts_descriptor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_descriptor.cc b/src/kudu/master/ts_descriptor.cc
index f3f4633..26228fa 100644
--- a/src/kudu/master/ts_descriptor.cc
+++ b/src/kudu/master/ts_descriptor.cc
@@ -27,8 +27,9 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/tserver/tserver_admin.proxy.h"
-#include "kudu/util/net/net_util.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 
 DEFINE_int32(tserver_unresponsive_timeout_ms, 60 * 1000,
              "The period of time that a Master can go without receiving a heartbeat from a "
@@ -98,8 +99,8 @@ Status TSDescriptor::Register(const NodeInstancePB& instance,
         "Tablet server $0 is attempting to re-register with a different host/port. "
         "This is not currently supported. Old: {$1} New: {$2}",
         instance.permanent_uuid(),
-        registration_->ShortDebugString(),
-        registration.ShortDebugString());
+        SecureShortDebugString(*registration_),
+        SecureShortDebugString(registration));
     LOG(ERROR) << msg;
     return Status::InvalidArgument(msg);
   }
@@ -108,7 +109,7 @@ Status TSDescriptor::Register(const NodeInstancePB& instance,
       registration.http_addresses().empty()) {
     return Status::InvalidArgument(
         "invalid registration: must have at least one RPC and one HTTP address",
-        registration.ShortDebugString());
+        SecureShortDebugString(registration));
   }
 
   if (instance.instance_seqno() < latest_seqno_) {
@@ -121,7 +122,7 @@ Status TSDescriptor::Register(const NodeInstancePB& instance,
     // It's possible that the TS registered, but our response back to it
     // got lost, so it's trying to register again with the same sequence
     // number. That's fine.
-    LOG(INFO) << "Processing retry of TS registration from " << instance.ShortDebugString();
+    LOG(INFO) << "Processing retry of TS registration from " << SecureShortDebugString(instance);
   }
 
   latest_seqno_ = instance.instance_seqno();
@@ -214,7 +215,8 @@ Status TSDescriptor::ResolveSockaddr(Sockaddr* addr) const {
   }
 
   if (addrs.size() == 0) {
-    return Status::NetworkError("Unable to find the TS address: ", registration_->DebugString());
+    return Status::NetworkError("Unable to find the TS address: ",
+                                SecureDebugString(*registration_));
   }
 
   if (addrs.size() > 1) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/master/ts_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_manager.cc b/src/kudu/master/ts_manager.cc
index fd1b36d..bfecd84 100644
--- a/src/kudu/master/ts_manager.cc
+++ b/src/kudu/master/ts_manager.cc
@@ -24,6 +24,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/ts_descriptor.h"
+#include "kudu/util/pb_util.h"
 
 using std::shared_ptr;
 using std::string;
@@ -45,12 +46,13 @@ Status TSManager::LookupTS(const NodeInstancePB& instance,
   const shared_ptr<TSDescriptor>* found_ptr =
     FindOrNull(servers_by_id_, instance.permanent_uuid());
   if (!found_ptr) {
-    return Status::NotFound("unknown tablet server ID", instance.ShortDebugString());
+    return Status::NotFound("unknown tablet server ID", SecureShortDebugString(instance));
   }
   const shared_ptr<TSDescriptor>& found = *found_ptr;
 
   if (instance.instance_seqno() != found->latest_seqno()) {
-    return Status::NotFound("mismatched instance sequence number", instance.ShortDebugString());
+    return Status::NotFound("mismatched instance sequence number",
+                            SecureShortDebugString(instance));
   }
 
   *ts_desc = found;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/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 097e838..28537e0 100644
--- a/src/kudu/rpc/exactly_once_rpc-test.cc
+++ b/src/kudu/rpc/exactly_once_rpc-test.cc
@@ -16,8 +16,9 @@
 // under the License.
 
 #include "kudu/rpc/retriable_rpc.h"
-#include "kudu/rpc/rpc.h"
 #include "kudu/rpc/rpc-test-base.h"
+#include "kudu/rpc/rpc.h"
+#include "kudu/util/pb_util.h"
 
 DECLARE_int64(remember_clients_ttl_ms);
 DECLARE_int64(remember_responses_ttl_ms);
@@ -114,7 +115,8 @@ class CalculatorServiceRpc : public RetriableRpc<CalculatorServiceProxy,
     if (!successful_response_.IsInitialized()) {
       successful_response_.CopyFrom(resp_);
     } else {
-      CHECK_EQ(successful_response_.DebugString(), resp_.DebugString());
+      CHECK_EQ(SecureDebugString(successful_response_),
+               SecureDebugString(resp_));
     }
 
     if (sometimes_retry_successful_) {
@@ -297,10 +299,10 @@ class ExactlyOnceRpcTest : public RpcTestBase {
       Status s = MakeAddCall(sequence_number, 0, &response);
       if (s.ok()) {
         if (!result_gced) {
-          CHECK_EQ(response.ShortDebugString(), original_response.ShortDebugString());
+          CHECK_EQ(SecureDebugString(response), SecureDebugString(original_response));
         } else {
           client_gced = true;
-          CHECK_NE(response.ShortDebugString(), original_response.ShortDebugString());
+          CHECK_NE(SecureDebugString(response), SecureDebugString(original_response));
         }
         SleepFor(MonoDelta::FromMilliseconds(rand() % 10));
       } else if (s.IsRemoteError()) {
@@ -493,7 +495,7 @@ TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollection) {
   // Making the same request again, should return the same response.
   ExactlyOnceResponsePB resp;
   ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
-  ASSERT_EQ(original.ShortDebugString(), resp.ShortDebugString());
+  ASSERT_EQ(SecureShortDebugString(original), SecureShortDebugString(resp));
 
   // Now sleep for 'remember_responses_ttl_ms' and run GC, we should then
   // get a STALE back.
@@ -516,7 +518,7 @@ TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollection) {
 
   resp.Clear();
   ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
-  ASSERT_NE(resp.ShortDebugString(), original.ShortDebugString());
+  ASSERT_NE(SecureShortDebugString(resp), SecureShortDebugString(original));
 }
 
 // This test creates a thread continuously making requests to the server, some lasting longer

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/rpc/result_tracker.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/result_tracker.cc b/src/kudu/rpc/result_tracker.cc
index f7ac154..11ff8d2 100644
--- a/src/kudu/rpc/result_tracker.cc
+++ b/src/kudu/rpc/result_tracker.cc
@@ -27,8 +27,8 @@
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/mem_tracker.h"
-#include "kudu/util/trace.h"
 #include "kudu/util/pb_util.h"
+#include "kudu/util/trace.h"
 
 DEFINE_int64(remember_clients_ttl_ms, 3600 * 1000 /* 1 hour */,
     "Maximum amount of time, in milliseconds, the server \"remembers\" a client for the "
@@ -140,7 +140,7 @@ ResultTracker::RpcState ResultTracker::TrackRpcUnlocked(const RequestIdPB& reque
       context->call_->RespondFailure(
           ErrorStatusPB::ERROR_REQUEST_STALE,
           Status::Incomplete(Substitute("Request with id { $0 } is stale.",
-                                        request_id.ShortDebugString())));
+                                        SecureShortDebugString(request_id))));
       delete context;
     }
     return RpcState::STALE;
@@ -245,7 +245,7 @@ void ResultTracker::LogAndTraceAndRespondSuccess(RpcContext* context,
                                                  const Message& msg) {
   InboundCall* call = context->call_;
   VLOG(1) << this << " " << call->remote_method().service_name() << ": Sending RPC success "
-      "response for " << call->ToString() << ":" << std::endl << msg.DebugString();
+      "response for " << call->ToString() << ":" << std::endl << SecureDebugString(msg);
   TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
                          "response", pb_util::PbTracer::TracePb(msg),
                          "trace", context->trace()->DumpToString());
@@ -257,7 +257,7 @@ void ResultTracker::LogAndTraceFailure(RpcContext* context,
                                        const Message& msg) {
   InboundCall* call = context->call_;
   VLOG(1) << this << " " << call->remote_method().service_name() << ": Sending RPC failure "
-      "response for " << call->ToString() << ": " << msg.DebugString();
+      "response for " << call->ToString() << ": " << SecureDebugString(msg);
   TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
                          "response", pb_util::PbTracer::TracePb(msg),
                          "trace", context->trace()->DumpToString());
@@ -307,7 +307,7 @@ void ResultTracker::RecordCompletionAndRespond(const RequestIdPB& request_id,
     CHECK_EQ(completion_record->driver_attempt_no, request_id.attempt_no())
         << "Called RecordCompletionAndRespond() from an executor identified with an "
         << "attempt number that was not marked as the driver for the RPC. RequestId: "
-        << request_id.ShortDebugString() << "\nTracker state:\n " << ToStringUnlocked();
+        << SecureShortDebugString(request_id) << "\nTracker state:\n " << ToStringUnlocked();
     DCHECK_EQ(completion_record->state, RpcState::IN_PROGRESS);
     completion_record->response.reset(DCHECK_NOTNULL(response)->New());
     completion_record->response->CopyFrom(*response);
@@ -354,7 +354,7 @@ void ResultTracker::FailAndRespondInternal(const RequestIdPB& request_id,
     lock_guard<simple_spinlock> l(lock_);
     auto state_and_record = FindClientStateAndCompletionRecordOrNullUnlocked(request_id);
     if (PREDICT_FALSE(state_and_record.first == nullptr)) {
-      LOG(FATAL) << "Couldn't find ClientState for request: " << request_id.ShortDebugString()
+      LOG(FATAL) << "Couldn't find ClientState for request: " << SecureShortDebugString(request_id)
                  << ". \nTracker state:\n" << ToStringUnlocked();
     }
 
@@ -563,7 +563,7 @@ string ResultTracker::CompletionRecord::ToString() const {
                              "Cached response: $2, $3 OngoingRpcs:",
                              state,
                              driver_attempt_no,
-                             response ? response->ShortDebugString() : "None",
+                             response ? SecureShortDebugString(*response) : "None",
                              ongoing_rpcs.size());
   for (auto& orpc : ongoing_rpcs) {
     SubstituteAndAppend(&result, Substitute("\n\t$0", orpc.ToString()));
@@ -574,7 +574,8 @@ string ResultTracker::CompletionRecord::ToString() const {
 
 string ResultTracker::OnGoingRpcInfo::ToString() const {
   return Substitute("OngoingRpc[Handler: $0, Context: $1, Response: $2]",
-                    handler_attempt_no, context, response ? response->ShortDebugString() : "NULL");
+                    handler_attempt_no, context,
+                    response ? SecureShortDebugString(*response) : "NULL");
 }
 
 } // namespace rpc

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/rpc/rpc-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc-test-base.h b/src/kudu/rpc/rpc-test-base.h
index c6c87a8..8122a1b 100644
--- a/src/kudu/rpc/rpc-test-base.h
+++ b/src/kudu/rpc/rpc-test-base.h
@@ -42,6 +42,7 @@
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/random.h"
 #include "kudu/util/random_util.h"
 #include "kudu/util/stopwatch.h"
@@ -168,7 +169,7 @@ class GenericCalculatorService : public ServiceIf {
       return;
     }
 
-    LOG(INFO) << "got call: " << req.ShortDebugString();
+    LOG(INFO) << "got call: " << SecureShortDebugString(req);
     SleepFor(MonoDelta::FromMicroseconds(req.sleep_micros()));
     SleepResponsePB resp;
     incoming->RespondSuccess(resp);

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/rpc/rpc_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.cc b/src/kudu/rpc/rpc_context.cc
index 02ca2cc..a953b6d 100644
--- a/src/kudu/rpc/rpc_context.cc
+++ b/src/kudu/rpc/rpc_context.cc
@@ -27,8 +27,8 @@
 #include "kudu/rpc/service_if.h"
 #include "kudu/util/hdr_histogram.h"
 #include "kudu/util/metrics.h"
-#include "kudu/util/trace.h"
 #include "kudu/util/pb_util.h"
+#include "kudu/util/trace.h"
 
 using google::protobuf::Message;
 
@@ -44,7 +44,7 @@ RpcContext::RpcContext(InboundCall *call,
     response_pb_(response_pb),
     result_tracker_(result_tracker) {
   VLOG(4) << call_->remote_method().service_name() << ": Received RPC request for "
-          << call_->ToString() << ":" << std::endl << request_pb_->DebugString();
+          << call_->ToString() << ":" << std::endl << SecureDebugString(*request_pb_);
   TRACE_EVENT_ASYNC_BEGIN2("rpc_call", "RPC", this,
                            "call", call_->ToString(),
                            "request", pb_util::PbTracer::TracePb(*request_pb_));
@@ -59,7 +59,7 @@ void RpcContext::RespondSuccess() {
                                                 response_pb_.get());
   } else {
     VLOG(4) << call_->remote_method().service_name() << ": Sending RPC success response for "
-        << call_->ToString() << ":" << std::endl << response_pb_->DebugString();
+        << call_->ToString() << ":" << std::endl << SecureDebugString(*response_pb_);
     TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
                            "response", pb_util::PbTracer::TracePb(*response_pb_),
                            "trace", trace()->DumpToString());
@@ -74,7 +74,7 @@ void RpcContext::RespondNoCache() {
                                     response_pb_.get());
   } else {
     VLOG(4) << call_->remote_method().service_name() << ": Sending RPC failure response for "
-        << call_->ToString() << ": " << response_pb_->DebugString();
+        << call_->ToString() << ": " << SecureDebugString(*response_pb_);
     TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
                            "response", pb_util::PbTracer::TracePb(*response_pb_),
                            "trace", trace()->DumpToString());
@@ -126,7 +126,7 @@ void RpcContext::RespondApplicationError(int error_ext_id, const std::string& me
       InboundCall::ApplicationErrorToPB(error_ext_id, message, app_error_pb, &err);
       VLOG(4) << call_->remote_method().service_name()
           << ": Sending application error response for " << call_->ToString()
-          << ":" << std::endl << err.DebugString();
+          << ":" << std::endl << SecureDebugString(err);
     }
     TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
                            "response", pb_util::PbTracer::TracePb(app_error_pb),
@@ -172,7 +172,7 @@ void RpcContext::Panic(const char* filepath, int line_number, const string& mess
 #define MY_FATAL google::LogMessageFatal(filepath, line_number).stream()
 
   MY_ERROR << "Panic handling " << call_->ToString() << ": " << message;
-  MY_ERROR << "Request:\n" << request_pb_->DebugString();
+  MY_ERROR << "Request:\n" << SecureDebugString(*request_pb_);
   Trace* t = trace();
   if (t) {
     MY_ERROR << "RPC trace:";

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/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 07d7397..09504fd 100644
--- a/src/kudu/rpc/rpc_stub-test.cc
+++ b/src/kudu/rpc/rpc_stub-test.cc
@@ -33,6 +33,7 @@
 #include "kudu/rpc/rpc-test-base.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_util.h"
 #include "kudu/util/user.h"
@@ -279,7 +280,8 @@ TEST_F(RpcStubTest, TestApplicationError) {
   EXPECT_EQ("message: \"Got some error\"\n"
             "[kudu.rpc_test.CalculatorError.app_error_ext] {\n"
             "  extra_error_data: \"some application-specific error data\"\n"
-            "}\n", controller.error_response()->DebugString());
+            "}\n",
+            SecureDebugString(*controller.error_response()));
 }
 
 TEST_F(RpcStubTest, TestRpcPanic) {
@@ -482,7 +484,7 @@ TEST_F(RpcStubTest, TestDumpCallsInFlight) {
   dump_req.set_include_traces(true);
 
   ASSERT_OK(client_messenger_->DumpRunningRpcs(dump_req, &dump_resp));
-  LOG(INFO) << "client messenger: " << dump_resp.DebugString();
+  LOG(INFO) << "client messenger: " << SecureDebugString(dump_resp);
   ASSERT_EQ(1, dump_resp.outbound_connections_size());
   ASSERT_EQ(1, dump_resp.outbound_connections(0).calls_in_flight_size());
   ASSERT_EQ("Sleep", dump_resp.outbound_connections(0).calls_in_flight(0).
@@ -502,7 +504,7 @@ TEST_F(RpcStubTest, TestDumpCallsInFlight) {
     SleepFor(MonoDelta::FromMilliseconds(1));
   }
 
-  LOG(INFO) << "server messenger: " << dump_resp.DebugString();
+  LOG(INFO) << "server messenger: " << SecureDebugString(dump_resp);
   ASSERT_EQ(1, dump_resp.inbound_connections_size());
   ASSERT_EQ(1, dump_resp.inbound_connections(0).calls_in_flight_size());
   ASSERT_EQ("Sleep", dump_resp.inbound_connections(0).calls_in_flight(0).
@@ -535,24 +537,24 @@ TEST_F(RpcStubTest, TestDumpSampledCalls) {
   DumpRpczStoreResponsePB sampled_rpcs;
   server_messenger_->rpcz_store()->DumpPB(DumpRpczStoreRequestPB(), &sampled_rpcs);
   EXPECT_EQ(sampled_rpcs.methods_size(), 1);
-  ASSERT_STR_CONTAINS(sampled_rpcs.DebugString(),
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs),
                       "    metrics {\n"
                       "      key: \"test_sleep_us\"\n"
                       "      value: 150000\n"
                       "    }\n");
-  ASSERT_STR_CONTAINS(sampled_rpcs.DebugString(),
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs),
                       "    metrics {\n"
                       "      key: \"test_sleep_us\"\n"
                       "      value: 1500000\n"
                       "    }\n");
-  ASSERT_STR_CONTAINS(sampled_rpcs.DebugString(),
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs),
                       "    metrics {\n"
                       "      child_path: \"test_child\"\n"
                       "      key: \"related_trace_metric\"\n"
                       "      value: 1\n"
                       "    }");
-  ASSERT_STR_CONTAINS(sampled_rpcs.DebugString(), "SleepRequestPB");
-  ASSERT_STR_CONTAINS(sampled_rpcs.DebugString(), "duration_ms");
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs), "SleepRequestPB");
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs), "duration_ms");
 }
 
 namespace {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/tablet/row_op.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/row_op.cc b/src/kudu/tablet/row_op.cc
index 2c9ecb8..50d4fc3 100644
--- a/src/kudu/tablet/row_op.cc
+++ b/src/kudu/tablet/row_op.cc
@@ -18,6 +18,7 @@
 #include "kudu/common/wire_protocol.h"
 #include "kudu/tablet/row_op.h"
 #include "kudu/tablet/tablet.pb.h"
+#include "kudu/util/pb_util.h"
 
 namespace kudu {
 namespace tablet {
@@ -31,19 +32,19 @@ RowOp::~RowOp() {
 }
 
 void RowOp::SetFailed(const Status& s) {
-  DCHECK(!result) << result->DebugString();
+  DCHECK(!result) << SecureDebugString(*result);
   result.reset(new OperationResultPB());
   StatusToPB(s, result->mutable_failed_status());
 }
 
 void RowOp::SetInsertSucceeded(int mrs_id) {
-  DCHECK(!result) << result->DebugString();
+  DCHECK(!result) << SecureDebugString(*result);
   result.reset(new OperationResultPB());
   result->add_mutated_stores()->set_mrs_id(mrs_id);
 }
 
 void RowOp::SetMutateSucceeded(gscoped_ptr<OperationResultPB> result) {
-  DCHECK(!this->result) << result->DebugString();
+  DCHECK(!this->result) << SecureDebugString(*result);
   this->result = std::move(result);
 }
 
@@ -52,7 +53,7 @@ string RowOp::ToString(const Schema& schema) const {
 }
 
 void RowOp::SetAlreadyFlushed() {
-  DCHECK(!result) << result->DebugString();
+  DCHECK(!result) << SecureDebugString(*result);
   result.reset(new OperationResultPB());
   result->set_flushed(true);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/tablet/tablet_bootstrap-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc
index 2a223bc..fa2394a 100644
--- a/src/kudu/tablet/tablet_bootstrap-test.cc
+++ b/src/kudu/tablet/tablet_bootstrap-test.cc
@@ -338,11 +338,11 @@ TEST_F(BootstrapTest, TestOrphanedReplicate) {
 
   // The consensus bootstrap info should include the orphaned REPLICATE.
   ASSERT_EQ(1, boot_info.orphaned_replicates.size());
-  ASSERT_STR_CONTAINS(boot_info.orphaned_replicates[0]->ShortDebugString(),
+  ASSERT_STR_CONTAINS(SecureShortDebugString(*boot_info.orphaned_replicates[0]),
                       "this is a test mutate");
 
   // And it should also include the latest opids.
-  EXPECT_EQ("term: 1 index: 1", boot_info.last_id.ShortDebugString());
+  EXPECT_EQ("term: 1 index: 1", SecureShortDebugString(boot_info.last_id));
 }
 
 // Bootstrap should fail if no ConsensusMetadata file exists.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index af7e494..a7cd658 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -57,6 +57,7 @@
 #include "kudu/util/locks.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
 
 DEFINE_bool(skip_remove_old_recovery_dir, false,
@@ -411,7 +412,7 @@ static string DebugInfo(const string& tablet_id,
   // Truncate the debug string to a reasonable length for logging.
   // Otherwise, glog will truncate for us and we may miss important
   // information which came after this long string.
-  string debug_str = entry.ShortDebugString();
+  string debug_str = SecureShortDebugString(entry);
   if (debug_str.size() > 500) {
     debug_str.resize(500);
     debug_str.append("...");
@@ -464,7 +465,7 @@ Status TabletBootstrap::Bootstrap(shared_ptr<Tablet>* rebuilt_tablet,
   if (VLOG_IS_ON(1)) {
     TabletSuperBlockPB super_block;
     RETURN_NOT_OK(meta_->ToSuperBlock(&super_block));
-    VLOG_WITH_PREFIX(1) << "Tablet Metadata: " << super_block.DebugString();
+    VLOG_WITH_PREFIX(1) << "Tablet Metadata: " << SecureDebugString(super_block);
   }
 
   RETURN_NOT_OK(flushed_stores_.InitFrom(*meta_.get()));
@@ -725,7 +726,7 @@ struct ReplayState {
   void AddEntriesToStrings(const OpIndexToEntryMap& entries, vector<string>* strings) const {
     for (const OpIndexToEntryMap::value_type& map_entry : entries) {
       LogEntryPB* entry = DCHECK_NOTNULL(map_entry.second);
-      strings->push_back(Substitute("   $0", entry->ShortDebugString()));
+      strings->push_back(Substitute("   $0", SecureShortDebugString(*entry)));
     }
   }
 
@@ -762,7 +763,7 @@ struct ReplayState {
 // Otherwise, caller frees.
 Status TabletBootstrap::HandleEntry(ReplayState* state, LogEntryPB* entry) {
   if (VLOG_IS_ON(1)) {
-    VLOG_WITH_PREFIX(1) << "Handling entry: " << entry->ShortDebugString();
+    VLOG_WITH_PREFIX(1) << "Handling entry: " << SecureShortDebugString(*entry);
   }
 
   switch (entry->type()) {
@@ -825,7 +826,8 @@ Status TabletBootstrap::HandleReplicateMessage(ReplayState* state, LogEntryPB* r
 
 // Takes ownership of 'commit_entry' on OK status.
 Status TabletBootstrap::HandleCommitMessage(ReplayState* state, LogEntryPB* commit_entry) {
-  DCHECK(commit_entry->has_commit()) << "Not a commit message: " << commit_entry->DebugString();
+  DCHECK(commit_entry->has_commit()) << "Not a commit message: "
+                                     << SecureDebugString(*commit_entry);
 
   // Match up the COMMIT record with the original entry that it's applied to.
   const OpId& committed_op_id = commit_entry->commit().commited_op_id();
@@ -847,7 +849,7 @@ Status TabletBootstrap::HandleCommitMessage(ReplayState* state, LogEntryPB* comm
   if ((*state->pending_replicates.begin()).first != committed_op_id.index()) {
     if (!ContainsKey(state->pending_replicates, committed_op_id.index())) {
       return Status::Corruption(Substitute("Could not find replicate for commit: $0",
-                                           commit_entry->ShortDebugString()));
+                                           SecureShortDebugString(*commit_entry)));
     }
     VLOG_WITH_PREFIX(2) << "Adding pending commit for " << committed_op_id;
     InsertOrDie(&state->pending_commits, committed_op_id.index(), commit_entry);
@@ -898,8 +900,8 @@ Status TabletBootstrap::CheckOrphanedCommitDoesntNeedReplay(const CommitMsg& com
     TabletSuperBlockPB super;
     WARN_NOT_OK(meta_->ToSuperBlock(&super), LogPrefix() + "Couldn't build TabletSuperBlockPB");
     return Status::Corruption(Substitute("CommitMsg was orphaned but it referred to "
-        "stores which need replay. Commit: $0. TabletMetadata: $1", commit.ShortDebugString(),
-        super.ShortDebugString()));
+        "stores which need replay. Commit: $0. TabletMetadata: $1", SecureShortDebugString(commit),
+        SecureShortDebugString(super)));
   }
 
   return Status::OK();
@@ -922,10 +924,10 @@ Status TabletBootstrap::ApplyCommitMessage(ReplayState* state, LogEntryPB* commi
     if (!OpIdEquals(committed_op_id, pending_replicate_entry->replicate().id())) {
       string error_msg = Substitute("Committed operation's OpId: $0 didn't match the"
           "commit message's committed OpId: $1. Pending operation: $2, Commit message: $3",
-          pending_replicate_entry->replicate().id().ShortDebugString(),
-          committed_op_id.ShortDebugString(),
-          pending_replicate_entry->replicate().ShortDebugString(),
-          commit_entry->commit().ShortDebugString());
+          SecureShortDebugString(pending_replicate_entry->replicate().id()),
+          SecureShortDebugString(committed_op_id),
+          SecureShortDebugString(pending_replicate_entry->replicate()),
+          SecureShortDebugString(commit_entry->commit()));
       LOG_WITH_PREFIX(DFATAL) << error_msg;
       return Status::Corruption(error_msg);
     }
@@ -943,10 +945,11 @@ Status TabletBootstrap::ApplyCommitMessage(ReplayState* state, LogEntryPB* commi
 Status TabletBootstrap::HandleEntryPair(LogEntryPB* replicate_entry, LogEntryPB* commit_entry) {
   const char* error_fmt = "Failed to play $0 request. ReplicateMsg: { $1 }, CommitMsg: { $2 }";
 
-#define RETURN_NOT_OK_REPLAY(ReplayMethodName, replicate, commit) \
-  RETURN_NOT_OK_PREPEND(ReplayMethodName(replicate, commit), \
+#define RETURN_NOT_OK_REPLAY(ReplayMethodName, replicate, commit)       \
+  RETURN_NOT_OK_PREPEND(ReplayMethodName(replicate, commit),            \
                         Substitute(error_fmt, OperationType_Name(op_type), \
-                                   replicate->ShortDebugString(), commit.ShortDebugString()))
+                                   SecureShortDebugString(*replicate),   \
+                                   SecureShortDebugString(commit)))
 
   ReplicateMsg* replicate = replicate_entry->mutable_replicate();
   const CommitMsg& commit = commit_entry->commit();
@@ -1116,7 +1119,7 @@ Status TabletBootstrap::PlaySegments(ConsensusBootstrapInfo* consensus_info) {
         WARN_NOT_OK(meta_->ToSuperBlock(&super), "Couldn't build TabletSuperBlockPB.");
         return Status::Corruption(Substitute("CommitMsg was pending but it did not refer "
             "to any active memory stores. Commit: $0. TabletMetadata: $1",
-            entry.second->commit().ShortDebugString(), super.ShortDebugString()));
+            SecureShortDebugString(entry.second->commit()), SecureShortDebugString(super)));
       }
     }
   }
@@ -1231,7 +1234,7 @@ Status TabletBootstrap::PlayWriteRequest(ReplicateMsg* replicate_msg,
   if (tracking_results) {
     VLOG(1) << result_tracker_.get() << " Boostrapping request for tablet: "
         << write->tablet_id() << ". State: " << 0 << " id: "
-        << replicate_msg->request_id().DebugString();
+        << SecureDebugString(replicate_msg->request_id());
     // We only replay committed requests so the result of tracking this request can be:
     // NEW:
     //   This is a previously untracked request, or we changed the driver -> store the result
@@ -1372,7 +1375,7 @@ Status TabletBootstrap::PlayRowOperations(WriteTransactionState* tx_state,
 
   RETURN_NOT_OK_PREPEND(tablet_->DecodeWriteOperations(&inserts_schema, tx_state),
                         Substitute("Could not decode row operations: $0",
-                                   ops_pb.ShortDebugString()));
+                                   SecureShortDebugString(ops_pb)));
   DCHECK_EQ(tx_state->row_ops().size(), already_flushed.size());
 
   // Propagate the 'already_flushed' information into the decoded operations.
@@ -1436,7 +1439,7 @@ Status TabletBootstrap::ApplyOperations(WriteTransactionState* tx_state,
       Status status = StatusFromPB(orig_op_result.failed_status());
       if (VLOG_IS_ON(1)) {
         VLOG_WITH_PREFIX(1) << "Skipping operation that originally resulted in error. OpId: "
-                            << tx_state->op_id().DebugString() << " op index: "
+                            << SecureDebugString(tx_state->op_id()) << " op index: "
                             << op_idx - 1 << " original error: "
                             << status.ToString();
       }
@@ -1458,7 +1461,7 @@ Status TabletBootstrap::ApplyOperations(WriteTransactionState* tx_state,
                                 "during log replay",
                                 Substitute("Op: $0\nFailure: $1",
                                            op->ToString(*tablet_->schema()),
-                                           op->result->failed_status().ShortDebugString()));
+                                           SecureShortDebugString(op->result->failed_status())));
     }
   }
   return Status::OK();
@@ -1475,7 +1478,7 @@ Status TabletBootstrap::FilterOperation(const OperationResultPB& op_result,
   int num_mutated_stores = op_result.mutated_stores_size();
   if (PREDICT_FALSE(num_mutated_stores > 2)) {
     return Status::Corruption(Substitute("All operations must have at most two mutated_stores: $0",
-                                         op_result.ShortDebugString()));
+                                         SecureShortDebugString(op_result)));
   }
   // NOTE: it's possible that num_mutated_stores = 0 in the case of an
   // UPSERT which only specified the primary key. In that case, if the
@@ -1503,7 +1506,7 @@ Status TabletBootstrap::FilterOperation(const OperationResultPB& op_result,
     // the 'second' store was live. But at no time should the metadata refer to both the
     // 'input' and 'output' stores of a compaction.
     return Status::Corruption("Mutation was duplicated to two stores that are considered live",
-                              op_result.ShortDebugString());
+                              SecureShortDebugString(op_result));
   }
 
   *already_flushed = false;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/tablet/tablet_metadata-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_metadata-test.cc b/src/kudu/tablet/tablet_metadata-test.cc
index a56ac80..5378921 100644
--- a/src/kudu/tablet/tablet_metadata-test.cc
+++ b/src/kudu/tablet/tablet_metadata-test.cc
@@ -23,6 +23,7 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/tablet-test-util.h"
+#include "kudu/util/pb_util.h"
 
 namespace kudu {
 namespace tablet {
@@ -86,11 +87,11 @@ TEST_F(TestTabletMetadata, TestLoadFromSuperBlock) {
   // Compare the 2 dumped superblock PBs.
   ASSERT_EQ(superblock_pb_1.SerializeAsString(),
             superblock_pb_2.SerializeAsString())
-    << superblock_pb_1.DebugString()
-    << superblock_pb_2.DebugString();
+    << SecureDebugString(superblock_pb_1)
+    << SecureDebugString(superblock_pb_2);
 
   LOG(INFO) << "Superblocks match:\n"
-            << superblock_pb_1.DebugString();
+            << SecureDebugString(superblock_pb_1);
 }
 
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/tablet/tablet_metadata.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_metadata.cc b/src/kudu/tablet/tablet_metadata.cc
index a0ad971..c71b9e3 100644
--- a/src/kudu/tablet/tablet_metadata.cc
+++ b/src/kudu/tablet/tablet_metadata.cc
@@ -35,10 +35,10 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/rowset_metadata.h"
 #include "kudu/util/debug/trace_event.h"
+#include "kudu/util/flag_tags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
-#include "kudu/util/flag_tags.h"
 #include "kudu/util/trace.h"
 
 DEFINE_bool(enable_tablet_orphaned_block_deletion, true,
@@ -276,7 +276,7 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock)
   vector<BlockId> orphaned_blocks;
 
   VLOG(2) << "Loading TabletMetadata from SuperBlockPB:" << std::endl
-          << superblock.DebugString();
+          << SecureDebugString(superblock);
 
   {
     std::lock_guard<LockType> l(data_lock_);
@@ -285,7 +285,7 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock)
     if (superblock.tablet_id() != tablet_id_) {
       return Status::Corruption("Expected id=" + tablet_id_ +
                                 " found " + superblock.tablet_id(),
-                                superblock.DebugString());
+                                SecureDebugString(superblock));
     }
 
     last_durable_mrs_id_ = superblock.last_durable_mrs_id();
@@ -296,7 +296,7 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock)
     gscoped_ptr<Schema> schema(new Schema());
     RETURN_NOT_OK_PREPEND(SchemaFromPB(superblock.schema(), schema.get()),
                           "Failed to parse Schema from superblock " +
-                          superblock.ShortDebugString());
+                          SecureShortDebugString(superblock));
     SetSchemaUnlocked(std::move(schema), schema_version);
 
     if (!superblock.has_partition()) {
@@ -306,7 +306,7 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock)
           << " version is not supported. Please upgrade to 0.6.0 before"
           << " moving to a higher version.";
       return Status::NotFound("Missing partition in superblock "+
-                              superblock.DebugString());
+                              SecureDebugString(superblock));
     }
 
     // Some metadata fields are assumed to be immutable and thus are