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

kudu git commit: KUDU-1812. Redact calls to Slice::ToDebugString

Repository: kudu
Updated Branches:
  refs/heads/master 47f09e30f -> a5a33b3ac


KUDU-1812. Redact calls to Slice::ToDebugString

The Slice::ToDebugString method is used pervasively to log errors in the
case of malformed inputs, which almost always can contain sensitive user
data. Unfortunately Slice::ToDebugString is part of our public client
API, so the redaction can not be done centrally in ToDebugString itself.

Change-Id: I4d1ebde79dea5cae89956b70d13f67fbc6972034
Reviewed-on: http://gerrit.cloudera.org:8080/5568
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: a5a33b3ac8c8729d5dd5b790ba490720fba0b6de
Parents: 47f09e3
Author: Dan Burkert <da...@apache.org>
Authored: Wed Dec 21 23:17:44 2016 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Dec 23 02:00:18 2016 +0000

----------------------------------------------------------------------
 src/kudu/cfile/binary_prefix_block.cc |  4 ++--
 src/kudu/cfile/block_compression.cc   |  8 +++++---
 src/kudu/cfile/bloomfile.cc           |  3 ++-
 src/kudu/cfile/cfile_reader.cc        |  2 +-
 src/kudu/cfile/cfile_writer.cc        |  5 +++--
 src/kudu/cfile/compression_codec.cc   |  3 ++-
 src/kudu/common/encoded_key.cc        | 15 +++++++++------
 src/kudu/common/key_encoder.h         |  9 +++++----
 src/kudu/common/partition-test.cc     | 13 ++++++++++++-
 src/kudu/common/partition.cc          |  6 ++++--
 src/kudu/common/row_changelist.cc     |  8 ++++----
 src/kudu/consensus/log_util.cc        |  5 +++--
 src/kudu/rpc/serialization.cc         | 17 +++++++++--------
 src/kudu/rpc/transfer.cc              |  5 +++++
 src/kudu/tablet/cfile_set.cc          | 10 +++++-----
 src/kudu/tablet/concurrent_btree.h    | 29 +++++++++++++++--------------
 src/kudu/tablet/delta_key.h           |  8 ++++++--
 src/kudu/tablet/diskrowset.cc         | 17 +++++++++--------
 src/kudu/tablet/lock_manager.cc       |  9 +++++----
 src/kudu/tablet/rowset_info.cc        | 17 +++++++++--------
 src/kudu/util/slice.cc                |  3 ++-
 21 files changed, 117 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/cfile/binary_prefix_block.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/binary_prefix_block.cc b/src/kudu/cfile/binary_prefix_block.cc
index faff8d9..cc6eab7 100644
--- a/src/kudu/cfile/binary_prefix_block.cc
+++ b/src/kudu/cfile/binary_prefix_block.cc
@@ -365,8 +365,8 @@ Status BinaryPrefixBlockDecoder::SeekAtOrAfterValue(const void *value_void,
 #ifndef NDEBUG
     VLOG(3) << "loop iter:\n"
             << "cur_idx = " << cur_idx_ << "\n"
-            << "target  =" << target.ToString() << "\n"
-            << "cur_val_=" << Slice(cur_val_).ToString();
+            << "target  =" << KUDU_REDACT(target.ToDebugString()) << "\n"
+            << "cur_val_=" << KUDU_REDACT(Slice(cur_val_).ToDebugString());
 #endif
     int cmp = Slice(cur_val_).compare(target);
     if (cmp >= 0) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/cfile/block_compression.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/block_compression.cc b/src/kudu/cfile/block_compression.cc
index de43ae9..48b598e 100644
--- a/src/kudu/cfile/block_compression.cc
+++ b/src/kudu/cfile/block_compression.cc
@@ -26,6 +26,7 @@
 #include "kudu/util/coding-inl.h"
 #include "kudu/util/coding.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
 
 DEFINE_int64(max_cfile_block_size, 16 * 1024 * 1024,
              "The maximum size of an uncompressed CFile block when using compression. "
@@ -95,7 +96,7 @@ Status CompressedBlockDecoder::ValidateHeader(const Slice& data, uint32_t *uncom
       StringPrintf("data size %lu is not enough to contains the header. "
         "required %lu, buffer",
         data.size(), CompressedBlockBuilder::kHeaderReservedLength),
-        data.ToDebugString(50));
+        KUDU_REDACT(data.ToDebugString(50)));
   }
 
   // Decode the header
