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:34 UTC

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

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>


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

Branch: refs/heads/master
Commit: c09751508b1e20638fae4ad474d0a41b12321eab
Parents: f3447a1
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 00:28:39 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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 5102dcd..d2acdb7 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"
 
 // This file uses C++14 'generalized lambda capture' syntax, which is supported
@@ -225,7 +226,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();
 
   request_pending_ = true;
@@ -284,7 +285,7 @@ void Peer::ProcessResponse() {
     });
   if (PREDICT_FALSE(!s.ok())) {
     LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to process peer response: " << s.ToString()
-        << ": " << response_.ShortDebugString();
+        << ": " << SecureShortDebugString(response_);
     request_pending_ = false;
   }
 }
@@ -292,7 +293,7 @@ void Peer::ProcessResponse() {
 void Peer::DoProcessResponse() {
 
   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);
@@ -340,7 +341,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_);
     }
   }
 }
@@ -470,7 +471,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);
@@ -492,7 +493,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/c0975150/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 6d4ef7e..1d5d46b 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"
 
@@ -809,11 +810,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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/src/kudu/consensus/peer_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/peer_manager.cc b/src/kudu/consensus/peer_manager.cc
index 1f28faa..45cf9bf 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 {
@@ -51,7 +52,7 @@ PeerManager::~PeerManager() {
 }
 
 Status PeerManager::UpdateRaftConfig(const RaftConfigPB& config) {
-  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
@@ -63,7 +64,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.");
@@ -89,7 +90,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++);
     } else {
       ++iter;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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/c0975150/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 0ba1a51..1ac83b9 100644
--- a/src/kudu/integration-tests/delete_table-test.cc
+++ b/src/kudu/integration-tests/delete_table-test.cc
@@ -40,6 +40,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"
 
@@ -270,9 +271,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");
   }
 
@@ -310,7 +311,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");
 }
@@ -792,7 +793,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
@@ -911,7 +912,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();
@@ -923,8 +924,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/c0975150/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 6b2acf5..a89a129 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/c0975150/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 8afff37..ec613b9 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();
 }
@@ -714,7 +714,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/c0975150/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());
   }