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

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

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/src/kudu/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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 ef2d8a8..625d341 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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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 585e8bd..22edce7 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"
 
@@ -141,8 +142,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);
@@ -224,7 +225,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;
@@ -260,10 +261,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) {
@@ -295,9 +296,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);
@@ -410,9 +411,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);
@@ -434,9 +435,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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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