@@ -107,14 +108,15 @@ Status CompressedBlockDecoder::ValidateHeader(const Slice& data, uint32_t *uncom
     return Status::Corruption(
       StringPrintf("compressed size %u does not match remaining length in buffer %lu, buffer",
         compressed_size, data.size() - CompressedBlockBuilder::kHeaderReservedLength),
-        data.ToDebugString(50));
+        KUDU_REDACT(data.ToDebugString(50)));
   }
 
   // Check if uncompressed size seems to be reasonable
   if (*uncompressed_size > FLAGS_max_cfile_block_size) {
     return Status::Corruption(
       StringPrintf("uncompressed size %u overflows the maximum length %lu, buffer",
-        compressed_size, FLAGS_max_cfile_block_size), data.ToDebugString(50));
+                   compressed_size, FLAGS_max_cfile_block_size),
+      KUDU_REDACT(data.ToDebugString(50)));
   }
 
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/cfile/bloomfile.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/bloomfile.cc b/src/kudu/cfile/bloomfile.cc
index f49efab..7d23863 100644
--- a/src/kudu/cfile/bloomfile.cc
+++ b/src/kudu/cfile/bloomfile.cc
@@ -106,7 +106,8 @@ Status BloomFileWriter::AppendKeys(
 }
 
 Status BloomFileWriter::FinishCurrentBloomBlock() {
-  VLOG(1) << "Appending a new bloom block, first_key=" << Slice(first_key_).ToDebugString();
+  VLOG(1) << "Appending a new bloom block, first_key="
+          << KUDU_REDACT(Slice(first_key_).ToDebugString());
 
   // Encode the header.
   BloomBlockHeaderPB hdr;

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/cfile/cfile_reader.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_reader.cc b/src/kudu/cfile/cfile_reader.cc
index 4bbc03b..facf474 100644
--- a/src/kudu/cfile/cfile_reader.cc
+++ b/src/kudu/cfile/cfile_reader.cc
@@ -662,7 +662,7 @@ Status CFileIterator::SeekAtOrAfter(const EncodedKey &key,
     *exact_match = false;
     if (PREDICT_FALSE(!validx_iter_->HasNext())) {
       return Status::NotFound("key after last block in file",
-                              key.encoded_key().ToDebugString());
+                              KUDU_REDACT(key.encoded_key().ToDebugString()));
     }
     RETURN_NOT_OK(validx_iter_->Next());
     RETURN_NOT_OK(ReadCurrentDataBlock(*validx_iter_, b.get()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/cfile/cfile_writer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_writer.cc b/src/kudu/cfile/cfile_writer.cc
index 578d857..4e53df0 100644
--- a/src/kudu/cfile/cfile_writer.cc
+++ b/src/kudu/cfile/cfile_writer.cc
@@ -22,16 +22,17 @@
 #include <utility>
 
 #include "kudu/cfile/block_pointer.h"
+#include "kudu/cfile/cfile_util.h"
 #include "kudu/cfile/index_block.h"
 #include "kudu/cfile/index_btree.h"
 #include "kudu/cfile/type_encodings.h"
-#include "kudu/cfile/cfile_util.h"
 #include "kudu/common/key_encoder.h"
 #include "kudu/gutil/endian.h"
 #include "kudu/util/coding.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/hexdump.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/pb_util.h"
 
 using google::protobuf::RepeatedPtrField;
@@ -268,7 +269,7 @@ string CFileWriter::GetMetaValueOrDie(Slice key) const {
       return entry.second;
     }
   }
-  LOG(FATAL) << "Missing metadata entry: " << key.ToDebugString();
+  LOG(FATAL) << "Missing metadata entry: " << KUDU_REDACT(key.ToDebugString());
 }
 
 void CFileWriter::FlushMetadataToPB(RepeatedPtrField<FileMetadataPairPB> *field) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/cfile/compression_codec.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/compression_codec.cc b/src/kudu/cfile/compression_codec.cc
index 2594f79..e8e8c27 100644
--- a/src/kudu/cfile/compression_codec.cc
+++ b/src/kudu/cfile/compression_codec.cc
@@ -26,6 +26,7 @@
 #include "kudu/cfile/compression_codec.h"
 #include "kudu/gutil/singleton.h"
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/util/logging.h"
 
 namespace kudu {
 namespace cfile {
@@ -172,7 +173,7 @@ class Lz4Codec : public CompressionCodec {
     if (n != compressed.size()) {
       return Status::Corruption(
         StringPrintf("unable to uncompress the buffer. error near %d, buffer", -n),
-          compressed.ToDebugString(100));
+                     KUDU_REDACT(compressed.ToDebugString(100)));
     }
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/common/encoded_key.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/encoded_key.cc b/src/kudu/common/encoded_key.cc
index dafe150..e016bdb 100644
--- a/src/kudu/common/encoded_key.cc
+++ b/src/kudu/common/encoded_key.cc
@@ -21,6 +21,7 @@
 #include "kudu/common/key_encoder.h"
 #include "kudu/common/key_util.h"
 #include "kudu/common/row.h"
+#include "kudu/util/logging.h"
 
 namespace kudu {
 
@@ -180,17 +181,19 @@ string EncodedKey::RangeToString(const EncodedKey* lower, const EncodedKey* uppe
   string ret;
   if (lower && upper) {
     ret.append("encoded key BETWEEN ");
-    ret.append(lower->encoded_key().ToDebugString());
+    ret.append(KUDU_REDACT(lower->encoded_key().ToDebugString()));
     ret.append(" AND ");
-    ret.append(upper->encoded_key().ToDebugString());
+    ret.append(KUDU_REDACT(upper->encoded_key().ToDebugString()));
     return ret;
-  } else if (lower) {
+  }
+  if (lower) {
     ret.append("encoded key >= ");
-    ret.append(lower->encoded_key().ToDebugString());
+    ret.append(KUDU_REDACT(lower->encoded_key().ToDebugString()));
     return ret;
-  } else if (upper) {
+  }
+  if (upper) {
     ret.append("encoded key <= ");
-    ret.append(upper->encoded_key().ToDebugString());
+    ret.append(KUDU_REDACT(upper->encoded_key().ToDebugString()));
   } else {
     LOG(DFATAL) << "Invalid key!";
     ret = "invalid key range";

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/common/key_encoder.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/key_encoder.h b/src/kudu/common/key_encoder.h
index ac8886c..910c003 100644
--- a/src/kudu/common/key_encoder.h
+++ b/src/kudu/common/key_encoder.h
@@ -29,6 +29,7 @@
 #include "kudu/gutil/mathlimits.h"
 #include "kudu/gutil/strings/memutil.h"
 #include "kudu/gutil/type_traits.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/status.h"
 
@@ -92,11 +93,11 @@ struct KeyEncoderTraits<Type,
   }
 
   static Status DecodeKeyPortion(Slice* encoded_key,
-                                 bool is_last,
-                                 Arena* arena,
+                                 bool /*is_last*/,
+                                 Arena* /*arena*/,
                                  uint8_t* cell_ptr) {
     if (PREDICT_FALSE(encoded_key->size() < sizeof(cpp_type))) {
-      return Status::InvalidArgument("key too short", encoded_key->ToDebugString());
+      return Status::InvalidArgument("key too short", KUDU_REDACT(encoded_key->ToDebugString()));
     }
 
     unsigned_cpp_type val;
@@ -205,7 +206,7 @@ struct KeyEncoderTraits<BINARY, Buffer> {
                                                       "\0\0", 2));
     if (PREDICT_FALSE(separator == NULL)) {
       return Status::InvalidArgument("Missing separator after composite key string component",
-                                     encoded_key->ToDebugString());
+                                     KUDU_REDACT(encoded_key->ToDebugString()));
     }
 
     uint8_t* src = encoded_key->mutable_data();

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/common/partition-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition-test.cc b/src/kudu/common/partition-test.cc
index d338a82..30bd138 100644
--- a/src/kudu/common/partition-test.cc
+++ b/src/kudu/common/partition-test.cc
@@ -295,6 +295,18 @@ TEST_F(PartitionTest, TestPartitionKeyEncoding) {
       R"(HASH (a, b): 0, HASH (c): 29, RANGE (a, b, c): (<redacted>, <redacted>, <redacted>))";
     EXPECT_EQ(expected, partition_schema.PartitionKeyDebugString(row));
     EXPECT_EQ(expected, partition_schema.PartitionKeyDebugString(key, schema));
+
+    // Check that row values are redacted from error messages when the
+    // log_redact_user_data flag is set.
+
+    EXPECT_EQ("<hash-decode-error>",
+              partition_schema.PartitionKeyDebugString(string("\0\1\0\1", 4), schema));
+    EXPECT_EQ("HASH (a, b): 0, HASH (c): 0, RANGE (a, b, c): "
+              "<range-key-decode-error: Invalid argument: "
+              "Error decoding partition key range component 'a': key too short: <redacted>>",
+              partition_schema.PartitionKeyDebugString(string("\0\0\0\0"
+                                                              "\0\0\0\0"
+                                                              "a", 9), schema));
   }
 }
 
@@ -792,5 +804,4 @@ TEST_F(PartitionTest, TestIncrementRangePartitionStringBounds) {
     check(test, false);
   }
 }
-
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/common/partition.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition.cc b/src/kudu/common/partition.cc
index 084d897..c600b2a 100644
--- a/src/kudu/common/partition.cc
+++ b/src/kudu/common/partition.cc
@@ -26,7 +26,6 @@
 #include "kudu/common/partial_row.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/gutil/map-util.h"
-#include "kudu/gutil/strings/escaping.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/hash_util.h"
@@ -512,7 +511,7 @@ Status PartitionSchema::DecodeRangeKey(Slice* encoded_key,
   }
   if (!encoded_key->empty()) {
     return Status::InvalidArgument("unable to fully decode range key",
-                                   CHexEscape(encoded_key->ToString()));
+                                   KUDU_REDACT(encoded_key->ToDebugString()));
   }
   return Status::OK();
 }
