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 02:23:25 UTC

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

Repository: kudu
Updated Branches:
  refs/heads/branch-1.2.x 5a6ef37bd -> e7b45d375


http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tablet/tablet_peer-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_peer-test.cc b/src/kudu/tablet/tablet_peer-test.cc
index fd6a3c8..95c7d0c 100644
--- a/src/kudu/tablet/tablet_peer-test.cc
+++ b/src/kudu/tablet/tablet_peer-test.cc
@@ -41,8 +41,9 @@
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/maintenance_manager.h"
 #include "kudu/util/metrics.h"
-#include "kudu/util/test_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
 #include "kudu/util/threadpool.h"
 
 METRIC_DECLARE_entity(tablet);
@@ -203,7 +204,7 @@ class TabletPeerTest : public KuduTabletTest {
     CHECK_OK(tablet_peer->SubmitWrite(std::move(tx_state)));
     rpc_latch.Wait();
     CHECK(!resp->has_error())
-        << "\nReq:\n" << req.DebugString() << "Resp:\n" << resp->DebugString();
+        << "\nReq:\n" << SecureDebugString(req) << "Resp:\n" << SecureDebugString(*resp);
 
     // Roll the log after each write.
     // Usually the append thread does the roll and no additional sync is required. However in
@@ -249,7 +250,7 @@ class TabletPeerTest : public KuduTabletTest {
     tablet_peer_->log_->GetLatestEntryOpId(&last_log_opid);
     CHECK_LT(retention.for_durability, last_log_opid.index())
       << "Expected valid log anchor, got earliest opid: " << retention.for_durability
-      << " (expected any value earlier than last log id: " << last_log_opid.ShortDebugString()
+      << " (expected any value earlier than last log id: " << SecureShortDebugString(last_log_opid)
       << ")";
   }
 
@@ -361,7 +362,7 @@ TEST_F(TabletPeerTest, TestDMSAnchorPreventsLogGC) {
 
   OpId id;
   log->GetLatestEntryOpId(&id);
-  LOG(INFO) << "Before: " << id.ShortDebugString();
+  LOG(INFO) << "Before: " << SecureShortDebugString(id);
 
 
   // We currently have no anchors and the last operation in the log is 0.3

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tablet/tablet_peer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_peer.cc b/src/kudu/tablet/tablet_peer.cc
index 5e901a0..d9bddb4 100644
--- a/src/kudu/tablet/tablet_peer.cc
+++ b/src/kudu/tablet/tablet_peer.cc
@@ -50,6 +50,7 @@
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/threadpool.h"
 #include "kudu/util/trace.h"
@@ -195,7 +196,7 @@ Status TabletPeer::Start(const ConsensusBootstrapInfo& bootstrap_info) {
 
   VLOG(2) << "T " << tablet_id() << " P " << consensus_->peer_uuid() << ": Peer starting";
 
-  VLOG(2) << "RaftConfig before starting: " << consensus_->CommittedConfig().DebugString();
+  VLOG(2) << "RaftConfig before starting: " << SecureDebugString(consensus_->CommittedConfig());
 
   RETURN_NOT_OK(consensus_->Start(bootstrap_info));
   {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tablet/transactions/alter_schema_transaction.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/alter_schema_transaction.cc b/src/kudu/tablet/transactions/alter_schema_transaction.cc
index bb262fd..92a626d 100644
--- a/src/kudu/tablet/transactions/alter_schema_transaction.cc
+++ b/src/kudu/tablet/transactions/alter_schema_transaction.cc
@@ -26,6 +26,7 @@
 #include "kudu/tablet/tablet_peer.h"
 #include "kudu/tablet/tablet_metrics.h"
 #include "kudu/tserver/tserver.pb.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/trace.h"
 
 namespace kudu {
@@ -47,7 +48,7 @@ string AlterSchemaTransactionState::ToString() const {
                     "[timestamp=$0, schema=$1, request=$2]",
                     has_timestamp() ? timestamp().ToString() : "<unassigned>",
                     schema_ == nullptr ? "(none)" : schema_->ToString(),
-                    request_ == nullptr ? "(none)" : request_->ShortDebugString());
+                    request_ == nullptr ? "(none)" : SecureShortDebugString(*request_));
 }
 
 void AlterSchemaTransactionState::AcquireSchemaLock(rw_semaphore* l) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tablet/transactions/transaction_driver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/transaction_driver.cc b/src/kudu/tablet/transactions/transaction_driver.cc
index 8c5b4b9..3050f34 100644
--- a/src/kudu/tablet/transactions/transaction_driver.cc
+++ b/src/kudu/tablet/transactions/transaction_driver.cc
@@ -28,6 +28,7 @@
 #include "kudu/util/debug-util.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/logging.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/threadpool.h"
 #include "kudu/util/trace.h"
 
@@ -231,8 +232,8 @@ void TransactionDriver::RegisterFollowerTransactionOnResultTracker() {
       mutable_state()->set_completion_callback(
           gscoped_ptr<TransactionCompletionCallback>(new TransactionCompletionCallback()));
       VLOG(2) << state()->result_tracker() << " Follower Rpc was already COMPLETED or STALE: "
-          << rpc_state << " OpId: " << state()->op_id().ShortDebugString()
-          << " RequestId: " << state()->request_id().ShortDebugString();
+          << rpc_state << " OpId: " << SecureShortDebugString(state()->op_id())
+          << " RequestId: " << SecureShortDebugString(state()->request_id());
       return;
     }
     default:
@@ -273,7 +274,7 @@ Status TransactionDriver::Prepare() {
           && !state()->result_tracker()->IsCurrentDriver(state()->request_id())) {
         transaction_status_ = Status::AlreadyPresent(strings::Substitute(
             "There's already an attempt of the same operation on the server for request id: $0",
-            state()->request_id().ShortDebugString()));
+            SecureShortDebugString(state()->request_id())));
         replication_state_ = REPLICATION_FAILED;
         return transaction_status_;
       }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tablet/transactions/write_transaction.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/write_transaction.cc b/src/kudu/tablet/transactions/write_transaction.cc
index a56fc19..bfbc867 100644
--- a/src/kudu/tablet/transactions/write_transaction.cc
+++ b/src/kudu/tablet/transactions/write_transaction.cc
@@ -34,6 +34,7 @@
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/trace.h"
 
 DEFINE_int32(tablet_inject_latency_on_apply_write_txn_ms, 0,
@@ -372,7 +373,7 @@ string WriteTransactionState::ToString() const {
 
   return Substitute("WriteTransactionState $0 [op_id=($1), ts=$2, rows=$3]",
                     this,
-                    op_id().ShortDebugString(),
+                    SecureShortDebugString(op_id()),
                     ts_str,
                     row_ops_str);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 163ef63..6d42519 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -70,6 +70,7 @@
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
@@ -704,7 +705,7 @@ TEST_F(ToolTest, TestLocalReplicaDumpMeta) {
 
   TabletSuperBlockPB pb1;
   meta->ToSuperBlock(&pb1);
-  debug_str = pb1.DebugString();
+  debug_str = SecureDebugString(pb1);
   StripWhiteSpace(&debug_str);
   ASSERT_STR_CONTAINS(stdout, "Superblock:");
   ASSERT_STR_CONTAINS(stdout, debug_str);
@@ -834,7 +835,7 @@ TEST_F(ToolTest, TestLocalReplicaOps) {
 
     TabletSuperBlockPB pb1;
     meta->ToSuperBlock(&pb1);
-    debug_str = pb1.DebugString();
+    debug_str = SecureDebugString(pb1);
     StripWhiteSpace(&debug_str);
     ASSERT_STR_CONTAINS(stdout, "Superblock:");
     ASSERT_STR_CONTAINS(stdout, debug_str);

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tools/tool_action_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_common.cc b/src/kudu/tools/tool_action_common.cc
index 80c14ba..5f12072 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -143,7 +143,7 @@ void PrintIdOnly(const LogEntryPB& entry) {
       break;
     }
     default:
-      cout << "UNKNOWN: " << entry.ShortDebugString();
+      cout << "UNKNOWN: " << SecureShortDebugString(entry);
   }
 
   cout << endl;
@@ -163,7 +163,7 @@ Status PrintDecodedWriteRequestPB(const string& indent,
 
   cout << indent << "Tablet: " << write.tablet_id() << endl;
   cout << indent << "RequestId: "
-      << (request_id ? request_id->ShortDebugString() : "None") << endl;
+      << (request_id ? SecureShortDebugString(*request_id) : "None") << endl;
   cout << indent << "Consistency: "
        << ExternalConsistencyMode_Name(write.external_consistency_mode()) << endl;
   if (write.has_propagated_timestamp()) {
@@ -195,11 +195,11 @@ Status PrintDecoded(const LogEntryPB& entry, const Schema& tablet_schema) {
           replicate.write_request(),
           replicate.has_request_id() ? &replicate.request_id() : nullptr));
     } else {
-      cout << indent << replicate.ShortDebugString() << endl;
+      cout << indent << SecureShortDebugString(replicate) << endl;
     }
   } else if (entry.has_commit()) {
     // For COMMIT we'll just dump the PB
-    cout << indent << entry.commit().ShortDebugString() << endl;
+    cout << indent << SecureShortDebugString(entry.commit()) << endl;
   }
 
   return Status::OK();
@@ -259,7 +259,7 @@ Status GetServerStatus(const string& address, uint16_t default_port,
 Status PrintSegment(const scoped_refptr<ReadableLogSegment>& segment) {
   PrintEntryType print_type = ParsePrintType();
   if (FLAGS_print_meta) {
-    cout << "Header:\n" << segment->header().DebugString();
+    cout << "Header:\n" << SecureDebugString(segment->header());
   }
   if (print_type != DONT_PRINT) {
     Schema tablet_schema;
@@ -277,7 +277,7 @@ Status PrintSegment(const scoped_refptr<ReadableLogSegment>& segment) {
           pb_util::TruncateFields(&entry, FLAGS_truncate_data);
         }
 
-        cout << "Entry:\n" << entry.DebugString();
+        cout << "Entry:\n" << SecureDebugString(entry);
       } else if (print_type == PRINT_DECODED) {
         RETURN_NOT_OK(PrintDecoded(entry, tablet_schema));
       } else if (print_type == PRINT_ID) {
@@ -286,7 +286,7 @@ Status PrintSegment(const scoped_refptr<ReadableLogSegment>& segment) {
     }
   }
   if (FLAGS_print_meta && segment->HasFooter()) {
-    cout << "Footer:\n" << segment->footer().DebugString();
+    cout << "Footer:\n" << SecureDebugString(segment->footer());
   }
 
   return Status::OK();
@@ -314,14 +314,14 @@ Status SetServerFlag(const string& address, uint16_t default_port,
       return Status::RemoteError(resp.msg() +
                                  " (use --force flag to allow anyway)");
     default:
-      return Status::RemoteError(resp.ShortDebugString());
+      return Status::RemoteError(SecureShortDebugString(resp));
   }
 }
 
 Status PrintServerStatus(const string& address, uint16_t default_port) {
   ServerStatusPB status;
   RETURN_NOT_OK(GetServerStatus(address, default_port, &status));
-  cout << status.DebugString() << endl;
+  cout << SecureDebugString(status) << endl;
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tools/tool_action_fs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_fs.cc b/src/kudu/tools/tool_action_fs.cc
index cb5e483..6245b87 100644
--- a/src/kudu/tools/tool_action_fs.cc
+++ b/src/kudu/tools/tool_action_fs.cc
@@ -30,6 +30,7 @@
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tools/tool_action_common.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 
 DECLARE_bool(print_meta);
@@ -92,8 +93,8 @@ Status DumpCFile(const RunnerContext& context) {
   RETURN_NOT_OK(CFileReader::Open(std::move(block), ReaderOptions(), &reader));
 
   if (FLAGS_print_meta) {
-    cout << "Header:\n" << reader->header().DebugString() << endl;
-    cout << "Footer:\n" << reader->footer().DebugString() << endl;
+    cout << "Header:\n" << SecureDebugString(reader->header()) << endl;
+    cout << "Footer:\n" << SecureDebugString(reader->footer()) << endl;
   }
 
   if (FLAGS_print_rows) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tools/tool_action_local_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index ec4e8b5..ad728b5 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -426,7 +426,7 @@ Status DumpTabletMeta(FsManager* fs_manager,
 
   tablet::TabletSuperBlockPB pb;
   RETURN_NOT_OK_PREPEND(meta->ToSuperBlock(&pb), "Could not get superblock");
-  cout << "Superblock:\n" << pb.DebugString() << endl;
+  cout << "Superblock:\n" << SecureDebugString(pb) << endl;
 
   return Status::OK();
 }
@@ -457,7 +457,7 @@ Status DumpCFileBlockInternal(FsManager* fs_manager,
   RETURN_NOT_OK(CFileReader::Open(std::move(block), ReaderOptions(), &reader));
 
   cout << Indent(indent) << "CFile Header: "
-       << reader->header().ShortDebugString() << endl;
+       << SecureShortDebugString(reader->header()) << endl;
   if (!FLAGS_dump_data) {
     return Status::OK();
   }
@@ -587,7 +587,7 @@ Status DumpRowSetInternal(FsManager* fs_manager,
   tablet::RowSetDataPB pb;
   rs_meta->ToProtobuf(&pb);
 
-  cout << Indent(indent) << "RowSet metadata: " << pb.DebugString()
+  cout << Indent(indent) << "RowSet metadata: " << SecureDebugString(pb)
        << endl << endl;
 
   RowSetMetadata::ColumnIdToBlockIdMap col_blocks =

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tools/tool_action_remote_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_remote_replica.cc b/src/kudu/tools/tool_action_remote_replica.cc
index 57fa19d..3e6656d 100644
--- a/src/kudu/tools/tool_action_remote_replica.cc
+++ b/src/kudu/tools/tool_action_remote_replica.cc
@@ -45,6 +45,7 @@
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_admin.proxy.h"
 #include "kudu/tserver/tserver_service.proxy.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
@@ -114,7 +115,7 @@ class ReplicaDumper {
 
       if (resp.has_error()) {
         return Status::IOError("Failed to read: ",
-                               resp.error().ShortDebugString());
+                               SecureShortDebugString(resp.error()));
       }
 
       // The first response has a scanner ID. We use this for all subsequent
@@ -225,7 +226,7 @@ Status DeleteReplica(const RunnerContext& context) {
                         "DeleteTablet() failed");
   if (resp.has_error()) {
     return Status::IOError("Failed to delete tablet: ",
-                           resp.error().ShortDebugString());
+                           SecureShortDebugString(resp.error()));
   }
   return Status::OK();
 }
@@ -322,7 +323,7 @@ Status CopyReplica(const RunnerContext& context) {
     req.set_caller_term(std::numeric_limits<int64_t>::max());
   }
 
-  LOG(INFO) << "Sending copy replica request:\n" << req.DebugString();
+  LOG(INFO) << "Sending copy replica request:\n" << SecureDebugString(req);
   LOG(WARNING) << "NOTE: this copy may happen asynchronously "
                << "and may timeout if the tablet size is large. Watch the logs on "
                << "the target tablet server for indication of progress.";

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/heartbeater.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/heartbeater.cc b/src/kudu/tserver/heartbeater.cc
index f16214c..e8aca16 100644
--- a/src/kudu/tserver/heartbeater.cc
+++ b/src/kudu/tserver/heartbeater.cc
@@ -34,10 +34,11 @@
 #include "kudu/tserver/tablet_server_options.h"
 #include "kudu/tserver/ts_tablet_manager.h"
 #include "kudu/util/flag_tags.h"
-#include "kudu/util/thread.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
+#include "kudu/util/thread.h"
 #include "kudu/util/version_info.h"
 
 DEFINE_int32(heartbeat_rpc_timeout_ms, 15000,
@@ -384,7 +385,7 @@ Status Heartbeater::Thread::DoHeartbeat() {
   RpcController rpc;
   rpc.set_timeout(MonoDelta::FromMilliseconds(FLAGS_heartbeat_rpc_timeout_ms));
 
-  VLOG(2) << "Sending heartbeat:\n" << req.DebugString();
+  VLOG(2) << "Sending heartbeat:\n" << SecureDebugString(req);
   master::TSHeartbeatResponsePB resp;
   RETURN_NOT_OK_PREPEND(proxy_->TSHeartbeat(req, &resp, &rpc),
                         "Failed to send heartbeat to master");
@@ -393,7 +394,7 @@ Status Heartbeater::Thread::DoHeartbeat() {
   }
 
   VLOG(2) << Substitute("Received heartbeat response from $0:\n$1",
-                        master_address_.ToString(), resp.DebugString());
+                        master_address_.ToString(), SecureDebugString(resp));
 
   // If we've detected that our master was elected leader, send a full tablet
   // report in the next heartbeat.

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_copy_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client.cc b/src/kudu/tserver/tablet_copy_client.cc
index 72fd7c9..e0e0e97 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -44,6 +44,7 @@
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 
 DEFINE_int32(tablet_copy_begin_session_timeout_ms, 3000,
              "Tablet server RPC client timeout for BeginTabletCopySession calls. "
@@ -184,7 +185,7 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
   if (resp.superblock().tablet_data_state() != tablet::TABLET_DATA_READY) {
     Status s = Status::IllegalState("Remote peer (" + copy_peer_uuid + ")" +
                                     " is currently copying itself!",
-                                    resp.superblock().ShortDebugString());
+                                    SecureShortDebugString(resp.superblock()));
     LOG_WITH_PREFIX(WARNING) << s.ToString();
     return s;
   }
@@ -295,7 +296,7 @@ Status TabletCopyClient::ExtractRemoteError(const rpc::ErrorStatusPB& remote_err
               TabletCopyErrorPB::Code_Name(error.code()) + " from remote service");
   } else {
     return Status::InvalidArgument("Unable to decode tablet copy RPC error message",
-                                   remote_error.ShortDebugString());
+                                   SecureShortDebugString(remote_error));
   }
 }
 
@@ -519,7 +520,8 @@ Status TabletCopyClient::DownloadFile(const DataIdPB& data_id,
 
     // Sanity-check for corruption.
     RETURN_NOT_OK_PREPEND(VerifyData(offset, resp.chunk()),
-                          Substitute("Error validating data item $0", data_id.ShortDebugString()));
+                          Substitute("Error validating data item $0",
+                                     SecureShortDebugString(data_id)));
 
     // Write the data.
     RETURN_NOT_OK(appendable->Append(resp.chunk().data()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_copy_service-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service-test.cc b/src/kudu/tserver/tablet_copy_service-test.cc
index 9eaaa6c..fc88155 100644
--- a/src/kudu/tserver/tablet_copy_service-test.cc
+++ b/src/kudu/tserver/tablet_copy_service-test.cc
@@ -34,6 +34,7 @@
 #include "kudu/util/crc.h"
 #include "kudu/util/env_util.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util.h"
 
@@ -172,7 +173,7 @@ class TabletCopyServiceTest : public TabletCopyTest {
     const Status app_status = ExtractRemoteError(remote_error);
     const TabletCopyErrorPB& error =
         remote_error->GetExtension(TabletCopyErrorPB::tablet_copy_error_ext);
-    ASSERT_EQ(app_code, error.code()) << error.ShortDebugString();
+    ASSERT_EQ(app_code, error.code()) << SecureShortDebugString(error);
     ASSERT_EQ(status_code_string, app_status.CodeAsString()) << app_status.ToString();
     LOG(INFO) << app_status.ToString();
   }
@@ -239,7 +240,7 @@ TEST_F(TabletCopyServiceTest, TestBeginConcurrently) {
   }
   // Verify that all threads got the same result.
   for (int i = 1; i < threads.size(); i++) {
-    ASSERT_EQ(sblocks[i].DebugString(), sblocks[0].DebugString());
+    ASSERT_EQ(SecureDebugString(sblocks[i]), SecureDebugString(sblocks[0]));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_copy_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service.cc b/src/kudu/tserver/tablet_copy_service.cc
index 78b0fec..f3fcc7e 100644
--- a/src/kudu/tserver/tablet_copy_service.cc
+++ b/src/kudu/tserver/tablet_copy_service.cc
@@ -34,6 +34,7 @@
 #include "kudu/util/crc.h"
 #include "kudu/util/fault_injection.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/pb_util.h"
 
 // Note, this macro assumes the existence of a local var named 'context'.
 #define RPC_RETURN_APP_ERROR(app_err, message, s) \
@@ -285,24 +286,24 @@ Status TabletCopyServiceImpl::ValidateFetchRequestDataId(
     *app_error = TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST;
     return Status::InvalidArgument(
         Substitute("Only one of BlockId or segment sequence number are required, "
-            "but both were specified. DataTypeID: $0", data_id.ShortDebugString()));
+            "but both were specified. DataTypeID: $0", SecureShortDebugString(data_id)));
   } else if (PREDICT_FALSE(!data_id.has_block_id() && !data_id.has_wal_segment_seqno())) {
     *app_error = TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST;
     return Status::InvalidArgument(
         Substitute("Only one of BlockId or segment sequence number are required, "
-            "but neither were specified. DataTypeID: $0", data_id.ShortDebugString()));
+            "but neither were specified. DataTypeID: $0", SecureShortDebugString(data_id)));
   }
 
   if (data_id.type() == DataIdPB::BLOCK) {
     if (PREDICT_FALSE(!data_id.has_block_id())) {
       return Status::InvalidArgument("block_id must be specified for type == BLOCK",
-                                     data_id.ShortDebugString());
+                                     SecureShortDebugString(data_id));
     }
   } else {
     if (PREDICT_FALSE(!data_id.wal_segment_seqno())) {
       return Status::InvalidArgument(
           "segment sequence number must be specified for type == LOG_SEGMENT",
-          data_id.ShortDebugString());
+          SecureShortDebugString(data_id));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_copy_source_session-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_source_session-test.cc b/src/kudu/tserver/tablet_copy_source_session-test.cc
index 522ce6f..c39c098 100644
--- a/src/kudu/tserver/tablet_copy_source_session-test.cc
+++ b/src/kudu/tserver/tablet_copy_source_session-test.cc
@@ -37,6 +37,7 @@
 #include "kudu/tablet/tablet_peer.h"
 #include "kudu/util/crc.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_util.h"
 #include "kudu/util/threadpool.h"
 
@@ -171,8 +172,8 @@ class TabletCopyTest : public KuduTabletTest {
           new tablet::LatchTransactionCompletionCallback<WriteResponsePB>(&latch, &resp)));
       ASSERT_OK(tablet_peer_->SubmitWrite(std::move(state)));
       latch.Wait();
-      ASSERT_FALSE(resp.has_error()) << "Request failed: " << resp.error().ShortDebugString();
-      ASSERT_EQ(0, resp.per_row_errors_size()) << "Insert error: " << resp.ShortDebugString();
+      ASSERT_FALSE(resp.has_error()) << "Request failed: " << SecureShortDebugString(resp.error());
+      ASSERT_EQ(0, resp.per_row_errors_size()) << "Insert error: " << SecureShortDebugString(resp);
     }
     ASSERT_OK(tablet()->Flush());
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_copy_source_session.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_source_session.cc b/src/kudu/tserver/tablet_copy_source_session.cc
index caa22d1..dc2f727 100644
--- a/src/kudu/tserver/tablet_copy_source_session.cc
+++ b/src/kudu/tserver/tablet_copy_source_session.cc
@@ -30,6 +30,7 @@
 #include "kudu/tablet/tablet_peer.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/mutex.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/trace.h"
 
@@ -90,7 +91,7 @@ Status TabletCopySourceSession::Init() {
   vector<BlockIdPB> data_blocks;
   TabletMetadata::CollectBlockIdPBs(tablet_superblock_, &data_blocks);
   for (const BlockIdPB& block_id : data_blocks) {
-    VLOG(1) << "Opening block " << block_id.DebugString();
+    VLOG(1) << "Opening block " << SecureDebugString(block_id);
     RETURN_NOT_OK(OpenBlockUnlocked(BlockId::FromPB(block_id)));
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_server-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test-base.h b/src/kudu/tserver/tablet_server-test-base.h
index e245c3f..52448d4 100644
--- a/src/kudu/tserver/tablet_server-test-base.h
+++ b/src/kudu/tserver/tablet_server-test-base.h
@@ -52,6 +52,7 @@
 #include "kudu/tserver/ts_tablet_manager.h"
 #include "kudu/util/maintenance_manager.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_graph.h"
 #include "kudu/util/test_util.h"
 
@@ -148,8 +149,8 @@ class TabletServerTestBase : public KuduTest {
                    req.mutable_row_operations());
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
 
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error())<< resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error())<< SecureShortDebugString(resp);
     ASSERT_EQ(0, resp.per_row_errors_size());
     if (ts) {
       ts->AddValue(1);
@@ -231,7 +232,7 @@ class TabletServerTestBase : public KuduTest {
       if (resp.has_error() || resp.per_row_errors_size() > 0) {
         LOG(FATAL) << "Failed to insert batch "
                    << first_row_in_batch << "-" << last_row_in_batch
-                   << ": " << resp.DebugString();
+                   << ": " << SecureDebugString(resp);
       }
 
       inserted_since_last_report += count / num_batches;
@@ -267,10 +268,10 @@ class TabletServerTestBase : public KuduTest {
       AddTestKeyToPB(RowOperationsPB::DELETE, schema_, rowid, ops);
     }
 
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error()) << resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error()) << SecureShortDebugString(resp);
   }
 
   void BuildTestRow(int index, KuduPartialRow* row) {
@@ -302,9 +303,9 @@ class TabletServerTestBase : public KuduTest {
       rpc.Reset();
       req.set_batch_size_bytes(10000);
       req.set_call_seq_id(call_seq_id);
-      SCOPED_TRACE(req.DebugString());
+      SCOPED_TRACE(SecureDebugString(req));
       ASSERT_OK(DCHECK_NOTNULL(proxy)->Scan(req, &resp, &rpc));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
       ASSERT_FALSE(resp.has_error());
 
       StringifyRowsFromResponse(projection, rpc, resp, results);
@@ -417,9 +418,9 @@ class TabletServerTestBase : public KuduTest {
 
     // Send the call
     {
-      SCOPED_TRACE(req.DebugString());
+      SCOPED_TRACE(SecureDebugString(req));
       ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
       ASSERT_TRUE(resp.has_error());
       ASSERT_EQ(expected_code, resp.error().code());
       ASSERT_STR_CONTAINS(resp.error().status().message(), expected_message);
@@ -441,9 +442,9 @@ class TabletServerTestBase : public KuduTest {
 
     // Send the call
     {
-      SCOPED_TRACE(req.DebugString());
+      SCOPED_TRACE(SecureDebugString(req));
       ASSERT_OK(proxy_->Scan(req, resp, &rpc));
-      SCOPED_TRACE(resp->DebugString());
+      SCOPED_TRACE(SecureDebugString(*resp));
       ASSERT_FALSE(resp->has_error());
       ASSERT_TRUE(resp->has_more_results());
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_server-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index a82ed4b..971a76c 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -32,6 +32,7 @@
 #include "kudu/tserver/heartbeater.h"
 #include "kudu/util/crc.h"
 #include "kudu/util/curl_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/url-coding.h"
 #include "kudu/util/zlib.h"
 
@@ -112,7 +113,7 @@ TEST_F(TabletServerTest, TestSetFlags) {
     req.set_flag("foo");
     req.set_value("bar");
     ASSERT_OK(proxy.SetFlag(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_EQ(server::SetFlagResponsePB::NO_SUCH_FLAG, resp.result());
     EXPECT_TRUE(resp.msg().empty());
   }
@@ -124,7 +125,7 @@ TEST_F(TabletServerTest, TestSetFlags) {
     req.set_flag("metrics_retirement_age_ms");
     req.set_value("12345");
     ASSERT_OK(proxy.SetFlag(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_EQ(server::SetFlagResponsePB::SUCCESS, resp.result());
     EXPECT_EQ(resp.msg(), "metrics_retirement_age_ms set to 12345\n");
     EXPECT_EQ(Substitute("$0", old_val), resp.old_value());
@@ -137,7 +138,7 @@ TEST_F(TabletServerTest, TestSetFlags) {
     req.set_flag("metrics_retirement_age_ms");
     req.set_value("foo");
     ASSERT_OK(proxy.SetFlag(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_EQ(server::SetFlagResponsePB::BAD_VALUE, resp.result());
     EXPECT_EQ(resp.msg(), "Unable to set flag: bad value");
     EXPECT_EQ(12345, FLAGS_metrics_retirement_age_ms);
@@ -149,7 +150,7 @@ TEST_F(TabletServerTest, TestSetFlags) {
     req.set_flag("tablet_bloom_target_fp_rate");
     req.set_value("1.0");
     ASSERT_OK(proxy.SetFlag(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_EQ(server::SetFlagResponsePB::NOT_SAFE, resp.result());
   }
 
@@ -160,7 +161,7 @@ TEST_F(TabletServerTest, TestSetFlags) {
     req.set_value("1.0");
     req.set_force(true);
     ASSERT_OK(proxy.SetFlag(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_EQ(server::SetFlagResponsePB::SUCCESS, resp.result());
   }
 }
@@ -283,9 +284,9 @@ TEST_F(TabletServerTest, TestInsert) {
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 1234, 5678, "hello world via RPC",
                    req.mutable_row_operations());
 
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::MISMATCHED_SCHEMA, resp.error().code());
     Status s = StatusFromPB(resp.error().status());
@@ -300,9 +301,9 @@ TEST_F(TabletServerTest, TestInsert) {
   {
     controller.Reset();
     ASSERT_OK(SchemaToPB(schema_, req.mutable_schema()));
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
     req.clear_row_operations();
   }
@@ -315,9 +316,9 @@ TEST_F(TabletServerTest, TestInsert) {
 
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 1234, 5678,
                    "hello world via RPC", data);
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
     req.clear_row_operations();
     ASSERT_EQ(1, rows_inserted->value());
@@ -336,10 +337,10 @@ TEST_F(TabletServerTest, TestInsert) {
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 2, 1, "also not a dupe key", data);
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 1234, 1, "I am a duplicate key", data);
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 3, 1, kTooLargeValue, data);
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error()) << resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error()) << SecureShortDebugString(resp);
     ASSERT_EQ(3, rows_inserted->value());  // This counter only counts successful inserts.
     ASSERT_EQ(2, resp.per_row_errors().size());
 
@@ -399,9 +400,9 @@ TEST_F(TabletServerTest, TestExternalConsistencyModes_ClientPropagated) {
   req.set_external_consistency_mode(CLIENT_PROPAGATED);
 
   req.set_propagated_timestamp(current.ToUint64());
-  SCOPED_TRACE(req.DebugString());
+  SCOPED_TRACE(SecureDebugString(req));
   ASSERT_OK(proxy_->Write(req, &resp, &controller));
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   ASSERT_FALSE(resp.has_error());
   req.clear_row_operations();
   ASSERT_EQ(1, rows_inserted->value());
@@ -451,9 +452,9 @@ TEST_F(TabletServerTest, TestExternalConsistencyModes_CommitWait) {
   // set the external consistency mode to COMMIT_WAIT
   req.set_external_consistency_mode(COMMIT_WAIT);
 
-  SCOPED_TRACE(req.DebugString());
+  SCOPED_TRACE(SecureDebugString(req));
   ASSERT_OK(proxy_->Write(req, &resp, &controller));
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   ASSERT_FALSE(resp.has_error());
   req.clear_row_operations();
   ASSERT_EQ(1, rows_inserted->value());
@@ -514,10 +515,10 @@ TEST_F(TabletServerTest, TestInsertAndMutate) {
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 1, 1, "original1", data);
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 2, 2, "original2", data);
     AddTestRowToPB(RowOperationsPB::INSERT, schema_, 3, 3, "original3", data);
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error()) << resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error()) << SecureShortDebugString(resp);
     ASSERT_EQ(0, resp.per_row_errors().size());
     ASSERT_EQ(3, rows_inserted->value());
     ASSERT_EQ(0, rows_updated->value());
@@ -537,10 +538,10 @@ TEST_F(TabletServerTest, TestInsertAndMutate) {
                    req.mutable_row_operations());
     AddTestRowToPB(RowOperationsPB::UPDATE, schema_, 3, 4, "mutation3",
                    req.mutable_row_operations());
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error()) << resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error()) << SecureShortDebugString(resp);
     ASSERT_EQ(0, resp.per_row_errors().size());
     ASSERT_EQ(3, rows_inserted->value());
     ASSERT_EQ(3, rows_updated->value());
@@ -556,10 +557,10 @@ TEST_F(TabletServerTest, TestInsertAndMutate) {
 
     AddTestRowToPB(RowOperationsPB::UPDATE, schema_, 1234, 2, "mutated",
                    req.mutable_row_operations());
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error()) << resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error()) << SecureShortDebugString(resp);
     ASSERT_EQ(1, resp.per_row_errors().size());
     ASSERT_EQ(3, rows_updated->value());
     controller.Reset();
@@ -573,10 +574,10 @@ TEST_F(TabletServerTest, TestInsertAndMutate) {
     ASSERT_OK(SchemaToPB(schema_, req.mutable_schema()));
 
     AddTestKeyToPB(RowOperationsPB::DELETE, schema_, 1, req.mutable_row_operations());
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error())<< resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error())<< SecureShortDebugString(resp);
     ASSERT_EQ(0, resp.per_row_errors().size());
     ASSERT_EQ(3, rows_updated->value());
     ASSERT_EQ(1, rows_deleted->value());
@@ -592,10 +593,10 @@ TEST_F(TabletServerTest, TestInsertAndMutate) {
 
     AddTestRowToPB(RowOperationsPB::UPDATE, schema_, 1, 2, "mutated1",
                    req.mutable_row_operations());
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error())<< resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error())<< SecureShortDebugString(resp);
     ASSERT_EQ(1, resp.per_row_errors().size());
     controller.Reset();
   }
@@ -626,19 +627,19 @@ TEST_F(TabletServerTest, TestInsertAndMutate) {
     // op 4: update a row with a too-large value (fail)
     AddTestRowToPB(RowOperationsPB::UPDATE, schema_, 4, 6, kTooLargeValue, ops);
 
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
-    ASSERT_FALSE(resp.has_error())<< resp.ShortDebugString();
+    SCOPED_TRACE(SecureDebugString(resp));
+    ASSERT_FALSE(resp.has_error())<< SecureShortDebugString(resp);
     ASSERT_EQ(3, resp.per_row_errors().size());
     EXPECT_EQ("row_index: 0 error { code: NOT_FOUND message: \"key not found\" }",
-              resp.per_row_errors(0).ShortDebugString());
+              SecureShortDebugString(resp.per_row_errors(0)));
     EXPECT_EQ("row_index: 2 error { code: NOT_FOUND message: \"key not found\" }",
-              resp.per_row_errors(1).ShortDebugString());
+              SecureShortDebugString(resp.per_row_errors(1)));
     EXPECT_EQ("row_index: 4 error { code: INVALID_ARGUMENT message: "
               "\"value too large for column \\'string_val\\' (102400 bytes, "
               "maximum is 65536 bytes)\" }",
-              resp.per_row_errors(2).ShortDebugString());
+              SecureShortDebugString(resp.per_row_errors(2)));
     controller.Reset();
   }
 
@@ -683,9 +684,9 @@ TEST_F(TabletServerTest, TestInvalidWriteRequest_BadSchema) {
     RowOperationsPBEncoder enc(data);
     enc.Add(RowOperationsPB::INSERT, row);
 
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::MISMATCHED_SCHEMA, resp.error().code());
     ASSERT_STR_CONTAINS(resp.error().status().message(),
@@ -704,9 +705,9 @@ TEST_F(TabletServerTest, TestInvalidWriteRequest_BadSchema) {
 
     AddTestKeyToPB(RowOperationsPB::UPDATE, bad_schema_with_ids, 1,
                    req.mutable_row_operations());
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::INVALID_SCHEMA, resp.error().code());
     ASSERT_STR_CONTAINS(resp.error().status().message(),
@@ -1104,10 +1105,10 @@ TEST_F(TabletServerTest, TestSnapshotScan) {
     const Timestamp pre_scan_ts = mini_server_->server()->clock()->Now();
     // Send the call
     {
-      SCOPED_TRACE(req.DebugString());
+      SCOPED_TRACE(SecureDebugString(req));
       req.set_batch_size_bytes(0); // so it won't return data right away
       ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
       ASSERT_FALSE(resp.has_error());
     }
 
@@ -1163,10 +1164,10 @@ TEST_F(TabletServerTest, TestSnapshotScan_WithoutSnapshotTimestamp) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     req.set_batch_size_bytes(0); // so it won't return data right away
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1211,9 +1212,9 @@ TEST_F(TabletServerTest, TestSnapshotScan_SnapshotInTheFutureFails) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::INVALID_SNAPSHOT, resp.error().code());
   }
@@ -1243,9 +1244,9 @@ TEST_F(TabletServerTest, TestSnapshotScan_OpenScanner) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
   // Intentionally do not drain the scanner at the end, to leave it open.
@@ -1298,10 +1299,10 @@ TEST_F(TabletServerTest, TestSnapshotScan_LastRow) {
 
     // Send the call
     {
-      SCOPED_TRACE(req.DebugString());
+      SCOPED_TRACE(SecureDebugString(req));
       req.set_batch_size_bytes(0); // so it won't return data right away
       ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-      SCOPED_TRACE(resp.DebugString());
+      SCOPED_TRACE(SecureDebugString(resp));
       ASSERT_FALSE(resp.has_error());
     }
 
@@ -1310,10 +1311,10 @@ TEST_F(TabletServerTest, TestSnapshotScan_LastRow) {
       rpc.Reset();
       // Send the call.
       {
-        SCOPED_TRACE(req.DebugString());
+        SCOPED_TRACE(SecureDebugString(req));
         req.set_batch_size_bytes(i);
         ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-        SCOPED_TRACE(resp.DebugString());
+        SCOPED_TRACE(SecureDebugString(resp));
         ASSERT_FALSE(resp.has_error());
       }
       // Save the rows into 'results' vector.
@@ -1372,9 +1373,9 @@ TEST_F(TabletServerTest, TestSnapshotScan_SnapshotInTheFutureWithPropagatedTimes
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1431,9 +1432,9 @@ TEST_F(TabletServerTest, TestSnapshotScan__SnapshotInTheFutureBeyondPropagatedTi
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::INVALID_SNAPSHOT, resp.error().code());
   }
@@ -1460,9 +1461,9 @@ TEST_F(TabletServerTest, TestScanWithStringPredicates) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1505,9 +1506,9 @@ TEST_F(TabletServerTest, TestScanWithPredicates) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1551,9 +1552,9 @@ TEST_F(TabletServerTest, TestScanWithEncodedPredicates) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1577,9 +1578,9 @@ TEST_F(TabletServerTest, TestBadScannerID) {
 
   req.set_scanner_id("does-not-exist");
 
-  SCOPED_TRACE(req.DebugString());
+  SCOPED_TRACE(SecureDebugString(req));
   ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-  SCOPED_TRACE(resp.DebugString());
+  SCOPED_TRACE(SecureDebugString(resp));
   ASSERT_TRUE(resp.has_error());
   ASSERT_EQ(TabletServerErrorPB::SCANNER_EXPIRED, resp.error().code());
 }
@@ -1595,7 +1596,7 @@ TEST_F(TabletServerTest, TestInvalidScanRequest_NewScanAndScannerID) {
   scan->set_tablet_id(kTabletId);
   req.set_batch_size_bytes(0); // so it won't return data right away
   req.set_scanner_id("x");
-  SCOPED_TRACE(req.DebugString());
+  SCOPED_TRACE(SecureDebugString(req));
   Status s = proxy_->Scan(req, &resp, &rpc);
   ASSERT_FALSE(s.ok());
   ASSERT_STR_CONTAINS(s.ToString(), "Must not pass both a scanner_id and new_scan_request");
@@ -1679,9 +1680,9 @@ TEST_F(TabletServerTest, TestScan_NoResults) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
 
     // Because there are no entries, we should immediately return "no results".
@@ -1707,7 +1708,7 @@ TEST_F(TabletServerTest, TestScan_InvalidScanSeqId) {
     req.set_batch_size_bytes(0); // so it won't return data right away
 
     // Create the scanner
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
     ASSERT_FALSE(resp.has_error());
     ASSERT_TRUE(resp.has_more_results());
@@ -1724,7 +1725,7 @@ TEST_F(TabletServerTest, TestScan_InvalidScanSeqId) {
     req.set_batch_size_bytes(0); // so it won't return data right away
     req.set_call_seq_id(42); // should be 1
 
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::INVALID_SCAN_CALL_SEQ_ID, resp.error().code());
@@ -1766,10 +1767,10 @@ void TabletServerTest::DoOrderedScanTest(const Schema& projection,
   scan->set_order_mode(ORDERED);
 
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     req.set_batch_size_bytes(0); // so it won't return data right away
     ASSERT_OK(proxy_->Scan(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1842,9 +1843,9 @@ TEST_F(TabletServerTest, TestAlterSchema) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(admin_proxy_->AlterSchema(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1897,9 +1898,9 @@ TEST_F(TabletServerTest, TestAlterSchema_AddColWithoutWriteDefault) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(admin_proxy_->AlterSchema(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -1936,9 +1937,9 @@ TEST_F(TabletServerTest, TestCreateTablet_TabletExists) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(admin_proxy_->CreateTablet(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::TABLET_ALREADY_EXISTS, resp.error().code());
   }
@@ -1987,9 +1988,9 @@ TEST_F(TabletServerTest, TestDeleteTablet) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(admin_proxy_->DeleteTablet(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }
 
@@ -2028,9 +2029,9 @@ TEST_F(TabletServerTest, TestDeleteTablet_TabletNotCreated) {
 
   // Send the call
   {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(admin_proxy_->DeleteTablet(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::TABLET_NOT_FOUND, resp.error().code());
   }
@@ -2054,7 +2055,7 @@ TEST_F(TabletServerTest, TestConcurrentDeleteTablet) {
   req.set_delete_type(tablet::TABLET_DATA_DELETED);
 
   for (int i = 0; i < kNumDeletes; i++) {
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     admin_proxy_->DeleteTabletAsync(req, &responses[i], &rpcs[i],
                                     boost::bind(&CountDownLatch::CountDown, &latch));
   }
@@ -2064,7 +2065,7 @@ TEST_F(TabletServerTest, TestConcurrentDeleteTablet) {
   for (int i = 0; i < kNumDeletes; i++) {
     ASSERT_TRUE(rpcs[i].finished());
     LOG(INFO) << "STATUS " << i << ": " << rpcs[i].status().ToString();
-    LOG(INFO) << "RESPONSE " << i << ": " << responses[i].DebugString();
+    LOG(INFO) << "RESPONSE " << i << ": " << SecureDebugString(responses[i]);
     if (!responses[i].has_error()) {
       num_success++;
     }
@@ -2171,9 +2172,9 @@ TEST_F(TabletServerTest, TestWriteOutOfBounds) {
   for (const RowOperationsPB::Type &op : ops) {
     RowOperationsPB* data = req.mutable_row_operations();
     AddTestRowToPB(op, schema_, 20, 1, "1", data);
-    SCOPED_TRACE(req.DebugString());
+    SCOPED_TRACE(SecureDebugString(req));
     ASSERT_OK(proxy_->Write(req, &resp, &controller));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
 
     ASSERT_TRUE(resp.has_error());
     ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, resp.error().code());
@@ -2236,7 +2237,7 @@ TEST_F(TabletServerTest, TestChecksumScan) {
   total_crc += CalcTestRowChecksum(key);
   uint64_t first_crc = total_crc; // Cache first record checksum.
 
-  ASSERT_FALSE(resp.has_error()) << resp.error().DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp.error());
   ASSERT_EQ(total_crc, resp.checksum());
   ASSERT_FALSE(resp.has_more_results());
   EXPECT_TRUE(resp.has_resource_metrics());
@@ -2249,7 +2250,7 @@ TEST_F(TabletServerTest, TestChecksumScan) {
   ASSERT_OK(proxy_->Checksum(req, &resp, &controller));
   total_crc += CalcTestRowChecksum(key, false);
 
-  ASSERT_FALSE(resp.has_error()) << resp.error().DebugString();
+  ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp.error());
   ASSERT_EQ(total_crc, resp.checksum());
   ASSERT_FALSE(resp.has_more_results());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index bff3400..3aea832 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -56,6 +56,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/status_callback.h"
 #include "kudu/util/trace.h"
@@ -185,7 +186,7 @@ bool CheckUuidMatchOrRespond(TabletPeerLookupIf* tablet_manager,
   if (PREDICT_FALSE(!req->has_dest_uuid())) {
     // Maintain compat in release mode, but complain.
     string msg = Substitute("$0: Missing destination UUID in request from $1: $2",
-                            method_name, context->requestor_string(), req->ShortDebugString());
+                            method_name, context->requestor_string(), SecureShortDebugString(*req));
 #ifdef NDEBUG
     KLOG_EVERY_N(ERROR, 100) << msg;
 #else
@@ -198,7 +199,7 @@ bool CheckUuidMatchOrRespond(TabletPeerLookupIf* tablet_manager,
                                                   "Local UUID: $1. Requested UUID: $2",
                                                   method_name, local_uuid, req->dest_uuid()));
     LOG(WARNING) << s.ToString() << ": from " << context->requestor_string()
-                 << ": " << req->ShortDebugString();
+                 << ": " << SecureShortDebugString(*req);
     SetupErrorAndRespond(resp->mutable_error(), s,
                          TabletServerErrorPB::WRONG_SERVER_UUID, context);
     return false;
@@ -529,7 +530,7 @@ void TabletServiceAdminImpl::AlterSchema(const AlterSchemaRequestPB* req,
   if (!CheckUuidMatchOrRespond(server_->tablet_manager(), "AlterSchema", req, resp, context)) {
     return;
   }
-  DVLOG(3) << "Received Alter Schema RPC: " << req->DebugString();
+  DVLOG(3) << "Received Alter Schema RPC: " << SecureDebugString(*req);
 
   scoped_refptr<TabletPeer> tablet_peer;
   if (!LookupTabletPeerOrRespond(server_->tablet_manager(), req->tablet_id(), resp, context,
@@ -631,7 +632,7 @@ void TabletServiceAdminImpl::CreateTablet(const CreateTabletRequestPB* req,
             << " (table=" << req->table_name()
             << " [id=" << req->table_id() << "]), partition="
             << partition_schema.PartitionDebugString(partition, schema);
-  VLOG(1) << "Full request: " << req->DebugString();
+  VLOG(1) << "Full request: " << SecureDebugString(*req);
 
   s = server_->tablet_manager()->CreateNewTablet(req->table_id(),
                                                  req->tablet_id(),
@@ -672,7 +673,7 @@ void TabletServiceAdminImpl::DeleteTablet(const DeleteTabletRequestPB* req,
             << " with delete_type " << TabletDataState_Name(delete_type)
             << (req->has_reason() ? (" (" + req->reason() + ")") : "")
             << " from " << context->requestor_string();
-  VLOG(1) << "Full request: " << req->DebugString();
+  VLOG(1) << "Full request: " << SecureDebugString(*req);
 
   boost::optional<int64_t> cas_config_opid_index_less_or_equal;
   if (req->has_cas_config_opid_index_less_or_equal()) {
@@ -695,7 +696,7 @@ void TabletServiceImpl::Write(const WriteRequestPB* req,
                               rpc::RpcContext* context) {
   TRACE_EVENT1("tserver", "TabletServiceImpl::Write",
                "tablet_id", req->tablet_id());
-  DVLOG(3) << "Received Write RPC: " << req->DebugString();
+  DVLOG(3) << "Received Write RPC: " << SecureDebugString(*req);
 
   scoped_refptr<TabletPeer> tablet_peer;
   if (!LookupTabletPeerOrRespond(server_->tablet_manager(), req->tablet_id(), resp, context,
@@ -797,7 +798,7 @@ ConsensusServiceImpl::~ConsensusServiceImpl() {
 void ConsensusServiceImpl::UpdateConsensus(const ConsensusRequestPB* req,
                                            ConsensusResponsePB* resp,
                                            rpc::RpcContext* context) {
-  DVLOG(3) << "Received Consensus Update RPC: " << req->DebugString();
+  DVLOG(3) << "Received Consensus Update RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "UpdateConsensus", req, resp, context)) {
     return;
   }
@@ -829,7 +830,7 @@ void ConsensusServiceImpl::UpdateConsensus(const ConsensusRequestPB* req,
 void ConsensusServiceImpl::RequestConsensusVote(const VoteRequestPB* req,
                                                 VoteResponsePB* resp,
                                                 rpc::RpcContext* context) {
-  DVLOG(3) << "Received Consensus Request Vote RPC: " << req->DebugString();
+  DVLOG(3) << "Received Consensus Request Vote RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "RequestConsensusVote", req, resp, context)) {
     return;
   }
@@ -854,7 +855,7 @@ void ConsensusServiceImpl::RequestConsensusVote(const VoteRequestPB* req,
 void ConsensusServiceImpl::ChangeConfig(const ChangeConfigRequestPB* req,
                                         ChangeConfigResponsePB* resp,
                                         RpcContext* context) {
-  DVLOG(3) << "Received ChangeConfig RPC: " << req->DebugString();
+  DVLOG(3) << "Received ChangeConfig RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "ChangeConfig", req, resp, context)) {
     return;
   }
@@ -878,7 +879,7 @@ void ConsensusServiceImpl::ChangeConfig(const ChangeConfigRequestPB* req,
 void ConsensusServiceImpl::GetNodeInstance(const GetNodeInstanceRequestPB* req,
                                            GetNodeInstanceResponsePB* resp,
                                            rpc::RpcContext* context) {
-  DVLOG(3) << "Received Get Node Instance RPC: " << req->DebugString();
+  DVLOG(3) << "Received Get Node Instance RPC: " << SecureDebugString(*req);
   resp->mutable_node_instance()->CopyFrom(tablet_manager_->NodeInstance());
   context->RespondSuccess();
 }
@@ -886,7 +887,7 @@ void ConsensusServiceImpl::GetNodeInstance(const GetNodeInstanceRequestPB* req,
 void ConsensusServiceImpl::RunLeaderElection(const RunLeaderElectionRequestPB* req,
                                              RunLeaderElectionResponsePB* resp,
                                              rpc::RpcContext* context) {
-  DVLOG(3) << "Received Run Leader Election RPC: " << req->DebugString();
+  DVLOG(3) << "Received Run Leader Election RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "RunLeaderElection", req, resp, context)) {
     return;
   }
@@ -912,7 +913,7 @@ void ConsensusServiceImpl::RunLeaderElection(const RunLeaderElectionRequestPB* r
 void ConsensusServiceImpl::LeaderStepDown(const LeaderStepDownRequestPB* req,
                                           LeaderStepDownResponsePB* resp,
                                           RpcContext* context) {
-  DVLOG(3) << "Received Leader stepdown RPC: " << req->DebugString();
+  DVLOG(3) << "Received Leader stepdown RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "LeaderStepDown", req, resp, context)) {
     return;
   }
@@ -936,7 +937,7 @@ void ConsensusServiceImpl::LeaderStepDown(const LeaderStepDownRequestPB* req,
 void ConsensusServiceImpl::GetLastOpId(const consensus::GetLastOpIdRequestPB *req,
                                        consensus::GetLastOpIdResponsePB *resp,
                                        rpc::RpcContext *context) {
-  DVLOG(3) << "Received GetLastOpId RPC: " << req->DebugString();
+  DVLOG(3) << "Received GetLastOpId RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "GetLastOpId", req, resp, context)) {
     return;
   }
@@ -971,7 +972,7 @@ void ConsensusServiceImpl::GetLastOpId(const consensus::GetLastOpIdRequestPB *re
 void ConsensusServiceImpl::GetConsensusState(const consensus::GetConsensusStateRequestPB *req,
                                              consensus::GetConsensusStateResponsePB *resp,
                                              rpc::RpcContext *context) {
-  DVLOG(3) << "Received GetConsensusState RPC: " << req->DebugString();
+  DVLOG(3) << "Received GetConsensusState RPC: " << SecureDebugString(*req);
   if (!CheckUuidMatchOrRespond(tablet_manager_, "GetConsensusState", req, resp, context)) {
     return;
   }
@@ -1146,7 +1147,7 @@ void TabletServiceImpl::ListTablets(const ListTabletsRequestPB* req,
 void TabletServiceImpl::Checksum(const ChecksumRequestPB* req,
                                  ChecksumResponsePB* resp,
                                  rpc::RpcContext* context) {
-  VLOG(1) << "Full request: " << req->DebugString();
+  VLOG(1) << "Full request: " << SecureDebugString(*req);
 
   // Validate the request: user must pass a new_scan_request or
   // a scanner ID, but not both.
@@ -1324,7 +1325,7 @@ static Status SetupScanSpec(const NewScanRequestPB& scan_pb,
   for (const ColumnRangePredicatePB& pred_pb : scan_pb.deprecated_range_predicates()) {
     if (!pred_pb.has_lower_bound() && !pred_pb.has_inclusive_upper_bound()) {
       return Status::InvalidArgument(
-        string("Invalid predicate ") + pred_pb.ShortDebugString() +
+        string("Invalid predicate ") + SecureShortDebugString(pred_pb) +
         ": has no lower or upper bound.");
     }
     ColumnSchema col(ColumnSchemaFromPB(pred_pb.column()));
@@ -1355,7 +1356,7 @@ static Status SetupScanSpec(const NewScanRequestPB& scan_pb,
     if (pred) {
       if (VLOG_IS_ON(3)) {
         VLOG(3) << "Parsed predicate " << pred->ToString()
-                << " from " << scan_pb.ShortDebugString();
+                << " from " << SecureShortDebugString(scan_pb);
       }
       ret->AddPredicate(*pred);
     }
@@ -1548,7 +1549,7 @@ Status TabletServiceImpl::HandleNewScanRequest(TabletPeer* tablet_peer,
     *error_code = tmp_error_code;
     return s;
   } else if (PREDICT_FALSE(!s.ok())) {
-    LOG(WARNING) << "Error setting up scanner with request " << req->ShortDebugString();
+    LOG(WARNING) << "Error setting up scanner with request " << SecureShortDebugString(*req);
     *error_code = TabletServerErrorPB::UNKNOWN_ERROR;
     return s;
   }
@@ -1641,7 +1642,7 @@ Status TabletServiceImpl::HandleContinueScanRequest(const ScanRequestPB* req,
   ScopedUnregisterScanner unreg_scanner(server_->scanner_manager(), scanner->id());
 
   VLOG(2) << "Found existing scanner " << scanner->id() << " for request: "
-          << req->ShortDebugString();
+          << SecureShortDebugString(*req);
   TRACE("Found scanner $0", scanner->id());
 
   if (batch_size_bytes == 0 && req->close_scanner()) {
@@ -1678,7 +1679,8 @@ Status TabletServiceImpl::HandleContinueScanRequest(const ScanRequestPB* req,
 
     Status s = iter->NextBlock(&block);
     if (PREDICT_FALSE(!s.ok())) {
-      LOG(WARNING) << "Copying rows from internal iterator for request " << req->ShortDebugString();
+      LOG(WARNING) << "Copying rows from internal iterator for request "
+                   << SecureShortDebugString(*req);
       *error_code = TabletServerErrorPB::UNKNOWN_ERROR;
       return s;
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/ts_tablet_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager-test.cc b/src/kudu/tserver/ts_tablet_manager-test.cc
index 5d55378..9aeca50 100644
--- a/src/kudu/tserver/ts_tablet_manager-test.cc
+++ b/src/kudu/tserver/ts_tablet_manager-test.cc
@@ -30,6 +30,7 @@
 #include "kudu/tserver/heartbeater.h"
 #include "kudu/tserver/mini_tablet_server.h"
 #include "kudu/tserver/tablet_server.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_util.h"
 
 #define ASSERT_REPORT_HAS_UPDATED_TABLET(report, tablet_id) \
@@ -160,18 +161,18 @@ static void AssertReportHasUpdatedTablet(const TabletReportPB& report,
       found_tablet = true;
       ASSERT_TRUE(reported_tablet.has_committed_consensus_state());
       ASSERT_TRUE(reported_tablet.committed_consensus_state().has_current_term())
-          << reported_tablet.ShortDebugString();
+          << SecureShortDebugString(reported_tablet);
       ASSERT_TRUE(reported_tablet.committed_consensus_state().has_leader_uuid())
-          << reported_tablet.ShortDebugString();
+          << SecureShortDebugString(reported_tablet);
       ASSERT_TRUE(reported_tablet.committed_consensus_state().has_config());
       const RaftConfigPB& committed_config = reported_tablet.committed_consensus_state().config();
       ASSERT_EQ(kInvalidOpIdIndex, committed_config.opid_index());
       ASSERT_EQ(1, committed_config.peers_size());
       ASSERT_TRUE(committed_config.peers(0).has_permanent_uuid())
-          << reported_tablet.ShortDebugString();
+          << SecureShortDebugString(reported_tablet);
       ASSERT_EQ(committed_config.peers(0).permanent_uuid(),
                 reported_tablet.committed_consensus_state().leader_uuid())
-          << reported_tablet.ShortDebugString();
+          << SecureShortDebugString(reported_tablet);
     }
   }
   ASSERT_TRUE(found_tablet);
@@ -236,8 +237,8 @@ TEST_F(TsTabletManagerTest, TestTabletReports) {
   while (true) {
     bool found_tablet_2 = false;
     GenerateIncrementalTabletReport(&report);
-    ASSERT_TRUE(report.is_incremental()) << report.ShortDebugString();
-    ASSERT_MONOTONIC_REPORT_SEQNO(&seqno, report) << report.ShortDebugString();
+    ASSERT_TRUE(report.is_incremental()) << SecureShortDebugString(report);
+    ASSERT_MONOTONIC_REPORT_SEQNO(&seqno, report) << SecureShortDebugString(report);
     for (const ReportedTabletPB& reported_tablet : report.updated_tablets()) {
       if (reported_tablet.tablet_id() == "tablet-2") {
         found_tablet_2  = true;
@@ -249,7 +250,7 @@ TEST_F(TsTabletManagerTest, TestTabletReports) {
     ASSERT_TRUE(elapsed < timeout)
         << "Waited too long for tablet-2 to be marked dirty: "
         << elapsed.ToString() << ". "
-        << "Latest report: " << report.ShortDebugString();
+        << "Latest report: " << SecureShortDebugString(report);
     SleepFor(MonoDelta::FromMilliseconds(10));
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/tserver/tserver-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver-path-handlers.cc b/src/kudu/tserver/tserver-path-handlers.cc
index 95e42c8..9dc4b53 100644
--- a/src/kudu/tserver/tserver-path-handlers.cc
+++ b/src/kudu/tserver/tserver-path-handlers.cc
@@ -39,6 +39,7 @@
 #include "kudu/tserver/tablet_server.h"
 #include "kudu/tserver/ts_tablet_manager.h"
 #include "kudu/util/maintenance_manager.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/url-coding.h"
 
 using kudu::consensus::GetConsensusRole;
@@ -146,13 +147,13 @@ void TabletServerPathHandlers::HandleTransactionsPage(const Webserver::WebReques
         (*output) << Substitute(
           "<tr><th>$0</th><th>$1</th><th>$2</th><th>$3</th><th>$4</th></tr>\n",
           EscapeForHtmlToString(peer->tablet_id()),
-          EscapeForHtmlToString(inflight_tx.op_id().ShortDebugString()),
+          EscapeForHtmlToString(SecureShortDebugString(inflight_tx.op_id())),
           OperationType_Name(inflight_tx.tx_type()),
           total_time_str,
           EscapeForHtmlToString(description));
       } else {
         (*output) << "Tablet: " << peer->tablet_id() << endl;
-        (*output) << "Op ID: " << inflight_tx.op_id().ShortDebugString() << endl;
+        (*output) << "Op ID: " << SecureShortDebugString(inflight_tx.op_id()) << endl;
         (*output) << "Type: " << OperationType_Name(inflight_tx.tx_type()) << endl;
         (*output) << "Running: " << total_time_str;
         (*output) << description << endl;
@@ -557,7 +558,7 @@ void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::Web
   MaintenanceManagerStatusPB pb;
   manager->GetMaintenanceManagerStatusDump(&pb);
   if (ContainsKey(req.parsed_args, "raw")) {
-    *output << pb.DebugString();
+    *output << SecureDebugString(pb);
     return;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/util/pb_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util.cc b/src/kudu/util/pb_util.cc
index 09a642b..4c0ff5c 100644
--- a/src/kudu/util/pb_util.cc
+++ b/src/kudu/util/pb_util.cc
@@ -884,11 +884,11 @@ Status ReadablePBContainerFile::Dump(ostream* os, bool oneline) {
       s.ok();
       s = ReadNextPB(msg.get())) {
     if (oneline) {
-      *os << count++ << "\t" << msg->ShortDebugString() << endl;
+      *os << count++ << "\t" << SecureShortDebugString(*msg) << endl;
     } else {
       *os << "Message " << count << endl;
       *os << "-------" << endl;
-      *os << msg->DebugString() << endl;
+      *os << SecureDebugString(*msg) << endl;
       count++;
     }
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/util/pb_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util.h b/src/kudu/util/pb_util.h
index 513f033..82bf929 100644
--- a/src/kudu/util/pb_util.h
+++ b/src/kudu/util/pb_util.h
@@ -486,5 +486,12 @@ class PbTracer : public debug::ConvertableToTraceFormat {
 };
 
 } // namespace pb_util
+
+// TODO(todd) Replacing all Message::ToString call sites for KUDU-1812
+// is much easier if these are available in the 'kudu' namespace. We should
+// consider removing these imports and move them to all call sites.
+using pb_util::SecureDebugString; // NOLINT
+using pb_util::SecureShortDebugString; // NOLINT
+
 } // namespace kudu
 #endif


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

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/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 bccedbb..b86aab7 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);
   }
 }
 
@@ -1191,7 +1192,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);
   }
 }
 
@@ -1254,7 +1255,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) {
@@ -1304,7 +1305,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);
@@ -1313,7 +1314,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.
@@ -1340,9 +1341,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
@@ -1380,7 +1382,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);
@@ -1391,7 +1393,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();
@@ -1399,7 +1401,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.
   vector<string> results;
@@ -1439,7 +1441,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.
@@ -1448,7 +1450,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.
   {
@@ -1464,7 +1466,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");
@@ -1478,7 +1480,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");
@@ -1497,7 +1499,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;
@@ -1515,7 +1517,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.
   {
@@ -1542,9 +1544,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");
@@ -1566,8 +1568,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
@@ -1578,7 +1580,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.
@@ -1619,7 +1621,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));
@@ -1709,11 +1711,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();
 }
@@ -1743,9 +1745,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);
   }
 }
 
@@ -2218,7 +2221,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;
@@ -2237,8 +2240,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();
@@ -2251,8 +2254,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
@@ -2308,7 +2311,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());
@@ -2877,10 +2880,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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/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/e7b45d37/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index 80e163d..8508e55 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -56,6 +56,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,
@@ -410,7 +411,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("...");
@@ -463,7 +464,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()));
@@ -724,7 +725,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)));
     }
   }
 
@@ -761,7 +762,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()) {
@@ -824,7 +825,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();
@@ -846,7 +848,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);
@@ -897,8 +899,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();
@@ -921,10 +923,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);
     }
@@ -942,10 +944,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();
@@ -1104,7 +1107,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)));
       }
     }
   }
@@ -1219,7 +1222,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
@@ -1360,7 +1363,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.
@@ -1424,7 +1427,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();
       }
@@ -1446,7 +1449,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();
@@ -1463,7 +1466,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
@@ -1491,7 +1494,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/e7b45d37/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/e7b45d37/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


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

Posted by da...@apache.org.
KUDU-1812. Replace PB DebugString calls with redactable variants

This uses the clang tool from the prior commit to replace all call sites
of Message::DebugString and Message::ShortDebugString with corresponding
SecureDebugString(msg) and SecureShortDebugString(msg) calls.

The commit was done using the tool except for the following:
- a few call sites inside macros weren't handled by the tool, which
  instead inserted 'TODO(PBString)' comments. I fixed those few cases by
  hand.
- re-wrapped to avoid long lines called out by ilint
- added appropriate #includes for the new calls

The only potentially controversial bit here is whether we should make
this change in the various tests. In fact, the tests that are checking
whether one PB matches another should probably not be redacting
anything. However, the tests also configure redaction to be disabled, so
the secure and regular variants should have identical output. I chose to
make the substitutions in test code as well as production code following
the reasoning that it is less cognitive load (and easier to check in
precommit) to say "thou shalt never use Message::DebugString" rather
than have different rules in the case of tests.

Change-Id: I2c5d1355bdfdbf2232aae8c0d809cc044790de28
Reviewed-on: http://gerrit.cloudera.org:8080/5562
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Reviewed-on: http://gerrit.cloudera.org:8080/5573
Reviewed-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/branch-1.2.x
Commit: e7b45d375096166102b3acb97ce9b81077001ebe
Parents: 5a6ef37
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Dec 22 21:40:39 2016 +0700
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Dec 23 02:23:13 2016 +0000

----------------------------------------------------------------------
 src/kudu/cfile/cfile_reader.cc                  |   9 +-
 src/kudu/cfile/index_block.cc                   |   3 +-
 src/kudu/client/batcher.cc                      |   8 +-
 src/kudu/client/client-internal.cc              |   5 +-
 src/kudu/client/meta_cache.cc                   |   5 +-
 src/kudu/common/partition.cc                    |   7 +-
 src/kudu/common/wire_protocol-test.cc           |   3 +-
 src/kudu/common/wire_protocol.cc                |  13 +-
 src/kudu/consensus/consensus-test-util.h        |  13 +-
 src/kudu/consensus/consensus_meta-test.cc       |   4 +-
 src/kudu/consensus/consensus_meta.cc            |   2 +-
 src/kudu/consensus/consensus_peers.cc           |  13 +-
 src/kudu/consensus/consensus_queue-test.cc      |   7 +-
 src/kudu/consensus/consensus_queue.cc           |  28 +--
 src/kudu/consensus/leader_election.cc           |   3 +-
 src/kudu/consensus/log-test-base.h              |  10 +-
 src/kudu/consensus/log.cc                       |   4 +-
 src/kudu/consensus/log_cache.cc                 |   3 +-
 src/kudu/consensus/log_reader.cc                |   6 +-
 src/kudu/consensus/peer_manager.cc              |   7 +-
 src/kudu/consensus/quorum_util.cc               |  28 +--
 src/kudu/consensus/raft_consensus.cc            |  70 +++----
 .../consensus/raft_consensus_quorum-test.cc     |  13 +-
 src/kudu/consensus/raft_consensus_state.cc      |  22 ++-
 src/kudu/fs/fs_manager.cc                       |   4 +-
 src/kudu/fs/log_block_manager.cc                |  12 +-
 .../integration-tests/cluster_itest_util.cc     |  15 +-
 .../create-table-stress-test.cc                 |   5 +-
 src/kudu/integration-tests/delete_table-test.cc |  15 +-
 .../exactly_once_writes-itest.cc                |   5 +-
 .../integration-tests/external_mini_cluster.cc  |   6 +-
 .../master_replication-itest.cc                 |   3 +-
 .../integration-tests/raft_consensus-itest.cc   |  81 +++++----
 src/kudu/integration-tests/registration-test.cc |   9 +-
 .../integration-tests/table_locations-itest.cc  |  11 +-
 src/kudu/integration-tests/ts_itest-base.h      |   5 +-
 .../ts_tablet_manager-itest.cc                  |   8 +-
 src/kudu/master/catalog_manager.cc              |  72 ++++----
 src/kudu/master/master-path-handlers.cc         |   5 +-
 src/kudu/master/master-test.cc                  |  40 ++--
 src/kudu/master/master_service.cc               |   5 +-
 src/kudu/master/sys_catalog-test.cc             |   5 +-
 src/kudu/master/sys_catalog.cc                  |   8 +-
 src/kudu/master/ts_descriptor.cc                |  14 +-
 src/kudu/master/ts_manager.cc                   |   6 +-
 src/kudu/rpc/exactly_once_rpc-test.cc           |  14 +-
 src/kudu/rpc/result_tracker.cc                  |  17 +-
 src/kudu/rpc/rpc-test-base.h                    |   3 +-
 src/kudu/rpc/rpc_context.cc                     |  12 +-
 src/kudu/rpc/rpc_stub-test.cc                   |  18 +-
 src/kudu/tablet/row_op.cc                       |   9 +-
 src/kudu/tablet/tablet_bootstrap-test.cc        |   4 +-
 src/kudu/tablet/tablet_bootstrap.cc             |  47 ++---
 src/kudu/tablet/tablet_metadata-test.cc         |   7 +-
 src/kudu/tablet/tablet_metadata.cc              |  10 +-
 src/kudu/tablet/tablet_peer-test.cc             |   9 +-
 src/kudu/tablet/tablet_peer.cc                  |   3 +-
 .../transactions/alter_schema_transaction.cc    |   3 +-
 .../tablet/transactions/transaction_driver.cc   |   7 +-
 .../tablet/transactions/write_transaction.cc    |   3 +-
 src/kudu/tools/kudu-tool-test.cc                |   5 +-
 src/kudu/tools/tool_action_common.cc            |  18 +-
 src/kudu/tools/tool_action_fs.cc                |   5 +-
 src/kudu/tools/tool_action_local_replica.cc     |   6 +-
 src/kudu/tools/tool_action_remote_replica.cc    |   7 +-
 src/kudu/tserver/heartbeater.cc                 |   7 +-
 src/kudu/tserver/tablet_copy_client.cc          |   8 +-
 src/kudu/tserver/tablet_copy_service-test.cc    |   5 +-
 src/kudu/tserver/tablet_copy_service.cc         |   9 +-
 .../tserver/tablet_copy_source_session-test.cc  |   5 +-
 src/kudu/tserver/tablet_copy_source_session.cc  |   3 +-
 src/kudu/tserver/tablet_server-test-base.h      |  25 +--
 src/kudu/tserver/tablet_server-test.cc          | 181 ++++++++++---------
 src/kudu/tserver/tablet_service.cc              |  42 +++--
 src/kudu/tserver/ts_tablet_manager-test.cc      |  15 +-
 src/kudu/tserver/tserver-path-handlers.cc       |   7 +-
 src/kudu/util/pb_util.cc                        |   4 +-
 src/kudu/util/pb_util.h                         |   7 +
 78 files changed, 620 insertions(+), 525 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/cfile/cfile_reader.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_reader.cc b/src/kudu/cfile/cfile_reader.cc
index 077d6f4..4bbc03b 100644
--- a/src/kudu/cfile/cfile_reader.cc
+++ b/src/kudu/cfile/cfile_reader.cc
@@ -38,6 +38,7 @@
 #include "kudu/util/malloc.h"
 #include "kudu/util/memory/overwrite.h"
 #include "kudu/util/object_pool.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/rle-encoding.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
@@ -142,8 +143,8 @@ Status CFileReader::InitOnce() {
                                       &type_encoding_info_));
 
   VLOG(2) << "Initialized CFile reader. "
-          << "Header: " << header_->DebugString()
-          << " Footer: " << footer_->DebugString()
+          << "Header: " << SecureDebugString(*header_)
+          << " Footer: " << SecureDebugString(*footer_)
           << " Type: " << type_info_->name();
 
   // The header/footer have been allocated; memory consumption has changed.
@@ -177,7 +178,7 @@ Status CFileReader::ReadAndParseHeader() {
     return Status::Corruption("Invalid cfile pb header");
   }
 
-  VLOG(2) << "Read header: " << header_->DebugString();
+  VLOG(2) << "Read header: " << SecureDebugString(*header_);
 
   return Status::OK();
 }
@@ -214,7 +215,7 @@ Status CFileReader::ReadAndParseFooter() {
     block_uncompressor_.reset(new CompressedBlockDecoder(codec));
   }
 
-  VLOG(2) << "Read footer: " << footer_->DebugString();
+  VLOG(2) << "Read footer: " << SecureDebugString(*footer_);
 
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/cfile/index_block.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/index_block.cc b/src/kudu/cfile/index_block.cc
index 92ec5a3..9ce3888 100644
--- a/src/kudu/cfile/index_block.cc
+++ b/src/kudu/cfile/index_block.cc
@@ -18,6 +18,7 @@
 #include "kudu/cfile/cfile_writer.h"
 #include "kudu/cfile/index_block.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/protobuf_util.h"
 
 namespace kudu {
@@ -161,7 +162,7 @@ Status IndexBlockReader::Parse(const Slice &data) {
   key_offsets_ = trailer_ptr - sizeof(uint32_t) * trailer_.num_entries();
   CHECK(trailer_ptr >= data_.data());
 
-  VLOG(2) << "Parsed index trailer: " << trailer_.DebugString();
+  VLOG(2) << "Parsed index trailer: " << SecureDebugString(trailer_);
 
   parsed_ = true;
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/client/batcher.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/batcher.cc b/src/kudu/client/batcher.cc
index 8b090f4..53a57d5 100644
--- a/src/kudu/client/batcher.cc
+++ b/src/kudu/client/batcher.cc
@@ -52,6 +52,7 @@
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/debug-util.h"
 #include "kudu/util/logging.h"
+#include "kudu/util/pb_util.h"
 
 using std::pair;
 using std::set;
@@ -294,7 +295,7 @@ WriteRpc::WriteRpc(const scoped_refptr<Batcher>& batcher,
   }
 
   if (VLOG_IS_ON(3)) {
-    VLOG(3) << "Created batch for " << tablet_id << ":\n" << req_.ShortDebugString();
+    VLOG(3) << "Created batch for " << tablet_id << ":\n" << SecureShortDebugString(req_);
   }
 }
 
@@ -742,12 +743,13 @@ void Batcher::ProcessWriteResponse(const WriteRpc& rpc,
       LOG(ERROR) << "Received a per_row_error for an out-of-bound op index "
                  << err_pb.row_index() << " (sent only "
                  << rpc.ops().size() << " ops)";
-      LOG(ERROR) << "Response from tablet " << rpc.tablet_id() << ":\n" << rpc.resp().DebugString();
+      LOG(ERROR) << "Response from tablet " << rpc.tablet_id() << ":\n"
+                 << SecureDebugString(rpc.resp());
       continue;
     }
     gscoped_ptr<KuduWriteOperation> op = std::move(rpc.ops()[err_pb.row_index()]->write_op);
     VLOG(2) << "Error on op " << op->ToString() << ": "
-            << err_pb.error().ShortDebugString();
+            << SecureShortDebugString(err_pb.error());
     Status op_status = StatusFromPB(err_pb.error());
     gscoped_ptr<KuduError> error(new KuduError(op.release(), op_status));
     error_collector_->AddError(std::move(error));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/client/client-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index e483be9..54ac7c5 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -43,6 +43,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/net/dns_resolver.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/thread_restrictions.h"
 
 using std::set;
@@ -187,7 +188,7 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
 
     if (s.IsNetworkError()) {
       KLOG_EVERY_N_SECS(WARNING, 1)
-          << "Unable to send the request (" << req.ShortDebugString()
+          << "Unable to send the request (" << SecureShortDebugString(req)
           << ") to leader Master (" << leader_master_hostport().ToString() << "): "
           << s.ToString();
       if (client->IsMultiMaster()) {
@@ -201,7 +202,7 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
     if (s.IsTimedOut()) {
       if (MonoTime::Now() < deadline) {
         KLOG_EVERY_N_SECS(WARNING, 1)
-            << "Unable to send the request (" << req.ShortDebugString()
+            << "Unable to send the request (" << SecureShortDebugString(req)
             << ") to leader Master (" << leader_master_hostport().ToString()
             << "): " << s.ToString();
         if (client->IsMultiMaster()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/client/meta_cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/meta_cache.cc b/src/kudu/client/meta_cache.cc
index 5de9652..1cee1fd 100644
--- a/src/kudu/client/meta_cache.cc
+++ b/src/kudu/client/meta_cache.cc
@@ -36,6 +36,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/net/dns_resolver.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 
 using std::map;
 using std::set;
@@ -769,7 +770,7 @@ void LookupRpc::SendRpcCb(const Status& status) {
 Status MetaCache::ProcessLookupResponse(const LookupRpc& rpc,
                                         MetaCacheEntry* cache_entry) {
   VLOG(2) << "Processing master response for " << rpc.ToString()
-          << ". Response: " << rpc.resp().ShortDebugString();
+          << ". Response: " << SecureShortDebugString(rpc.resp());
 
   MonoTime expiration_time = MonoTime::Now() +
       MonoDelta::FromMilliseconds(rpc.resp().ttl_millis());
@@ -859,7 +860,7 @@ Status MetaCache::ProcessLookupResponse(const LookupRpc& rpc,
         DCHECK_EQ(tablet_lower_bound, remote->partition().partition_key_start());
         DCHECK_EQ(tablet_upper_bound, remote->partition().partition_key_end());
 
-        VLOG(3) << "Refreshing tablet " << tablet_id << ": " << tablet.ShortDebugString();
+        VLOG(3) << "Refreshing tablet " << tablet_id << ": " << SecureShortDebugString(tablet);
         remote->Refresh(ts_cache_, tablet.replicas());
 
         // Update the entry TTL.

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/common/partition.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition.cc b/src/kudu/common/partition.cc
index fcd23b1..084d897 100644
--- a/src/kudu/common/partition.cc
+++ b/src/kudu/common/partition.cc
@@ -31,6 +31,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/hash_util.h"
 #include "kudu/util/logging.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/url-coding.h"
 
 using std::pair;
@@ -106,7 +107,7 @@ Status ExtractColumnIds(const RepeatedPtrField<PartitionSchemaPB_ColumnIdentifie
         case PartitionSchemaPB_ColumnIdentifierPB::kId: {
           ColumnId column_id(identifier.id());
           if (schema.find_column_by_id(column_id) == Schema::kColumnNotFound) {
-            return Status::InvalidArgument("unknown column id", identifier.DebugString());
+            return Status::InvalidArgument("unknown column id", SecureDebugString(identifier));
           }
           column_ids->push_back(column_id);
           continue;
@@ -114,12 +115,12 @@ Status ExtractColumnIds(const RepeatedPtrField<PartitionSchemaPB_ColumnIdentifie
         case PartitionSchemaPB_ColumnIdentifierPB::kName: {
           int32_t column_idx = schema.find_column(identifier.name());
           if (column_idx == Schema::kColumnNotFound) {
-            return Status::InvalidArgument("unknown column", identifier.DebugString());
+            return Status::InvalidArgument("unknown column", SecureDebugString(identifier));
           }
           column_ids->push_back(schema.column_id(column_idx));
           continue;
         }
-        default: return Status::InvalidArgument("unknown column", identifier.DebugString());
+        default: return Status::InvalidArgument("unknown column", SecureDebugString(identifier));
       }
     }
     return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/common/wire_protocol-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/wire_protocol-test.cc b/src/kudu/common/wire_protocol-test.cc
index aeaa6b2..4ade664 100644
--- a/src/kudu/common/wire_protocol-test.cc
+++ b/src/kudu/common/wire_protocol-test.cc
@@ -24,6 +24,7 @@
 #include "kudu/common/rowblock.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/wire_protocol.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_macros.h"
@@ -208,7 +209,7 @@ TEST_F(WireProtocolTest, TestColumnarRowBlockToPB) {
   RowwiseRowBlockPB pb;
   faststring direct, indirect;
   SerializeRowBlock(block, &pb, nullptr, &direct, &indirect);
-  SCOPED_TRACE(pb.DebugString());
+  SCOPED_TRACE(SecureDebugString(pb));
   SCOPED_TRACE("Row data: " + direct.ToString());
   SCOPED_TRACE("Indirect data: " + indirect.ToString());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/common/wire_protocol.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index 0c3d23b..361be0d 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -31,6 +31,7 @@
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/safe_math.h"
 #include "kudu/util/slice.h"
 
@@ -147,7 +148,7 @@ Status StatusFromPB(const AppStatusPB& pb) {
       return Status::EndOfFile(pb.message(), "", posix_code);
     case AppStatusPB::UNKNOWN_ERROR:
     default:
-      LOG(WARNING) << "Unknown error code in status: " << pb.ShortDebugString();
+      LOG(WARNING) << "Unknown error code in status: " << SecureShortDebugString(pb);
       return Status::RuntimeError("(unknown error code)", pb.message(), posix_code);
   }
 }
@@ -268,7 +269,7 @@ Status ColumnPBsToSchema(const RepeatedPtrField<ColumnSchemaPB>& column_pbs,
     if (pb.is_key()) {
       if (!is_handling_key) {
         return Status::InvalidArgument(
-          "Got out-of-order key column", pb.ShortDebugString());
+          "Got out-of-order key column", SecureShortDebugString(pb));
       }
       num_key_columns++;
     } else {
@@ -401,12 +402,12 @@ Status ColumnPredicateFromPB(const Schema& schema,
                              const ColumnPredicatePB& pb,
                              optional<ColumnPredicate>* predicate) {
   if (!pb.has_column()) {
-    return Status::InvalidArgument("Column predicate must include a column", pb.DebugString());
+    return Status::InvalidArgument("Column predicate must include a column", SecureDebugString(pb));
   }
   const string& column = pb.column();
   int32_t idx = schema.find_column(column);
   if (idx == Schema::kColumnNotFound) {
-    return Status::InvalidArgument("unknown column in predicate", pb.DebugString());
+    return Status::InvalidArgument("unknown column in predicate", SecureDebugString(pb));
   }
   const ColumnSchema& col = schema.column(idx);
 
@@ -558,14 +559,14 @@ Status FindLeaderHostPort(const RepeatedPtrField<ServerEntryPB>& entries,
                           HostPort* leader_hostport) {
   for (const ServerEntryPB& entry : entries) {
     if (entry.has_error()) {
-      LOG(WARNING) << "Error encountered for server entry " << entry.ShortDebugString()
+      LOG(WARNING) << "Error encountered for server entry " << SecureShortDebugString(entry)
                    << ": " << StatusFromPB(entry.error()).ToString();
       continue;
     }
     if (!entry.has_role()) {
       return Status::IllegalState(
           strings::Substitute("Every server in must have a role, but entry ($0) has no role.",
-                              entry.ShortDebugString()));
+                              SecureShortDebugString(entry)));
     }
     if (entry.role() == consensus::RaftPeerPB::LEADER) {
       return HostPortFromPB(entry.registration().rpc_addresses(0), leader_hostport);

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/consensus-test-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus-test-util.h b/src/kudu/consensus/consensus-test-util.h
index ce30bfb..69954c3 100644
--- a/src/kudu/consensus/consensus-test-util.h
+++ b/src/kudu/consensus/consensus-test-util.h
@@ -37,6 +37,7 @@
 #include "kudu/server/clock.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/threadpool.h"
 
@@ -47,8 +48,8 @@
   OpId TOKENPASTE2(_left, __LINE__) = (left); \
   OpId TOKENPASTE2(_right, __LINE__) = (right); \
   if (!consensus::OpIdEquals(TOKENPASTE2(_left, __LINE__), TOKENPASTE2(_right,__LINE__))) \
-    FAIL() << "Expected: " << TOKENPASTE2(_right,__LINE__).ShortDebugString() << "\n" \
-           << "Value: " << TOKENPASTE2(_left,__LINE__).ShortDebugString() << "\n"
+    FAIL() << "Expected: " << SecureShortDebugString(TOKENPASTE2(_right,__LINE__)) << "\n" \
+           << "Value: " << SecureShortDebugString(TOKENPASTE2(_left,__LINE__)) << "\n"
 
 namespace kudu {
 namespace consensus {
@@ -235,7 +236,7 @@ class MockedPeerProxy : public TestPeerProxy {
   }
 
   virtual void set_update_response(const ConsensusResponsePB& update_response) {
-    CHECK(update_response.IsInitialized()) << update_response.ShortDebugString();
+    CHECK(update_response.IsInitialized()) << SecureShortDebugString(update_response);
     {
       std::lock_guard<simple_spinlock> l(lock_);
       update_response_ = update_response;
@@ -465,7 +466,7 @@ class LocalTestPeerProxy : public TestPeerProxy {
       miss_comm_ = false;
     }
     if (PREDICT_FALSE(miss_comm_copy)) {
-      VLOG(2) << this << ": injecting fault on " << request->ShortDebugString();
+      VLOG(2) << this << ": injecting fault on " << SecureShortDebugString(*request);
       SetResponseError(Status::IOError("Artificial error caused by communication "
           "failure injection."), final_response);
     } else {
@@ -495,7 +496,7 @@ class LocalTestPeerProxy : public TestPeerProxy {
     }
     if (!s.ok()) {
       LOG(WARNING) << "Could not Update replica with request: "
-                   << other_peer_req.ShortDebugString()
+                   << SecureShortDebugString(other_peer_req)
                    << " Status: " << s.ToString();
       SetResponseError(s, &other_peer_resp);
     }
@@ -524,7 +525,7 @@ class LocalTestPeerProxy : public TestPeerProxy {
     }
     if (!s.ok()) {
       LOG(WARNING) << "Could not RequestVote from replica with request: "
-                   << other_peer_req.ShortDebugString()
+                   << SecureShortDebugString(other_peer_req)
                    << " Status: " << s.ToString();
       SetResponseError(s, &other_peer_resp);
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/consensus_meta-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_meta-test.cc b/src/kudu/consensus/consensus_meta-test.cc
index e2d033c..11214a2 100644
--- a/src/kudu/consensus/consensus_meta-test.cc
+++ b/src/kudu/consensus/consensus_meta-test.cc
@@ -27,6 +27,7 @@
 #include "kudu/consensus/quorum_util.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -236,7 +237,8 @@ static void AssertConsensusMergeExpected(const unique_ptr<ConsensusMetadata>& cm
   // See header docs for ConsensusMetadata::MergeCommittedConsensusStatePB() for
   // a "spec" of these assertions.
   ASSERT_TRUE(!cmeta->has_pending_config());
-  ASSERT_EQ(cmeta->committed_config().ShortDebugString(), cstate.config().ShortDebugString());
+  ASSERT_EQ(SecureShortDebugString(cmeta->committed_config()),
+            SecureShortDebugString(cstate.config()));
   ASSERT_EQ("", cmeta->leader_uuid());
   ASSERT_EQ(expected_term, cmeta->current_term());
   if (expected_voted_for.empty()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/consensus_meta.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_meta.cc b/src/kudu/consensus/consensus_meta.cc
index 34f4506..fdb2e27 100644
--- a/src/kudu/consensus/consensus_meta.cc
+++ b/src/kudu/consensus/consensus_meta.cc
@@ -236,7 +236,7 @@ void ConsensusMetadata::UpdateActiveRole() {
   ConsensusStatePB cstate = ToConsensusStatePB(CONSENSUS_CONFIG_ACTIVE);
   active_role_ = GetConsensusRole(peer_uuid_, cstate);
   VLOG_WITH_PREFIX(1) << "Updating active role to " << RaftPeerPB::Role_Name(active_role_)
-                      << ". Consensus state: " << cstate.ShortDebugString();
+                      << ". Consensus state: " << SecureShortDebugString(cstate);
 }
 
 } // namespace consensus

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 92bb35a..bb0aef7 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -38,6 +38,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/threadpool.h"
 
 DEFINE_int32(consensus_rpc_timeout_ms, 1000,
@@ -217,7 +218,7 @@ void Peer::SendNextRequest(bool even_if_queue_empty) {
 
 
   VLOG_WITH_PREFIX_UNLOCKED(2) << "Sending to peer " << peer_pb().permanent_uuid() << ": "
-      << request_.ShortDebugString();
+      << SecureShortDebugString(request_);
   controller_.Reset();
 
   proxy_->UpdateAsync(&request_, &response_, &controller_,
@@ -266,7 +267,7 @@ void Peer::ProcessResponse() {
   Status s = thread_pool_->SubmitClosure(Bind(&Peer::DoProcessResponse, Unretained(this)));
   if (PREDICT_FALSE(!s.ok())) {
     LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to process peer response: " << s.ToString()
-        << ": " << response_.ShortDebugString();
+        << ": " << SecureShortDebugString(response_);
     sem_.Release();
   }
 }
@@ -275,7 +276,7 @@ void Peer::DoProcessResponse() {
   failed_attempts_ = 0;
 
   VLOG_WITH_PREFIX_UNLOCKED(2) << "Response from peer " << peer_pb().permanent_uuid() << ": "
-      << response_.ShortDebugString();
+      << SecureShortDebugString(response_);
 
   bool more_pending;
   queue_->ResponseFromPeer(peer_pb_.permanent_uuid(), response_, &more_pending);
@@ -311,7 +312,7 @@ void Peer::ProcessTabletCopyResponse() {
       queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
     } else {
       LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin Tablet Copy on peer: "
-                                        << tc_response_.ShortDebugString();
+                                        << SecureShortDebugString(tc_response_);
     }
   }
   sem_.Release();
@@ -447,7 +448,7 @@ Status SetPermanentUuidForRemotePeer(const shared_ptr<Messenger>& messenger,
       MonoDelta::FromMilliseconds(FLAGS_raft_get_node_instance_timeout_ms);
   int attempt = 1;
   while (true) {
-    VLOG(2) << "Getting uuid from remote peer. Request: " << req.ShortDebugString();
+    VLOG(2) << "Getting uuid from remote peer. Request: " << SecureShortDebugString(req);
 
     controller.Reset();
     Status s = proxy->GetNodeInstance(req, &resp, &controller);
@@ -469,7 +470,7 @@ Status SetPermanentUuidForRemotePeer(const shared_ptr<Messenger>& messenger,
       VLOG(1) << "Sleeping " << delay_ms << " ms. before retrying to get uuid from remote peer...";
       SleepFor(MonoDelta::FromMilliseconds(delay_ms));
       LOG(INFO) << "Retrying to get permanent uuid for remote peer: "
-          << remote_peer->ShortDebugString() << " attempt: " << attempt++;
+          << SecureShortDebugString(*remote_peer) << " attempt: " << attempt++;
     } else {
       s = Status::TimedOut(Substitute("Getting permanent uuid from $0 timed out after $1 ms.",
                                       hostport.ToString(),

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/consensus_queue-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue-test.cc b/src/kudu/consensus/consensus_queue-test.cc
index bcf98af..3660ea9 100644
--- a/src/kudu/consensus/consensus_queue-test.cc
+++ b/src/kudu/consensus/consensus_queue-test.cc
@@ -31,6 +31,7 @@
 #include "kudu/fs/fs_manager.h"
 #include "kudu/server/hybrid_clock.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -811,11 +812,11 @@ TEST_F(ConsensusQueueTest, TestTriggerTabletCopyIfTabletNotFound) {
   StartTabletCopyRequestPB tc_req;
   ASSERT_OK(queue_->GetTabletCopyRequestForPeer(kPeerUuid, &tc_req));
 
-  ASSERT_TRUE(tc_req.IsInitialized()) << tc_req.ShortDebugString();
+  ASSERT_TRUE(tc_req.IsInitialized()) << SecureShortDebugString(tc_req);
   ASSERT_EQ(kTestTablet, tc_req.tablet_id());
   ASSERT_EQ(kLeaderUuid, tc_req.copy_peer_uuid());
-  ASSERT_EQ(FakeRaftPeerPB(kLeaderUuid).last_known_addr().ShortDebugString(),
-            tc_req.copy_peer_addr().ShortDebugString());
+  ASSERT_EQ(SecureShortDebugString(FakeRaftPeerPB(kLeaderUuid).last_known_addr()),
+            SecureShortDebugString(tc_req.copy_peer_addr()));
 }
 
 TEST_F(ConsensusQueueTest, TestFollowerCommittedIndexAndMetrics) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/consensus_queue.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue.cc b/src/kudu/consensus/consensus_queue.cc
index cdbbff0..bf746f3 100644
--- a/src/kudu/consensus/consensus_queue.cc
+++ b/src/kudu/consensus/consensus_queue.cc
@@ -46,6 +46,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/threadpool.h"
 #include "kudu/util/url-coding.h"
 
@@ -149,8 +150,8 @@ void PeerMessageQueue::SetLeaderMode(int64_t committed_index,
   queue_state_.majority_replicated_index = committed_index;
   queue_state_.active_config.reset(new RaftConfigPB(active_config));
   CHECK(IsRaftConfigVoter(local_peer_pb_.permanent_uuid(), *queue_state_.active_config))
-      << local_peer_pb_.ShortDebugString() << " not a voter in config: "
-      << queue_state_.active_config->ShortDebugString();
+      << SecureShortDebugString(local_peer_pb_) << " not a voter in config: "
+      << SecureShortDebugString(*queue_state_.active_config);
   queue_state_.majority_size_ = MajoritySize(CountVoters(*queue_state_.active_config));
   queue_state_.mode = LEADER;
 
@@ -462,11 +463,11 @@ Status PeerMessageQueue::RequestForPeer(const string& uuid,
     if (request->ops_size() > 0) {
       VLOG_WITH_PREFIX_UNLOCKED(2) << "Sending request with operations to Peer: " << uuid
           << ". Size: " << request->ops_size()
-          << ". From: " << request->ops(0).id().ShortDebugString() << ". To: "
-          << request->ops(request->ops_size() - 1).id().ShortDebugString();
+          << ". From: " << SecureShortDebugString(request->ops(0).id()) << ". To: "
+          << SecureShortDebugString(request->ops(request->ops_size() - 1).id());
     } else {
       VLOG_WITH_PREFIX_UNLOCKED(2) << "Sending status only request to Peer: " << uuid
-          << ": " << request->DebugString();
+          << ": " << SecureDebugString(*request);
     }
   }
 
@@ -592,7 +593,7 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
                                         const ConsensusResponsePB& response,
                                         bool* more_pending) {
   DCHECK(response.IsInitialized()) << "Error: Uninitialized: "
-      << response.InitializationErrorString() << ". Response: " << response.ShortDebugString();
+      << response.InitializationErrorString() << ". Response: " << SecureShortDebugString(response);
 
   boost::optional<int64_t> updated_commit_index;
   Mode mode_copy;
@@ -603,7 +604,7 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
     TrackedPeer* peer = FindPtrOrNull(peers_map_, peer_uuid);
     if (PREDICT_FALSE(queue_state_.state != kQueueOpen || peer == nullptr)) {
       LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Queue is closed or peer was untracked, disregarding "
-          "peer response. Response: " << response.ShortDebugString();
+          "peer response. Response: " << SecureShortDebugString(response);
       *more_pending = false;
       return;
     }
@@ -612,7 +613,7 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
     if (response.has_error()) {
       // We only let special types of errors through to this point from the peer.
       CHECK_EQ(tserver::TabletServerErrorPB::TABLET_NOT_FOUND, response.error().code())
-          << response.ShortDebugString();
+          << SecureShortDebugString(response);
 
       peer->needs_tablet_copy = true;
       VLOG_WITH_PREFIX_UNLOCKED(1) << "Marked peer as needing tablet copy: "
@@ -623,8 +624,9 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
 
     // Sanity checks.
     // Some of these can be eventually removed, but they are handy for now.
-    DCHECK(response.status().IsInitialized()) << "Error: Uninitialized: "
-        << response.InitializationErrorString() << ". Response: " << response.ShortDebugString();
+    DCHECK(response.status().IsInitialized())
+        << "Error: Uninitialized: " << response.InitializationErrorString()
+        << ". Response: "<< SecureShortDebugString(response);
     // TODO: Include uuid in error messages as well.
     DCHECK(response.has_responder_uuid() && !response.responder_uuid().empty())
         << "Got response from peer with empty UUID";
@@ -708,7 +710,7 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
         default: {
           LOG_WITH_PREFIX_UNLOCKED(FATAL) << "Unexpected consensus error. Code: "
               << ConsensusErrorPB::Code_Name(status.error().code()) << ". Response: "
-              << response.ShortDebugString();
+              << SecureShortDebugString(response);
         }
       }
     }
@@ -727,7 +729,7 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
 
     if (PREDICT_FALSE(VLOG_IS_ON(2))) {
       VLOG_WITH_PREFIX_UNLOCKED(2) << "Received Response from Peer (" << peer->ToString() << "). "
-          << "Response: " << response.ShortDebugString();
+          << "Response: " << SecureShortDebugString(response);
     }
 
     mode_copy = queue_state_.mode;
@@ -1014,7 +1016,7 @@ string PeerMessageQueue::QueueState::ToString() const {
       all_replicated_index, majority_replicated_index,
       committed_index, OpIdToString(last_appended), current_term,
       majority_size_, state, (mode == LEADER ? "LEADER" : "NON_LEADER"),
-      active_config ? ", active raft config: " + active_config->ShortDebugString() : "");
+      active_config ? ", active raft config: " + SecureShortDebugString(*active_config) : "");
 }
 
 }  // namespace consensus

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/leader_election.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/leader_election.cc b/src/kudu/consensus/leader_election.cc
index 49eb3c8..fad0eb1 100644
--- a/src/kudu/consensus/leader_election.cc
+++ b/src/kudu/consensus/leader_election.cc
@@ -34,6 +34,7 @@
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -171,7 +172,7 @@ LeaderElection::LeaderElection(const RaftConfigPB& config,
            vote_counter_->GetTotalExpectedVotes())
       << "Expected different number of followers. Follower UUIDs: ["
       << JoinStringsIterator(follower_uuids_.begin(), follower_uuids_.end(), ", ")
-      << "]; RaftConfig: {" << config.ShortDebugString() << "}";
+      << "]; RaftConfig: {" << SecureShortDebugString(config) << "}";
 }
 
 LeaderElection::~LeaderElection() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/log-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log-test-base.h b/src/kudu/consensus/log-test-base.h
index dac1935..44a7a90 100644
--- a/src/kudu/consensus/log-test-base.h
+++ b/src/kudu/consensus/log-test-base.h
@@ -43,9 +43,10 @@
 #include "kudu/util/env_util.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/stopwatch.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
-#include "kudu/util/stopwatch.h"
 
 METRIC_DECLARE_entity(tablet);
 
@@ -208,7 +209,7 @@ class LogTestBase : public KuduTest {
 
   void EntriesToIdList(vector<uint32_t>* ids) {
     for (const LogEntryPB* entry : entries_) {
-      VLOG(2) << "Entry contents: " << entry->DebugString();
+      VLOG(2) << "Entry contents: " << SecureDebugString(*entry);
       if (entry->type() == REPLICATE) {
         ids->push_back(entry->replicate().id().index());
       }
@@ -364,9 +365,10 @@ class LogTestBase : public KuduTest {
       strings::SubstituteAndAppend(&dump, "Segment: $0, Path: $1\n",
                                    segment->header().sequence_number(), segment->path());
       strings::SubstituteAndAppend(&dump, "Header: $0\n",
-                                   segment->header().ShortDebugString());
+                                   SecureShortDebugString(segment->header()));
       if (segment->HasFooter()) {
-        strings::SubstituteAndAppend(&dump, "Footer: $0\n", segment->footer().ShortDebugString());
+        strings::SubstituteAndAppend(&dump, "Footer: $0\n",
+                                     SecureShortDebugString(segment->footer()));
       } else {
         dump.append("Footer: None or corrupt.");
       }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/log.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc
index 3fee204..9c4699a 100644
--- a/src/kudu/consensus/log.cc
+++ b/src/kudu/consensus/log.cc
@@ -377,7 +377,7 @@ Status Log::CloseCurrentSegment() {
         "Segment: " << active_segment_->path();
   }
   VLOG(2) << "Segment footer for " << active_segment_->path()
-          << ": " << footer_builder_.ShortDebugString();
+          << ": " << SecureShortDebugString(footer_builder_);
 
   footer_builder_.set_close_timestamp_micros(GetCurrentTimeMicros());
   RETURN_NOT_OK(active_segment_->WriteFooterAndClose(footer_builder_));
@@ -416,7 +416,7 @@ Status Log::Reserve(LogEntryTypePB type,
   // In non-debug builds the foreach loop gets optimized out.
   #ifndef NDEBUG
   for (const LogEntryPB& entry : entry_batch->entry()) {
-    DCHECK_EQ(entry.type(), type) << "Bad batch: " << entry_batch->DebugString();
+    DCHECK_EQ(entry.type(), type) << "Bad batch: " << SecureDebugString(*entry_batch);
   }
   #endif
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/log_cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_cache.cc b/src/kudu/consensus/log_cache.cc
index 8ac6578..a7088c9 100644
--- a/src/kudu/consensus/log_cache.cc
+++ b/src/kudu/consensus/log_cache.cc
@@ -39,6 +39,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 
 DEFINE_int32(log_cache_size_limit_mb, 128,
              "The total per-tablet size of consensus entries which may be kept in memory. "
@@ -474,7 +475,7 @@ void LogCache::DumpToHtml(std::ostream& out) const {
                       "<td>$4</td><td>$5</td></tr>",
                       counter++, msg->id().term(), msg->id().index(),
                       OperationType_Name(msg->op_type()),
-                      msg->ByteSize(), msg->id().ShortDebugString()) << endl;
+                      msg->ByteSize(), SecureShortDebugString(msg->id())) << endl;
   }
   out << "</table>";
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/log_reader.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_reader.cc b/src/kudu/consensus/log_reader.cc
index 14b63c7..2aeef4d 100644
--- a/src/kudu/consensus/log_reader.cc
+++ b/src/kudu/consensus/log_reader.cc
@@ -176,7 +176,7 @@ Status LogReader::Init(const string& tablet_wal_path) {
     string previous_seg_path;
     int64_t previous_seg_seqno = -1;
     for (const SegmentSequence::value_type& entry : read_segments) {
-      VLOG(1) << " Log Reader Indexed: " << entry->footer().ShortDebugString();
+      VLOG(1) << " Log Reader Indexed: " << SecureShortDebugString(entry->footer());
       // Check that the log segments are in sequence.
       if (previous_seg_seqno != -1 && entry->header().sequence_number() != previous_seg_seqno + 1) {
         return Status::Corruption(Substitute("Segment sequence numbers are not consecutive. "
@@ -305,7 +305,7 @@ Status LogReader::ReadReplicatesInRange(int64_t starting_at,
         CHECK_GT(this_index, prev_index)
           << "Expected that an entry batch should only include increasing log indexes: "
           << index_entry.ToString()
-          << "\nBatch: " << batch->DebugString();
+          << "\nBatch: " << SecureDebugString(*batch);
         prev_index = this_index;
       }
     }
@@ -446,7 +446,7 @@ string LogReader::ToString() const {
   for (const SegmentSequence::value_type& entry : segments_) {
     ret.append(Substitute("Segment: $0 Footer: $1\n",
                           entry->header().sequence_number(),
-                          !entry->HasFooter() ? "NONE" : entry->footer().ShortDebugString()));
+                          !entry->HasFooter() ? "NONE" : SecureShortDebugString(entry->footer())));
   }
   return ret;
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/peer_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/peer_manager.cc b/src/kudu/consensus/peer_manager.cc
index 6d39ff5..9c8bcd6 100644
--- a/src/kudu/consensus/peer_manager.cc
+++ b/src/kudu/consensus/peer_manager.cc
@@ -24,6 +24,7 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/threadpool.h"
 
 namespace kudu {
@@ -53,7 +54,7 @@ PeerManager::~PeerManager() {
 Status PeerManager::UpdateRaftConfig(const RaftConfigPB& config) {
   unordered_set<string> new_peers;
 
-  VLOG(1) << "Updating peers from new config: " << config.ShortDebugString();
+  VLOG(1) << "Updating peers from new config: " << SecureShortDebugString(config);
 
   std::lock_guard<simple_spinlock> lock(lock_);
   // Create new peers
@@ -67,7 +68,7 @@ Status PeerManager::UpdateRaftConfig(const RaftConfigPB& config) {
       continue;
     }
 
-    VLOG(1) << GetLogPrefix() << "Adding remote peer. Peer: " << peer_pb.ShortDebugString();
+    VLOG(1) << GetLogPrefix() << "Adding remote peer. Peer: " << SecureShortDebugString(peer_pb);
     gscoped_ptr<PeerProxy> peer_proxy;
     RETURN_NOT_OK_PREPEND(peer_proxy_factory_->NewProxy(peer_pb, &peer_proxy),
                           "Could not obtain a remote proxy to the peer.");
@@ -94,7 +95,7 @@ void PeerManager::SignalRequest(bool force_if_queue_empty) {
     if (PREDICT_FALSE(!s.ok())) {
       LOG(WARNING) << GetLogPrefix()
                    << "Peer was closed, removing from peers. Peer: "
-                   << (*iter).second->peer_pb().ShortDebugString();
+                   << SecureShortDebugString((*iter).second->peer_pb());
       peers_.erase(iter);
     }
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/quorum_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util.cc b/src/kudu/consensus/quorum_util.cc
index 41572d5..17cc8eb 100644
--- a/src/kudu/consensus/quorum_util.cc
+++ b/src/kudu/consensus/quorum_util.cc
@@ -25,6 +25,7 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -130,43 +131,43 @@ Status VerifyRaftConfig(const RaftConfigPB& config, RaftConfigState type) {
   if (config.peers_size() == 0) {
     return Status::IllegalState(
         Substitute("RaftConfig must have at least one peer. RaftConfig: $0",
-                   config.ShortDebugString()));
+                   SecureShortDebugString(config)));
   }
 
   // All configurations must have 'opid_index' populated.
   if (!config.has_opid_index()) {
     return Status::IllegalState(
         Substitute("Configs must have opid_index set. RaftConfig: $0",
-                   config.ShortDebugString()));
+                   SecureShortDebugString(config)));
   }
 
   for (const RaftPeerPB& peer : config.peers()) {
     if (!peer.has_permanent_uuid() || peer.permanent_uuid() == "") {
       return Status::IllegalState(Substitute("One peer didn't have an uuid or had the empty"
-          " string. RaftConfig: $0", config.ShortDebugString()));
+          " string. RaftConfig: $0", SecureShortDebugString(config)));
     }
     if (ContainsKey(uuids, peer.permanent_uuid())) {
       return Status::IllegalState(
           Substitute("Found multiple peers with uuid: $0. RaftConfig: $1",
-                     peer.permanent_uuid(), config.ShortDebugString()));
+                     peer.permanent_uuid(), SecureShortDebugString(config)));
     }
     uuids.insert(peer.permanent_uuid());
 
     if (config.peers_size() > 1 && !peer.has_last_known_addr()) {
       return Status::IllegalState(
           Substitute("Peer: $0 has no address. RaftConfig: $1",
-                     peer.permanent_uuid(), config.ShortDebugString()));
+                     peer.permanent_uuid(), SecureShortDebugString(config)));
     }
     if (!peer.has_member_type()) {
       return Status::IllegalState(
           Substitute("Peer: $0 has no member type set. RaftConfig: $1", peer.permanent_uuid(),
-                     config.ShortDebugString()));
+                     SecureShortDebugString(config)));
     }
     if (peer.member_type() == RaftPeerPB::NON_VOTER) {
       return Status::IllegalState(
           Substitute(
               "Peer: $0 is a NON_VOTER, but this isn't supported yet. RaftConfig: $1",
-              peer.permanent_uuid(), config.ShortDebugString()));
+              peer.permanent_uuid(), SecureShortDebugString(config)));
     }
   }
 
@@ -175,10 +176,11 @@ Status VerifyRaftConfig(const RaftConfigPB& config, RaftConfigState type) {
 
 Status VerifyConsensusState(const ConsensusStatePB& cstate, RaftConfigState type) {
   if (!cstate.has_current_term()) {
-    return Status::IllegalState("ConsensusStatePB missing current_term", cstate.ShortDebugString());
+    return Status::IllegalState("ConsensusStatePB missing current_term",
+                                SecureShortDebugString(cstate));
   }
   if (!cstate.has_config()) {
-    return Status::IllegalState("ConsensusStatePB missing config", cstate.ShortDebugString());
+    return Status::IllegalState("ConsensusStatePB missing config", SecureShortDebugString(cstate));
   }
   RETURN_NOT_OK(VerifyRaftConfig(cstate.config(), type));
 
@@ -186,7 +188,7 @@ Status VerifyConsensusState(const ConsensusStatePB& cstate, RaftConfigState type
     if (!IsRaftConfigVoter(cstate.leader_uuid(), cstate.config())) {
       return Status::IllegalState(
           Substitute("Leader with UUID $0 is not a VOTER in the config! Consensus state: $1",
-                     cstate.leader_uuid(), cstate.ShortDebugString()));
+                     cstate.leader_uuid(), SecureShortDebugString(cstate)));
     }
   }
 
@@ -289,12 +291,12 @@ string DiffConsensusStates(const ConsensusStatePB& old_state,
   // it's still useful to report some change unless the protobufs are identical.
   // So, we fall back to just dumping the before/after debug strings.
   if (change_strs.empty()) {
-    if (old_state.ShortDebugString() == new_state.ShortDebugString()) {
+    if (SecureShortDebugString(old_state) == SecureShortDebugString(new_state)) {
       return "no change";
     }
     return Substitute("change from {$0} to {$1}",
-                      old_state.ShortDebugString(),
-                      new_state.ShortDebugString());
+                      SecureShortDebugString(old_state),
+                      SecureShortDebugString(new_state));
   }
 
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index 0016ea3..d4bca32 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -42,6 +42,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/random.h"
 #include "kudu/util/random_util.h"
 #include "kudu/util/threadpool.h"
@@ -287,7 +288,7 @@ Status RaftConsensus::Start(const ConsensusBootstrapInfo& info) {
     LOG_WITH_PREFIX_UNLOCKED(INFO) << "Replica starting. Triggering "
                                    << info.orphaned_replicates.size()
                                    << " pending transactions. Active config: "
-                                   << state_->GetActiveConfigUnlocked().ShortDebugString();
+                                   << SecureShortDebugString(state_->GetActiveConfigUnlocked());
     for (ReplicateMsg* replicate : info.orphaned_replicates) {
       ReplicateRefPtr replicate_ptr = make_scoped_refptr_replicate(new ReplicateMsg(*replicate));
       RETURN_NOT_OK(StartReplicaTransactionUnlocked(replicate_ptr));
@@ -405,7 +406,7 @@ Status RaftConsensus::StartElection(ElectionMode mode, ElectionReason reason) {
       SnoozeFailureDetectorUnlocked(); // Avoid excessive election noise while in this state.
       return Status::IllegalState("Not starting election: Node is currently "
                                   "a non-participant in the raft config",
-                                  state_->GetActiveConfigUnlocked().ShortDebugString());
+                                  SecureShortDebugString(state_->GetActiveConfigUnlocked()));
     }
     LOG_WITH_PREFIX_UNLOCKED(INFO)
         << "Starting " << mode_str
@@ -432,7 +433,7 @@ Status RaftConsensus::StartElection(ElectionMode mode, ElectionReason reason) {
 
     const RaftConfigPB& active_config = state_->GetActiveConfigUnlocked();
     LOG_WITH_PREFIX_UNLOCKED(INFO) << "Starting " << mode_str << " with config: "
-                                   << active_config.ShortDebugString();
+                                   << SecureShortDebugString(active_config);
 
     // Initialize the VoteCounter.
     int num_voters = CountVoters(active_config);
@@ -614,7 +615,7 @@ Status RaftConsensus::AddPendingOperationUnlocked(const scoped_refptr<ConsensusR
 
     Status s = state_->CheckNoConfigChangePendingUnlocked();
     if (PREDICT_FALSE(!s.ok())) {
-      s = s.CloneAndAppend(Substitute("\n  New config: $0", new_config.ShortDebugString()));
+      s = s.CloneAndAppend(Substitute("\n  New config: $0", SecureShortDebugString(new_config)));
       LOG_WITH_PREFIX_UNLOCKED(INFO) << s.ToString();
       return s;
     }
@@ -632,8 +633,8 @@ Status RaftConsensus::AddPendingOperationUnlocked(const scoped_refptr<ConsensusR
           << "Ignoring setting pending config change with OpId "
           << round->replicate_msg()->id() << " because the committed config has OpId index "
           << committed_config.opid_index() << ". The config change we are ignoring is: "
-          << "Old config: { " << change_record->old_config().ShortDebugString() << " }. "
-          << "New config: { " << new_config.ShortDebugString() << " }";
+          << "Old config: { " << SecureShortDebugString(change_record->old_config()) << " }. "
+          << "New config: { " << SecureShortDebugString(new_config) << " }";
     }
   }
 
@@ -741,7 +742,7 @@ Status RaftConsensus::Update(const ConsensusRequestPB* request,
 
   response->set_responder_uuid(state_->GetPeerUuid());
 
-  VLOG_WITH_PREFIX(2) << "Replica received request: " << request->ShortDebugString();
+  VLOG_WITH_PREFIX(2) << "Replica received request: " << SecureShortDebugString(*request);
 
   // see var declaration
   std::lock_guard<simple_spinlock> lock(update_lock_);
@@ -749,7 +750,8 @@ Status RaftConsensus::Update(const ConsensusRequestPB* request,
   if (PREDICT_FALSE(VLOG_IS_ON(1))) {
     if (request->ops_size() == 0) {
       VLOG_WITH_PREFIX(1) << "Replica replied to status only request. Replica: "
-                          << state_->ToString() << ". Response: " << response->ShortDebugString();
+                          << state_->ToString() << ". Response: "
+                          << SecureShortDebugString(*response);
     }
   }
   return s;
@@ -770,7 +772,8 @@ Status RaftConsensus::StartReplicaTransactionUnlocked(const ReplicateRefPtr& msg
                                 "is set to true.");
   }
 
-  VLOG_WITH_PREFIX_UNLOCKED(1) << "Starting transaction: " << msg->get()->id().ShortDebugString();
+  VLOG_WITH_PREFIX_UNLOCKED(1) << "Starting transaction: "
+                               << SecureShortDebugString(msg->get()->id());
   scoped_refptr<ConsensusRound> round(new ConsensusRound(this, msg));
   ConsensusRound* round_ptr = round.get();
   RETURN_NOT_OK(txn_factory_->StartReplicaTransaction(round));
@@ -903,8 +906,8 @@ Status RaftConsensus::EnforceLogMatchingPropertyMatchesUnlocked(const LeaderRequ
   string error_msg = Substitute(
     "Log matching property violated."
     " Preceding OpId in replica: $0. Preceding OpId from leader: $1. ($2 mismatch)",
-    queue_->GetLastOpIdInLog().ShortDebugString(),
-    req.preceding_opid->ShortDebugString(),
+    SecureShortDebugString(queue_->GetLastOpIdInLog()),
+    SecureShortDebugString(*req.preceding_opid),
     term_mismatch ? "term" : "index");
 
 
@@ -964,7 +967,7 @@ Status RaftConsensus::CheckLeaderRequestUnlocked(const ConsensusRequestPB* reque
     s = PendingRounds::CheckOpInSequence(*prev, message->get()->id());
     if (PREDICT_FALSE(!s.ok())) {
       LOG(ERROR) << "Leader request contained out-of-sequence messages. Status: "
-          << s.ToString() << ". Leader Request: " << request->ShortDebugString();
+          << s.ToString() << ". Leader Request: " << SecureShortDebugString(*request);
       break;
     }
     prev = &message->get()->id();
@@ -1318,7 +1321,7 @@ Status RaftConsensus::UpdateReplica(const ConsensusRequestPB* request,
 
   if (PREDICT_FALSE(VLOG_IS_ON(2))) {
     VLOG_WITH_PREFIX(2) << "Replica updated."
-        << state_->ToString() << " Request: " << request->ShortDebugString();
+        << state_->ToString() << " Request: " << SecureShortDebugString(*request);
   }
 
   TRACE("UpdateReplicas() finished");
@@ -1457,12 +1460,12 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req,
                                    boost::optional<TabletServerErrorPB::Code>* error_code) {
   if (PREDICT_FALSE(!req.has_type())) {
     return Status::InvalidArgument("Must specify 'type' argument to ChangeConfig()",
-                                   req.ShortDebugString());
+                                   SecureShortDebugString(req));
   }
   if (PREDICT_FALSE(!req.has_server())) {
     *error_code = TabletServerErrorPB::INVALID_CONFIG;
     return Status::InvalidArgument("Must specify 'server' argument to ChangeConfig()",
-                                   req.ShortDebugString());
+                                   SecureShortDebugString(req));
   }
   ChangeConfigType type = req.type();
   const RaftPeerPB& server = req.server();
@@ -1481,7 +1484,7 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req,
 
     if (!server.has_permanent_uuid()) {
       return Status::InvalidArgument("server must have permanent_uuid specified",
-                                     req.ShortDebugString());
+                                     SecureShortDebugString(req));
     }
     const RaftConfigPB& committed_config = state_->GetCommittedConfigUnlocked();
 
@@ -1506,15 +1509,15 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req,
         if (IsRaftConfigMember(server_uuid, committed_config)) {
           return Status::InvalidArgument(
               Substitute("Server with UUID $0 is already a member of the config. RaftConfig: $1",
-                        server_uuid, committed_config.ShortDebugString()));
+                        server_uuid, SecureShortDebugString(committed_config)));
         }
         if (!server.has_member_type()) {
           return Status::InvalidArgument("server must have member_type specified",
-                                         req.ShortDebugString());
+                                         SecureShortDebugString(req));
         }
         if (!server.has_last_known_addr()) {
           return Status::InvalidArgument("server must have last_known_addr specified",
-                                         req.ShortDebugString());
+                                         SecureShortDebugString(req));
         }
         *new_config.add_peers() = server;
         break;
@@ -1526,13 +1529,13 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req,
                          "Force another leader to be elected to remove this server. "
                          "Active consensus state: $1",
                          server_uuid,
-                         state_->ConsensusStateUnlocked(CONSENSUS_CONFIG_ACTIVE)
-                            .ShortDebugString()));
+                         SecureShortDebugString(state_->ConsensusStateUnlocked(
+                             CONSENSUS_CONFIG_ACTIVE))));
         }
         if (!RemoveFromRaftConfig(&new_config, server_uuid)) {
           return Status::NotFound(
               Substitute("Server with UUID $0 not a member of the config. RaftConfig: $1",
-                        server_uuid, committed_config.ShortDebugString()));
+                        server_uuid, SecureShortDebugString(committed_config)));
         }
         break;
 
@@ -1599,9 +1602,9 @@ Status RaftConsensus::StartConsensusOnlyRoundUnlocked(const ReplicateRefPtr& msg
   OperationType op_type = msg->get()->op_type();
   CHECK(IsConsensusOnlyOperation(op_type))
       << "Expected a consensus-only op type, got " << OperationType_Name(op_type)
-      << ": " << msg->get()->ShortDebugString();
+      << ": " << SecureShortDebugString(*msg->get());
   VLOG_WITH_PREFIX_UNLOCKED(1) << "Starting consensus round: "
-                               << msg->get()->id().ShortDebugString();
+                               << SecureShortDebugString(msg->get()->id());
   scoped_refptr<ConsensusRound> round(new ConsensusRound(this, msg));
   round->SetConsensusReplicatedCallback(Bind(&RaftConsensus::NonTxRoundReplicationFinished,
                                              Unretained(this),
@@ -1686,8 +1689,8 @@ Status RaftConsensus::RequestVoteRespondLastOpIdTooOld(const OpId& local_last_lo
                           GetRequestVoteLogPrefixUnlocked(*request),
                           request->candidate_uuid(),
                           request->candidate_term(),
-                          local_last_logged_opid.ShortDebugString(),
-                          request->candidate_status().last_received().ShortDebugString());
+                          SecureShortDebugString(local_last_logged_opid),
+                          SecureShortDebugString(request->candidate_status().last_received()));
   LOG(INFO) << msg;
   StatusToPB(Status::InvalidArgument(msg), response->mutable_consensus_error()->mutable_status());
   return Status::OK();
@@ -1941,7 +1944,7 @@ void RaftConsensus::DoElectionCallback(ElectionReason reason, const ElectionResu
                                       << " decision while not in active config. "
                                       << "Result: Term " << election_term << ": "
                                       << (result.decision == VOTE_GRANTED ? "won" : "lost")
-                                      << ". RaftConfig: " << active_config.ShortDebugString();
+                                      << ". RaftConfig: " << SecureShortDebugString(active_config);
     return;
   }
 
@@ -2094,18 +2097,19 @@ void RaftConsensus::CompleteConfigChangeRoundUnlocked(ConsensusRound* round, con
   // messages were delayed.
   const RaftConfigPB& committed_config = state_->GetCommittedConfigUnlocked();
   if (new_config.opid_index() > committed_config.opid_index()) {
-    LOG_WITH_PREFIX_UNLOCKED(INFO) << "Committing config change with OpId "
-                                   << op_id << ": "
-                                   << DiffRaftConfigs(old_config, new_config)
-                                   << ". New config: { " << new_config.ShortDebugString() << " }";
+    LOG_WITH_PREFIX_UNLOCKED(INFO)
+        << "Committing config change with OpId "
+        << op_id << ": "
+        << DiffRaftConfigs(old_config, new_config)
+        << ". New config: { " << SecureShortDebugString(new_config) << " }";
     CHECK_OK(state_->SetCommittedConfigUnlocked(new_config));
   } else {
     LOG_WITH_PREFIX_UNLOCKED(INFO)
         << "Ignoring commit of config change with OpId "
         << op_id << " because the committed config has OpId index "
         << committed_config.opid_index() << ". The config change we are ignoring is: "
-        << "Old config: { " << old_config.ShortDebugString() << " }. "
-        << "New config: { " << new_config.ShortDebugString() << " }";
+        << "Old config: { " << SecureShortDebugString(old_config) << " }. "
+        << "New config: { " << SecureShortDebugString(new_config) << " }";
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/raft_consensus_quorum-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus_quorum-test.cc b/src/kudu/consensus/raft_consensus_quorum-test.cc
index 79a7eb6..6f38cd6 100644
--- a/src/kudu/consensus/raft_consensus_quorum-test.cc
+++ b/src/kudu/consensus/raft_consensus_quorum-test.cc
@@ -42,6 +42,7 @@
 #include "kudu/util/auto_release_pool.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -470,8 +471,8 @@ class RaftConsensusQuorumTest : public KuduTest {
     ExtractReplicateIds(replica_entries, &replica_ids);
     ASSERT_EQ(leader_ids.size(), replica_ids.size());
     for (int i = 0; i < leader_ids.size(); i++) {
-      ASSERT_EQ(leader_ids[i].ShortDebugString(),
-                replica_ids[i].ShortDebugString());
+      ASSERT_EQ(SecureShortDebugString(leader_ids[i]),
+                SecureShortDebugString(replica_ids[i]));
     }
   }
 
@@ -483,10 +484,10 @@ class RaftConsensusQuorumTest : public KuduTest {
     for (const LogEntryPB* entry : entries) {
       if (entry->has_replicate()) {
         ASSERT_TRUE(InsertIfNotPresent(&replication_ops, entry->replicate().id()))
-          << "REPLICATE op id showed up twice: " << entry->ShortDebugString();
+          << "REPLICATE op id showed up twice: " << SecureShortDebugString(*entry);
       } else if (entry->has_commit()) {
         ASSERT_EQ(1, replication_ops.erase(entry->commit().commited_op_id()))
-          << "COMMIT came before associated REPLICATE: " << entry->ShortDebugString();
+          << "COMMIT came before associated REPLICATE: " << SecureShortDebugString(*entry);
       }
     }
   }
@@ -513,7 +514,7 @@ class RaftConsensusQuorumTest : public KuduTest {
     SubstituteAndAppend(&ret, "$1 log entries for replica $0:\n",
                         replica_id, replica_entries.size());
     for (LogEntryPB* replica_entry : replica_entries) {
-      StrAppend(&ret, "Replica log entry: ", replica_entry->ShortDebugString(), "\n");
+      StrAppend(&ret, "Replica log entry: ", SecureShortDebugString(*replica_entry), "\n");
     }
     return ret;
   }
@@ -1147,7 +1148,7 @@ TEST_F(RaftConsensusQuorumTest, TestRequestVote) {
   ASSERT_EQ(last_op_id.term() + 3, res.responder_term());
   ASSERT_TRUE(res.status().has_error());
   ASSERT_EQ(ConsensusErrorPB::INVALID_TERM, res.status().error().code());
-  LOG(INFO) << "Follower rejected old heartbeat, as expected: " << res.ShortDebugString();
+  LOG(INFO) << "Follower rejected old heartbeat, as expected: " << SecureShortDebugString(res);
 }
 
 }  // namespace consensus

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/consensus/raft_consensus_state.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus_state.cc b/src/kudu/consensus/raft_consensus_state.cc
index 2c3e973..a8f70ae 100644
--- a/src/kudu/consensus/raft_consensus_state.cc
+++ b/src/kudu/consensus/raft_consensus_state.cc
@@ -28,6 +28,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/logging.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/trace.h"
 
@@ -88,7 +89,7 @@ Status ReplicaState::LockForRead(UniqueLock* lock) const {
 
 Status ReplicaState::LockForReplicate(UniqueLock* lock, const ReplicateMsg& msg) const {
   ThreadRestrictions::AssertWaitAllowed();
-  DCHECK(!msg.has_id()) << "Should not have an ID yet: " << msg.ShortDebugString();
+  DCHECK(!msg.has_id()) << "Should not have an ID yet: " << SecureShortDebugString(msg);
   UniqueLock l(update_lock_);
   if (PREDICT_FALSE(state_ != kRunning)) {
     return Status::IllegalState("Replica not in running state");
@@ -121,7 +122,7 @@ Status ReplicaState::CheckActiveLeaderUnlocked() const {
                                              "Consensus state: $2",
                                              peer_uuid_,
                                              RaftPeerPB::Role_Name(role),
-                                             cstate.ShortDebugString()));
+                                             SecureShortDebugString(cstate)));
   }
 }
 
@@ -187,8 +188,8 @@ Status ReplicaState::CheckNoConfigChangePendingUnlocked() const {
     return Status::IllegalState(
         Substitute("RaftConfig change currently pending. Only one is allowed at a time.\n"
                    "  Committed config: $0.\n  Pending config: $1",
-                   GetCommittedConfigUnlocked().ShortDebugString(),
-                   GetPendingConfigUnlocked().ShortDebugString()));
+                   SecureShortDebugString(GetCommittedConfigUnlocked()),
+                   SecureShortDebugString(GetPendingConfigUnlocked())));
   }
   return Status::OK();
 }
@@ -199,8 +200,8 @@ Status ReplicaState::SetPendingConfigUnlocked(const RaftConfigPB& new_config) {
                         "Invalid config to set as pending");
   CHECK(!cmeta_->has_pending_config())
       << "Attempt to set pending config while another is already pending! "
-      << "Existing pending config: " << cmeta_->pending_config().ShortDebugString() << "; "
-      << "Attempted new pending config: " << new_config.ShortDebugString();
+      << "Existing pending config: " << SecureShortDebugString(cmeta_->pending_config()) << "; "
+      << "Attempted new pending config: " << SecureShortDebugString(new_config);
   cmeta_->set_pending_config(new_config);
   return Status::OK();
 }
@@ -229,7 +230,8 @@ Status ReplicaState::SetCommittedConfigUnlocked(const RaftConfigPB& committed_co
   CHECK_EQ(GetPendingConfigUnlocked().SerializeAsString(), committed_config.SerializeAsString())
       << Substitute("New committed config must equal pending config, but does not. "
                     "Pending config: $0, committed config: $1",
-                    pending_config.ShortDebugString(), committed_config.ShortDebugString());
+                    SecureShortDebugString(pending_config),
+                    SecureShortDebugString(committed_config));
 
   cmeta_->set_committed_config(committed_config);
   cmeta_->clear_pending_config();
@@ -372,7 +374,7 @@ Status PendingRounds::CancelPendingTransactions() {
     const scoped_refptr<ConsensusRound>& round = txn.second;
     // We cancel only transactions whose applies have not yet been triggered.
     LOG_WITH_PREFIX(INFO) << "Aborting transaction as it isn't in flight: "
-                                   << txn.second->replicate_msg()->ShortDebugString();
+                                   << SecureShortDebugString(*txn.second->replicate_msg());
     round->NotifyReplicationFinished(Status::Aborted("Transaction aborted"));
   }
   return Status::OK();
@@ -509,8 +511,8 @@ Status PendingRounds::SetInitialCommittedOpId(const OpId& committed_op) {
     }
 
     RETURN_NOT_OK(AdvanceCommittedIndex(committed_op.index()));
-    CHECK_EQ(last_committed_op_id_.ShortDebugString(),
-             committed_op.ShortDebugString());
+    CHECK_EQ(SecureShortDebugString(last_committed_op_id_),
+             SecureShortDebugString(committed_op));
 
   } else {
     last_committed_op_id_ = committed_op;

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/fs/fs_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index a8f468a..c3a7e99 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -249,7 +249,7 @@ Status FsManager::Open() {
 
   RETURN_NOT_OK(block_manager_->Open());
   LOG(INFO) << "Opened local filesystem: " << JoinStrings(canonicalized_all_fs_roots_, ",")
-            << std::endl << metadata_->DebugString();
+            << std::endl << SecureDebugString(*metadata_);
   return Status::OK();
 }
 
@@ -360,7 +360,7 @@ Status FsManager::WriteInstanceMetadata(const InstanceMetadataPB& metadata,
                                                 pb_util::NO_OVERWRITE,
                                                 pb_util::SYNC));
   LOG(INFO) << "Generated new instance metadata in path " << path << ":\n"
-            << metadata.DebugString();
+            << SecureDebugString(metadata);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/fs/log_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index 1afa93b..1cc137b 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -598,13 +598,13 @@ Status LogBlockContainer::CheckBlockRecord(const BlockRecordPB& record,
         record.length() < 0) {
       return Status::Corruption(Substitute(
           "Found malformed block record in data file: $0\nRecord: $1\n",
-          data_file_->filename(), record.DebugString()));
+          data_file_->filename(), SecureDebugString(record)));
     }
     if (record.offset() + record.length() > data_file_size) {
       return Status::Corruption(Substitute(
           "Found block extending beyond the end of data file: $0\n"
           "Record: $1\nData file size: $2",
-          data_file_->filename(), record.DebugString(), data_file_size));
+          data_file_->filename(), SecureDebugString(record), data_file_size));
     }
 
     // We could also check that the record's offset is aligned with the
@@ -615,7 +615,7 @@ Status LogBlockContainer::CheckBlockRecord(const BlockRecordPB& record,
       LOG(WARNING) << Substitute(
           "Found misaligned block in data file: $0\nRecord: $1\n"
           "This is likely because of KUDU-1793",
-          data_file_->filename(), record.DebugString());
+          data_file_->filename(), SecureDebugString(record));
     }
   }
   return Status::OK();
@@ -1733,7 +1733,7 @@ Status LogBlockManager::ProcessBlockRecord(const BlockRecordPB& record,
       if (!InsertIfNotPresent(block_map, block_id, lb)) {
         return Status::Corruption(Substitute(
             "found duplicate CREATE record for block $0 in container $1: $2",
-            block_id.ToString(), container->ToString(), record.DebugString()));
+            block_id.ToString(), container->ToString(), SecureDebugString(record)));
       }
 
       VLOG(2) << Substitute("Found CREATE block $0 at offset $1 with length $2",
@@ -1755,14 +1755,14 @@ Status LogBlockManager::ProcessBlockRecord(const BlockRecordPB& record,
       if (block_map->erase(block_id) != 1) {
         return Status::Corruption(Substitute(
             "Found DELETE record for invalid block $0 in container $1: $2",
-            block_id.ToString(), container->ToString(), record.DebugString()));
+            block_id.ToString(), container->ToString(), SecureDebugString(record)));
       }
       VLOG(2) << Substitute("Found DELETE block $0", block_id.ToString());
       break;
     default:
       return Status::Corruption(Substitute(
           "Found unknown op type in container $0: $1",
-          container->ToString(), record.DebugString()));
+          container->ToString(), SecureDebugString(record)));
   }
 
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/integration-tests/cluster_itest_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.cc b/src/kudu/integration-tests/cluster_itest_util.cc
index b700782..df8e0af 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -39,6 +39,7 @@
 #include "kudu/tserver/tserver_service.pb.h"
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/net/net_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_macros.h"
 
 namespace kudu {
@@ -95,7 +96,7 @@ const string& TServerDetails::uuid() const {
 string TServerDetails::ToString() const {
   return Substitute("TabletServer: $0, Rpc address: $1",
                     instance_id.permanent_uuid(),
-                    registration.rpc_addresses(0).ShortDebugString());
+                    SecureShortDebugString(registration.rpc_addresses(0)));
 }
 
 client::KuduSchema SimpleIntKeyKuduSchema() {
@@ -127,7 +128,7 @@ Status GetLastOpIdForEachReplica(const string& tablet_id,
     RETURN_NOT_OK_PREPEND(
       ts->consensus_proxy->GetLastOpId(opid_req, &opid_resp, &controller),
       Substitute("Failed to fetch last op id from $0",
-                 ts->instance_id.ShortDebugString()));
+                 SecureShortDebugString(ts->instance_id)));
     op_ids->push_back(opid_resp.opid());
   }
 
@@ -238,7 +239,7 @@ Status CreateTabletServerMap(MasterServiceProxy* master_proxy,
   RETURN_NOT_OK(master_proxy->ListTabletServers(req, &resp, &controller));
   RETURN_NOT_OK(controller.status());
   if (resp.has_error()) {
-    return Status::RemoteError("Response had an error", resp.error().ShortDebugString());
+    return Status::RemoteError("Response had an error", SecureShortDebugString(resp.error()));
   }
 
   ts_map->clear();
@@ -316,7 +317,7 @@ Status WaitUntilCommittedConfigNumVotersIs(int config_size,
   return Status::TimedOut(Substitute("Number of voters does not equal $0 after waiting for $1."
                                      "Last consensus state: $2. Last status: $3",
                                      config_size, timeout.ToString(),
-                                     cstate.ShortDebugString(), s.ToString()));
+                                     SecureShortDebugString(cstate), s.ToString()));
 }
 
 Status WaitUntilCommittedConfigOpIdIndexIs(int64_t opid_index,
@@ -343,7 +344,7 @@ Status WaitUntilCommittedConfigOpIdIndexIs(int64_t opid_index,
                                      "Last consensus state: $2. Last status: $3",
                                      opid_index,
                                      (MonoTime::Now() - start).ToString(),
-                                     cstate.ShortDebugString(), s.ToString()));
+                                     SecureShortDebugString(cstate), s.ToString()));
 }
 
 Status WaitUntilCommittedOpIdIndexIs(int64_t opid_index,
@@ -615,10 +616,10 @@ Status GetTabletLocations(const shared_ptr<MasterServiceProxy>& master_proxy,
     return StatusFromPB(resp.error().status());
   }
   if (resp.errors_size() > 0) {
-    CHECK_EQ(1, resp.errors_size()) << resp.ShortDebugString();
+    CHECK_EQ(1, resp.errors_size()) << 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();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/integration-tests/create-table-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/create-table-stress-test.cc b/src/kudu/integration-tests/create-table-stress-test.cc
index 357a0e7..97e7955 100644
--- a/src/kudu/integration-tests/create-table-stress-test.cc
+++ b/src/kudu/integration-tests/create-table-stress-test.cc
@@ -33,6 +33,7 @@
 #include "kudu/rpc/messenger.h"
 #include "kudu/tserver/mini_tablet_server.h"
 #include "kudu/tserver/tablet_server.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util.h"
 
@@ -151,7 +152,7 @@ TEST_F(CreateTableStressTest, CreateAndDeleteBigTable) {
   LOG(INFO) << "Created table successfully!";
   // Use std::cout instead of log, since these responses are large and log
   // messages have a max size.
-  std::cout << "Response:\n" << resp.DebugString();
+  std::cout << "Response:\n" << SecureDebugString(resp);
   std::cout << "CatalogManager state:\n";
   cluster_->mini_master()->master()->catalog_manager()->DumpState(&std::cerr);
 
@@ -316,7 +317,7 @@ TEST_F(CreateTableStressTest, TestGetTableLocationsOptions) {
     req.set_max_returned_locations(1);
     req.set_partition_key_start(start_key_middle);
     ASSERT_OK(catalog->GetTableLocations(&req, &resp));
-    ASSERT_EQ(1, resp.tablet_locations_size()) << "Response: [" << resp.DebugString() << "]";
+    ASSERT_EQ(1, resp.tablet_locations_size()) << "Response: [" << SecureDebugString(resp) << "]";
     ASSERT_EQ(start_key_middle, resp.tablet_locations(0).partition().partition_key_start());
   }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/integration-tests/delete_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-test.cc b/src/kudu/integration-tests/delete_table-test.cc
index 5f0f58b..b46f39a 100644
--- a/src/kudu/integration-tests/delete_table-test.cc
+++ b/src/kudu/integration-tests/delete_table-test.cc
@@ -39,6 +39,7 @@
 #include "kudu/tserver/tserver_admin.pb.h"
 #include "kudu/util/curl_util.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/pstack_watcher.h"
 #include "kudu/util/subprocess.h"
 
@@ -269,9 +270,9 @@ TEST_F(DeleteTableTest, TestDeleteEmptyTable) {
     rpc.set_timeout(MonoDelta::FromSeconds(10));
     req.add_tablet_ids()->assign(tablet_id);
     ASSERT_OK(cluster_->master_proxy()->GetTabletLocations(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_EQ(1, resp.errors_size());
-    ASSERT_STR_CONTAINS(resp.errors(0).ShortDebugString(),
+    ASSERT_STR_CONTAINS(SecureShortDebugString(resp.errors(0)),
                         "code: NOT_FOUND message: \"Tablet deleted: Table deleted");
   }
 
@@ -309,7 +310,7 @@ TEST_F(DeleteTableTest, TestDeleteTabletDestUuidValidation) {
   ASSERT_OK(ts->tserver_admin_proxy->DeleteTablet(req, &resp, &rpc));
   ASSERT_TRUE(resp.has_error());
   ASSERT_EQ(tserver::TabletServerErrorPB::WRONG_SERVER_UUID, resp.error().code())
-      << resp.ShortDebugString();
+      << SecureShortDebugString(resp);
   ASSERT_STR_CONTAINS(StatusFromPB(resp.error().status()).ToString(),
                       "Wrong destination UUID");
 }
@@ -788,7 +789,7 @@ TEST_F(DeleteTableTest, TestMergeConsensusMetadata) {
   // The election history should have been wiped out.
   ASSERT_OK(inspect_->ReadConsensusMetadataOnTS(kTsIndex, tablet_id, &cmeta_pb));
   ASSERT_EQ(3, cmeta_pb.current_term());
-  ASSERT_TRUE(!cmeta_pb.has_voted_for()) << cmeta_pb.ShortDebugString();
+  ASSERT_TRUE(!cmeta_pb.has_voted_for()) << SecureShortDebugString(cmeta_pb);
 }
 
 // Regression test for KUDU-987, a bug where followers with transactions in
@@ -907,7 +908,7 @@ TEST_F(DeleteTableTest, TestOrphanedBlocksClearedOnDelete) {
   }
   ASSERT_GT(superblock_pb.rowsets_size(), 0)
       << "Timed out waiting for rowset flush on TS " << follower_ts->uuid() << ": "
-      << "Superblock:\n" << superblock_pb.DebugString();
+      << "Superblock:\n" << SecureDebugString(superblock_pb);
 
   // Shut down the leader so it doesn't try to copy a new replica to our follower later.
   workload.StopAndJoin();
@@ -919,8 +920,8 @@ TEST_F(DeleteTableTest, TestOrphanedBlocksClearedOnDelete) {
                                 boost::none, timeout));
   NO_FATALS(WaitForTabletTombstonedOnTS(kFollowerIndex, tablet_id, CMETA_EXPECTED));
   ASSERT_OK(inspect_->ReadTabletSuperBlockOnTS(kFollowerIndex, tablet_id, &superblock_pb));
-  ASSERT_EQ(0, superblock_pb.rowsets_size()) << superblock_pb.DebugString();
-  ASSERT_EQ(0, superblock_pb.orphaned_blocks_size()) << superblock_pb.DebugString();
+  ASSERT_EQ(0, superblock_pb.rowsets_size()) << SecureDebugString(superblock_pb);
+  ASSERT_EQ(0, superblock_pb.orphaned_blocks_size()) << SecureDebugString(superblock_pb);
 }
 
 vector<const string*> Grep(const string& needle, const vector<string>& haystack) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/integration-tests/exactly_once_writes-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/exactly_once_writes-itest.cc b/src/kudu/integration-tests/exactly_once_writes-itest.cc
index 8841267..b3226f9 100644
--- a/src/kudu/integration-tests/exactly_once_writes-itest.cc
+++ b/src/kudu/integration-tests/exactly_once_writes-itest.cc
@@ -19,6 +19,7 @@
 #include "kudu/integration-tests/ts_itest-base.h"
 #include "kudu/util/barrier.h"
 #include "kudu/util/logging.h"
+#include "kudu/util/pb_util.h"
 
 namespace kudu {
 namespace tserver {
@@ -204,12 +205,12 @@ void ExactlyOnceSemanticsITest::DoTestWritesWithExactlyOnceSemantics(
   bool mismatched = false;
   for (int i = 0; i < num_batches; i++) {
     for (int j = 0; j < num_threads; j++) {
-      string expected_response = responses[j][i].ShortDebugString();
+      string expected_response = SecureShortDebugString(responses[j][i]);
       string expected_ts = strings::Substitute(
           "T:$0 TSidx:$1 TSuuid:$2", j, j % FLAGS_num_replicas,
           cluster_.get()->tablet_server(j % FLAGS_num_replicas)->instance_id().permanent_uuid());
       for (int k = 0; k < num_threads; k++) {
-        string got_response = responses[k][i].ShortDebugString();
+        string got_response = SecureShortDebugString(responses[k][i]);
         string got_ts = strings::Substitute(
             "T:$0 TSidx:$1 TSuuid:$2", k, k % FLAGS_num_replicas,
             cluster_.get()->tablet_server(k % FLAGS_num_replicas)->instance_id().permanent_uuid());

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/integration-tests/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.cc b/src/kudu/integration-tests/external_mini_cluster.cc
index dca5d37..13c86d4 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -453,7 +453,7 @@ Status ExternalMiniCluster::WaitForTabletsRunning(ExternalTabletServer* ts,
     SleepFor(MonoDelta::FromMilliseconds(10));
   }
 
-  return Status::TimedOut(resp.DebugString());
+  return Status::TimedOut(SecureDebugString(resp));
 }
 
 namespace {
@@ -579,7 +579,7 @@ Status ExternalMiniCluster::SetFlag(ExternalDaemon* daemon,
                         "rpc failed");
   if (resp.result() != server::SetFlagResponsePB::SUCCESS) {
     return Status::RemoteError("failed to set flag",
-                               resp.ShortDebugString());
+                               SecureShortDebugString(resp));
   }
   return Status::OK();
 }
@@ -710,7 +710,7 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
   RETURN_NOT_OK_PREPEND(pb_util::ReadPBFromPath(Env::Default(), info_path, status_.get()),
                         "Failed to read info file from " + info_path);
   LOG(INFO) << "Started " << exe_ << " as pid " << p->pid();
-  VLOG(1) << exe_ << " instance information:\n" << status_->DebugString();
+  VLOG(1) << exe_ << " instance information:\n" << SecureDebugString(*status_);
 
   process_.swap(p);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/e7b45d37/src/kudu/integration-tests/master_replication-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_replication-itest.cc b/src/kudu/integration-tests/master_replication-itest.cc
index c30acfe..8a3f38a 100644
--- a/src/kudu/integration-tests/master_replication-itest.cc
+++ b/src/kudu/integration-tests/master_replication-itest.cc
@@ -30,6 +30,7 @@
 #include "kudu/master/mini_master.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/rpc_controller.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/test_util.h"
 
 using std::vector;
@@ -243,7 +244,7 @@ TEST_F(MasterReplicationTest, TestHeartbeatAcceptedByAnyMaster) {
 
     // All masters (including followers) should accept the heartbeat.
     ASSERT_OK(proxy.TSHeartbeat(req, &resp, &rpc));
-    SCOPED_TRACE(resp.DebugString());
+    SCOPED_TRACE(SecureDebugString(resp));
     ASSERT_FALSE(resp.has_error());
   }