@@ -705,6 +704,9 @@ string PartitionSchema::RangePartitionDebugString(const KuduPartialRow& lower_bo
 string PartitionSchema::RangePartitionDebugString(Slice lower_bound,
                                                   Slice upper_bound,
                                                   const Schema& schema) const {
+  // Partitions are considered metadata, so don't redact them.
+  ScopedDisableRedaction no_redaction;
+
   Arena arena(1024, 128 * 1024);
   KuduPartialRow lower(&schema);
   KuduPartialRow upper(&schema);

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/common/row_changelist.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_changelist.cc b/src/kudu/common/row_changelist.cc
index 2ca7ac4..82b6cc3 100644
--- a/src/kudu/common/row_changelist.cc
+++ b/src/kudu/common/row_changelist.cc
@@ -78,7 +78,7 @@ string RowChangeList::ToString(const Schema &schema) const {
       if (dec.null) {
         ret.append("NULL");
       } else {
-        ret.append(dec.raw_value.ToDebugString());
+        ret.append(KUDU_REDACT(dec.raw_value.ToDebugString()));
       }
     } else {
       // Known column.
@@ -149,11 +149,11 @@ Status RowChangeListDecoder::Init() {
   if (PREDICT_FALSE(!was_valid || type_ == RowChangeList::kUninitialized)) {
     return Status::Corruption(Substitute("bad type enum value: $0 in $1",
                                          static_cast<int>(remaining_[0]),
-                                         remaining_.ToDebugString()));
+                                         KUDU_REDACT(remaining_.ToDebugString())));
   }
   if (PREDICT_FALSE(is_delete() && remaining_.size() != 1)) {
     return Status::Corruption("DELETE changelist too long",
-                              remaining_.ToDebugString());
+                              KUDU_REDACT(remaining_.ToDebugString()));
   }
 
   remaining_.remove_prefix(1);
@@ -351,7 +351,7 @@ Status RowChangeListDecoder::DecodedUpdate::Validate(const Schema& schema,
   if (PREDICT_FALSE(col.type_info()->size() != this->raw_value.size())) {
     return Status::Corruption(Substitute(
                                   "invalid value $0 for column $1",
-                                  this->raw_value.ToDebugString(), col.ToString()));
+                                  KUDU_REDACT(this->raw_value.ToDebugString()), col.ToString()));
   }
 
   *value = reinterpret_cast<const void*>(this->raw_value.data());

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/consensus/log_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_util.cc b/src/kudu/consensus/log_util.cc
index a7cd07d..881efdb 100644
--- a/src/kudu/consensus/log_util.cc
+++ b/src/kudu/consensus/log_util.cc
@@ -40,6 +40,7 @@
 #include "kudu/util/env_util.h"
 #include "kudu/util/fault_injection.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/scoped_cleanup.h"
 
@@ -468,13 +469,13 @@ Status ReadableLogSegment::ParseHeaderMagicAndHeaderLength(const Slice &data,
       // 12 bytes of NULLs, good enough for us to consider this a file that
       // was never written to (but apparently preallocated).
       LOG(WARNING) << "Log segment file " << path() << " has 12 initial NULL bytes instead of "
-                   << "magic and header length: " << data.ToDebugString()
+                   << "magic and header length: " << KUDU_REDACT(data.ToDebugString())
                    << " and will be treated as a blank segment.";
       return Status::Uninitialized("log magic and header length are all NULL bytes");
     }
     // If no magic and not uninitialized, the file is considered corrupt.
     return Status::Corruption(Substitute("Invalid log segment file $0: Bad magic. $1",
-                                         path(), data.ToDebugString()));
+                                         path(), KUDU_REDACT(data.ToDebugString())));
   }
 
   *parsed_len = DecodeFixed32(data.data() + strlen(kLogSegmentHeaderMagicString));

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/rpc/serialization.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/serialization.cc b/src/kudu/rpc/serialization.cc
index 91f6ec8..dbb0fc5 100644
--- a/src/kudu/rpc/serialization.cc
+++ b/src/kudu/rpc/serialization.cc
@@ -22,10 +22,11 @@
 #include <google/protobuf/io/coded_stream.h>
 
 #include "kudu/gutil/endian.h"
-#include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/constants.h"
 #include "kudu/util/faststring.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 
@@ -105,12 +106,12 @@ Status ParseMessage(const Slice& buf,
   // First grab the total length
   if (PREDICT_FALSE(buf.size() < kMsgLengthPrefixLength)) {
     return Status::Corruption("Invalid packet: not enough bytes for length header",
-                              buf.ToDebugString());
+                              KUDU_REDACT(buf.ToDebugString()));
   }
 
   int total_len = NetworkByteOrder::Load32(buf.data());
   DCHECK_EQ(total_len + kMsgLengthPrefixLength, buf.size())
-    << "Got mis-sized buffer: " << buf.ToDebugString();
+    << "Got mis-sized buffer: " << KUDU_REDACT(buf.ToDebugString());
 
   CodedInputStream in(buf.data(), buf.size());
   in.Skip(kMsgLengthPrefixLength);
@@ -118,33 +119,33 @@ Status ParseMessage(const Slice& buf,
   uint32_t header_len;
   if (PREDICT_FALSE(!in.ReadVarint32(&header_len))) {
     return Status::Corruption("Invalid packet: missing header delimiter",
-                              buf.ToDebugString());
+                              KUDU_REDACT(buf.ToDebugString()));
   }
 
   CodedInputStream::Limit l;
   l = in.PushLimit(header_len);
   if (PREDICT_FALSE(!parsed_header->ParseFromCodedStream(&in))) {
     return Status::Corruption("Invalid packet: header too short",
-                              buf.ToDebugString());
+                              KUDU_REDACT(buf.ToDebugString()));
   }
   in.PopLimit(l);
 
   uint32_t main_msg_len;
   if (PREDICT_FALSE(!in.ReadVarint32(&main_msg_len))) {
     return Status::Corruption("Invalid packet: missing main msg length",
-                              buf.ToDebugString());
+                              KUDU_REDACT(buf.ToDebugString()));
   }
 
   if (PREDICT_FALSE(!in.Skip(main_msg_len))) {
     return Status::Corruption(
         StringPrintf("Invalid packet: data too short, expected %d byte main_msg", main_msg_len),
-        buf.ToDebugString());
+        KUDU_REDACT(buf.ToDebugString()));
   }
 
   if (PREDICT_FALSE(in.BytesUntilLimit() > 0)) {
     return Status::Corruption(
       StringPrintf("Invalid packet: %d extra bytes at end of packet", in.BytesUntilLimit()),
-      buf.ToDebugString());
+      KUDU_REDACT(buf.ToDebugString()));
   }
 
   *parsed_main_message = Slice(buf.data() + buf.size() - main_msg_len,

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/rpc/transfer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/transfer.cc b/src/kudu/rpc/transfer.cc
index 5521573..d24e94d 100644
--- a/src/kudu/rpc/transfer.cc
+++ b/src/kudu/rpc/transfer.cc
@@ -29,6 +29,7 @@
 #include "kudu/rpc/constants.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
 
@@ -240,6 +241,10 @@ bool OutboundTransfer::TransferFinished() const {
 }
 
 string OutboundTransfer::HexDump() const {
+  if (KUDU_SHOULD_REDACT()) {
+    return kRedactionMessage;
+  }
+
   string ret;
   for (int i = 0; i < n_payload_slices_; i++) {
     ret.append(payload_slices_[i].ToDebugString());

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/tablet/cfile_set.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/cfile_set.cc b/src/kudu/tablet/cfile_set.cc
index f61556b..a98b918 100644
--- a/src/kudu/tablet/cfile_set.cc
+++ b/src/kudu/tablet/cfile_set.cc
@@ -154,9 +154,9 @@ Status CFileSet::LoadMinMaxKeys() {
     return Status::Corruption("No max key found", ToString());
   }
   if (Slice(min_encoded_key_).compare(max_encoded_key_) > 0) {
-    return Status::Corruption(StringPrintf("Min key %s > max key %s",
-                                           Slice(min_encoded_key_).ToDebugString().c_str(),
-                                           Slice(max_encoded_key_).ToDebugString().c_str()),
+    return Status::Corruption(Substitute("Min key $0 > max key $1",
+                                         KUDU_REDACT(Slice(min_encoded_key_).ToDebugString()),
+                                         KUDU_REDACT(Slice(max_encoded_key_).ToDebugString())),
                               ToString());
   }
 
@@ -373,9 +373,9 @@ Status CFileSet::Iterator::PushdownRangeScanPredicate(ScanSpec *spec) {
       LOG(DFATAL) << "CFileSet indicated upper bound was within range, but "
                   << "key iterator could not seek. "
                   << "CFileSet upper_bound = "
-                  << Slice(base_data_->max_encoded_key_).ToDebugString()
+                  << KUDU_REDACT(Slice(base_data_->max_encoded_key_).ToDebugString())
                   << ", enc_key = "
-                  << spec->exclusive_upper_bound_key()->encoded_key().ToDebugString();
+                  << KUDU_REDACT(spec->exclusive_upper_bound_key()->encoded_key().ToDebugString());
     } else {
       RETURN_NOT_OK(s);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/tablet/concurrent_btree.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/concurrent_btree.h b/src/kudu/tablet/concurrent_btree.h
index 795b5a4..b4c5012 100644
--- a/src/kudu/tablet/concurrent_btree.h
+++ b/src/kudu/tablet/concurrent_btree.h
@@ -49,14 +49,15 @@
 #include <memory>
 #include <string>
 
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/mathlimits.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
 #include "kudu/util/debug/sanitizer_scopes.h"
 #include "kudu/util/inline_slice.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/status.h"
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/mathlimits.h"
-#include "kudu/gutil/stringprintf.h"
-#include "kudu/gutil/port.h"
 
 //#define TRAVERSE_PREFETCH
 #define SCAN_PREFETCH
@@ -547,7 +548,7 @@ class PACKED InternalNode : public NodeBase<Traits> {
     bool exact;
     size_t idx = Find(key, &exact);
     CHECK(!exact)
-      << "Trying to insert duplicate key " << key.ToDebugString()
+      << "Trying to insert duplicate key " << KUDU_REDACT(key.ToDebugString())
       << " into an internal node! Internal node keys should result "
       << " from splits and therefore be unique.";
 
@@ -631,7 +632,7 @@ class PACKED InternalNode : public NodeBase<Traits> {
         ret.append(", ");
       }
       Slice k = keys_[i].as_slice();
-      ret.append(k.ToDebugString());
+      ret.append(KUDU_REDACT(k.ToDebugString()));
     }
     ret.append("]");
     return ret;
@@ -786,9 +787,9 @@ class LeafNode : public NodeBase<Traits> {
       Slice k = keys_[i].as_slice();
       Slice v = vals_[i].as_slice();
       ret.append("[");
-      ret.append(k.ToDebugString());
+      ret.append(KUDU_REDACT(k.ToDebugString()));
       ret.append("=");
-      ret.append(v.ToDebugString());
+      ret.append(KUDU_REDACT(v.ToDebugString()));
       ret.append("]");
     }
     return ret;
@@ -1240,7 +1241,7 @@ class CBTree {
           DebugPrint(inode->child_pointers_[i], inode, indent + 4);
           if (i < inode->key_count()) {
             SStringPrintf(&buf, "%*sKEY ", indent + 2, "");
-            buf.append(inode->GetKey(i).ToDebugString());
+            buf.append(KUDU_REDACT(inode->GetKey(i).ToDebugString()));
             LOG(INFO) << buf;
           }
         }
@@ -1488,8 +1489,8 @@ class CBTree {
     dst_leaf->PrepareMutation(mutation);
 
     CHECK_EQ(INSERT_SUCCESS, dst_leaf->Insert(mutation, val))
-      << "node split at " << split_key.ToDebugString()
-      << " did not result in enough space for key " << key.ToDebugString()
+      << "node split at " << KUDU_REDACT(split_key.ToDebugString())
+      << " did not result in enough space for key " << KUDU_REDACT(key.ToDebugString())
       << " in left node";
 
     // Insert the new node into the parents.
@@ -1536,7 +1537,7 @@ class CBTree {
       case INSERT_SUCCESS:
       {
         VLOG(3) << "Inserted new entry into internal node "
-                << parent << " for " << split_key.ToDebugString();
+                << parent << " for " << KUDU_REDACT(split_key.ToDebugString());
         left->Unlock();
         right->Unlock();
         parent->Unlock();
@@ -1557,8 +1558,8 @@ class CBTree {
           (split_key.compare(inode_split) < 0) ? parent : new_inode;
 
         VLOG(2) << "Split internal node " << parent << " for insert of "
-                << split_key.ToDebugString() << "[" << right << "]"
-                << " (split at " << inode_split.ToDebugString() << ")";
+                << KUDU_REDACT(split_key.ToDebugString()) << "[" << right << "]"
+                << " (split at " << KUDU_REDACT(inode_split.ToDebugString()) << ")";
 
         CHECK_EQ(INSERT_SUCCESS, dst_inode->Insert(split_key, right_ptr, arena_.get()));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/tablet/delta_key.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_key.h b/src/kudu/tablet/delta_key.h
index d5338b1..402bb74 100644
--- a/src/kudu/tablet/delta_key.h
+++ b/src/kudu/tablet/delta_key.h
@@ -18,10 +18,12 @@
 #define KUDU_TABLET_DELTA_KEY_H
 
 #include <string>
+
 #include "kudu/common/rowid.h"
 #include "kudu/gutil/endian.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/mvcc.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -73,11 +75,13 @@ class DeltaKey {
   Status DecodeFrom(Slice *key) {
     Slice orig(*key);
     if (!PREDICT_TRUE(DecodeRowId(key, &row_idx_))) {
-      return Status::Corruption("Bad delta key: bad rowid", orig.ToDebugString(20));
+      return Status::Corruption("Bad delta key: bad rowid",
+                                KUDU_REDACT(orig.ToDebugString(20)));
     }
 
     if (!PREDICT_TRUE(timestamp_.DecodeFrom(key))) {
-      return Status::Corruption("Bad delta key: bad timestamp", orig.ToDebugString(20));
+      return Status::Corruption("Bad delta key: bad timestamp",
+                                KUDU_REDACT(orig.ToDebugString(20)));
     }
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/tablet/diskrowset.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/diskrowset.cc b/src/kudu/tablet/diskrowset.cc
index 7cd312a..462ee32 100644
--- a/src/kudu/tablet/diskrowset.cc
+++ b/src/kudu/tablet/diskrowset.cc
@@ -20,24 +20,25 @@
 #include <mutex>
 #include <vector>
 
+#include "kudu/cfile/bloomfile.h"
+#include "kudu/cfile/cfile_writer.h"
+#include "kudu/cfile/type_encodings.h"
 #include "kudu/common/generic_iterators.h"
 #include "kudu/common/iterator.h"
 #include "kudu/common/schema.h"
 #include "kudu/consensus/log_anchor_registry.h"
-#include "kudu/cfile/bloomfile.h"
-#include "kudu/cfile/cfile_writer.h"
-#include "kudu/cfile/type_encodings.h"
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/tablet/cfile_set.h"
 #include "kudu/tablet/compaction.h"
+#include "kudu/tablet/delta_compaction.h"
 #include "kudu/tablet/delta_store.h"
 #include "kudu/tablet/diskrowset.h"
-#include "kudu/tablet/delta_compaction.h"
 #include "kudu/tablet/multi_column_writer.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/status.h"
 
 DEFINE_int32(tablet_delta_store_minor_compact_max, 1000,
@@ -185,8 +186,8 @@ Status DiskRowSetWriter::AppendBlock(const RowBlock &block) {
 
 #ifndef NDEBUG
     CHECK_LT(Slice(prev_key).compare(enc_key), 0)
-      << enc_key.ToDebugString() << " appended to file not > previous key "
-      << Slice(prev_key).ToDebugString();
+      << KUDU_REDACT(enc_key.ToDebugString()) << " appended to file not > previous key "
+      << KUDU_REDACT(Slice(prev_key).ToDebugString());
 #endif
   }
 
@@ -219,8 +220,8 @@ Status DiskRowSetWriter::FinishAndReleaseBlocks(ScopedWritableBlockCloser* close
   Slice first_enc_slice(first_encoded_key);
 
   CHECK_LE(first_enc_slice.compare(last_enc_slice), 0)
-      << "First Key not <= Last key: first_key=" << first_enc_slice.ToDebugString()
-      << "   last_key=" << last_enc_slice.ToDebugString();
+      << "First Key not <= Last key: first_key=" << KUDU_REDACT(first_enc_slice.ToDebugString())
+      << "   last_key=" << KUDU_REDACT(last_enc_slice.ToDebugString());
   key_index_writer()->AddMetadataPair(DiskRowSet::kMaxKeyMetaEntryName, last_enc_slice);
 
   // Finish writing the columns themselves.

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/tablet/lock_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/lock_manager.cc b/src/kudu/tablet/lock_manager.cc
index 2e951ac..3c37429 100644
--- a/src/kudu/tablet/lock_manager.cc
+++ b/src/kudu/tablet/lock_manager.cc
@@ -27,6 +27,7 @@
 #include "kudu/gutil/hash/city.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/semaphore.h"
 #include "kudu/util/trace.h"
 
@@ -56,7 +57,7 @@ class LockEntry {
   }
 
   std::string ToString() const {
-    return key_.ToDebugString();
+    return KUDU_REDACT(key_.ToDebugString());
   }
 
   // Mutex used by the LockManager
@@ -357,15 +358,15 @@ LockManager::LockStatus LockManager::Lock(const Slice& key,
     while (!(*entry)->sem.TimedAcquire(MonoDelta::FromSeconds(1))) {
       const TransactionState* cur_holder = ANNOTATE_UNPROTECTED_READ((*entry)->holder_);
       LOG(WARNING) << "Waited " << (++waited_seconds) << " seconds to obtain row lock on key "
-                   << key.ToDebugString() << " cur holder: " << cur_holder;
-      // TODO: would be nice to also include some info about the blocking transaction,
+                   << KUDU_REDACT(key.ToDebugString()) << " cur holder: " << cur_holder;
+      // TODO(unknown): would be nice to also include some info about the blocking transaction,
       // but it's a bit tricky to do in a non-racy fashion (the other transaction may
       // complete at any point)
     }
     MicrosecondsInt64 wait_us = GetMonoTimeMicros() - start_wait_us;
     TRACE_COUNTER_INCREMENT("row_lock_wait_us", wait_us);
     if (wait_us > 100 * 1000) {
-      TRACE("Waited $0us for lock on $1", wait_us, key.ToDebugString());
+      TRACE("Waited $0us for lock on $1", wait_us, KUDU_REDACT(key.ToDebugString()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/tablet/rowset_info.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/rowset_info.cc b/src/kudu/tablet/rowset_info.cc
index 123266d..f414dea 100644
--- a/src/kudu/tablet/rowset_info.cc
+++ b/src/kudu/tablet/rowset_info.cc
@@ -32,6 +32,7 @@
 #include "kudu/gutil/strings/util.h"
 #include "kudu/tablet/rowset.h"
 #include "kudu/tablet/rowset_tree.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/slice.h"
 
 using std::shared_ptr;
@@ -78,15 +79,15 @@ int CommonPrefix(const Slice& min, const Slice& max) {
 }
 
 void DCheckCommonPrefix(const Slice& min, const Slice& imin,
-                       const Slice& imax, int common_prefix) {
+                        const Slice& imax, int common_prefix) {
   DCHECK_EQ(memcmp(min.data(), imin.data(), common_prefix), 0)
     << "slices should share common prefix:\n"
-    << "\t" << min.ToDebugString() << "\n"
-    << "\t" << imin.ToDebugString();
+    << "\t" << KUDU_REDACT(min.ToDebugString()) << "\n"
+    << "\t" << KUDU_REDACT(imin.ToDebugString());
   DCHECK_EQ(memcmp(min.data(), imax.data(), common_prefix), 0)
     << "slices should share common prefix:\n"
-    << "\t" << min.ToDebugString() << "\n"
-    << "\t" << imin.ToDebugString();
+    << "\t" << KUDU_REDACT(min.ToDebugString()) << "\n"
+    << "\t" << KUDU_REDACT(imin.ToDebugString());
 }
 
 uint64_t SliceTailToInt(const Slice& slice, int start) {
@@ -240,7 +241,7 @@ void RowSetInfo::CollectOrdered(const RowSetTree& tree,
                  << " or RowSetTree::STOP=" << RowSetTree::STOP << ".\n"
                  << "\tRecieved:\n"
                  << "\t\tRowSet=" << rs->ToString() << "\n"
-                 << "\t\tKey=" << next << "\n"
+                 << "\t\tKey=" << KUDU_REDACT(next.ToDebugString()) << "\n"
                  << "\t\tEndpointType=" << rse.endpoint_;
     }
   }
@@ -281,8 +282,8 @@ string RowSetInfo::ToString() const {
   StringAppendF(&ret, "(% 3dM) [%.04f, %.04f]", size_mb_,
                 cdf_min_key_, cdf_max_key_);
   if (has_bounds_) {
-    ret.append(" [").append(Slice(min_key_).ToDebugString());
-    ret.append(",").append(Slice(max_key_).ToDebugString());
+    ret.append(" [").append(KUDU_REDACT(Slice(min_key_).ToDebugString()));
+    ret.append(",").append(KUDU_REDACT(Slice(max_key_).ToDebugString()));
     ret.append("]");
   }
   return ret;

http://git-wip-us.apache.org/repos/asf/kudu/blob/a5a33b3a/src/kudu/util/slice.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/slice.cc b/src/kudu/util/slice.cc
index 716bd4b..7e6ab50 100644
--- a/src/kudu/util/slice.cc
+++ b/src/kudu/util/slice.cc
@@ -19,13 +19,14 @@
 
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/util/status.h"
+#include "kudu/util/logging.h"
 
 namespace kudu {
 
 Status Slice::check_size(size_t expected_size) const {
   if (PREDICT_FALSE(size() != expected_size)) {
     return Status::Corruption(StringPrintf("Unexpected Slice size. "
-        "Expected %zu but got %zu.", expected_size, size()), ToDebugString(100));
+        "Expected %zu but got %zu.", expected_size, size()), KUDU_REDACT(ToDebugString(100)));
   }
   return Status::OK();
 }