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

[3/3] kudu git commit: KUDU-1812. Redact pretty-printed sensitive user data

KUDU-1812. Redact pretty-printed sensitive user data

This commit introduces a framework in logging.h to enable sensitive user
data to be redacted in the Kudu server, and to a lesser extent, in the
C++ client.

- The 'log_redact_user_data' gflag is added which controls whether row
  and predicate data is redacted from log, error and status messages. The
  flag defaults to true in the Kudu server binaries order to provide a
  safe default for Kudu users. The flag is explicitly set to false in
  tests and the 'kudu' CLI tool in order to aid debugging.

- There is a new thread-local boolean which indicates whether ToString
  functions should redact user data in the current thread.  This defaults
  to 'true', but it is always also combined with the
  'log_redact_user_data' flag to determine whether redaction will actually
  happen.

- A utility macro KUDU_DISABLE_REDACTION(...) and RAII equivalent
  ScopedDisableRedaction can disable redaction while evaluating a
  particular expression or scope, useful in contexts such as the web UI or
  tools where we don't want to redact.

- A macro KUDU_REDACT(expr) replaces its argument with '<redacted>' if
  redaction is enabled.

- ToString and equivalent calls have been changed to consult
  log_redact_user_data and the TLS flag to determine whether to perform
  redaction, where appropriate. Some ToString calls specifically disable
  redaction based on the type of data being stringified; this behavior
  has been explicitly called out in the header documentation, but this
  isn't expected to be a source of security issues in the future since
  in all such cases the data is not considered sensitive. A handful of
  ToString methods in our public API disable redaction in order to
  retain the same behavior; these methods should never be used for
  internal Kudu logging without manual redaction, and their doxygen has
  been updated to indicate this.

- Redaction in the C++ client uses the same mechanism as in the server,
  but the 'log_redact_user_data' is permanently set to 'true'. We can
  add a public API option to allow applications to turn off redaction in
  the future if it proves to be necessary.

The advantage of a TLS flag in addition to 'log_redact_user_data' is
that we're now able to apply KUDU_REDACT() at a very low level (the
stringification functions in types.cc which are used for stringifying
all user data). This means that we are by-default redacted anywhere that
stringifies a row, rather than having to look for all cases that may
lead to this stringification. Instead, we only have to find the places
that explicitly want to disable redaction, which should be the exception
rather than the rule.

Redaction of raw buffers and protobuf messages will be included in a
follow up commit.

Change-Id: I2b87a3065280116bb8af6f26f072dafdfd1ee077
Reviewed-on: http://gerrit.cloudera.org:8080/5555
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/1179bbdb
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/1179bbdb
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/1179bbdb

Branch: refs/heads/master
Commit: 1179bbdb43e637e431c9327f46734c7cf7d5c39c
Parents: a4e95c4
Author: Dan Burkert <da...@apache.org>
Authored: Tue Dec 20 17:59:00 2016 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Thu Dec 22 08:24:36 2016 +0000

----------------------------------------------------------------------
 src/kudu/client/batcher.cc                      | 15 +++--
 src/kudu/client/client.cc                       | 26 +++----
 src/kudu/client/client.h                        |  6 +-
 src/kudu/client/meta_cache.h                    |  5 ++
 src/kudu/client/scanner-internal.h              | 11 +++
 src/kudu/client/session-internal.cc             |  7 +-
 src/kudu/client/write_op.h                      |  3 +
 src/kudu/common/column_predicate-test.cc        | 10 +++
 src/kudu/common/partial_row.cc                  |  3 +
 src/kudu/common/partial_row.h                   |  2 +
 src/kudu/common/partition-test.cc               | 46 ++++++++++---
 src/kudu/common/partition.cc                    | 10 ++-
 src/kudu/common/partition.h                     | 43 ++++++++++--
 src/kudu/common/partition_pruner-test.cc        |  8 +--
 src/kudu/common/partition_pruner.h              |  2 +
 src/kudu/common/row_operations.cc               | 16 +++--
 src/kudu/common/row_operations.h                |  1 +
 src/kudu/common/scan_spec-test.cc               |  9 +++
 src/kudu/common/schema-test.cc                  | 27 ++++----
 src/kudu/common/types-test.cc                   | 11 +--
 src/kudu/common/types.cc                        |  7 +-
 .../integration-tests/external_mini_cluster.cc  |  3 +
 src/kudu/server/webserver.cc                    |  5 ++
 .../tablet/transactions/write_transaction.cc    |  2 -
 src/kudu/tools/tool_action_local_replica.cc     |  2 +-
 src/kudu/tools/tool_main.cc                     | 10 +--
 src/kudu/util/flag_tags-test.cc                 |  6 +-
 src/kudu/util/flags-test.cc                     |  6 +-
 src/kudu/util/logging-test.cc                   | 35 ++++++++++
 src/kudu/util/logging.cc                        |  8 +++
 src/kudu/util/logging.h                         | 71 ++++++++++++++++++++
 src/kudu/util/test_util.cc                      |  4 ++
 32 files changed, 340 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/batcher.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/batcher.cc b/src/kudu/client/batcher.cc
index 0c3c720..8b090f4 100644
--- a/src/kudu/client/batcher.cc
+++ b/src/kudu/client/batcher.cc
@@ -175,9 +175,14 @@ struct InFlightOp {
   // order of operations. This is important when multiple operations act on the same row.
   int sequence_number_;
 
+  // Stringifies the InFlightOp.
+  //
+  // This should be used in log messages instead of KuduWriteOperation::ToString
+  // because it handles redaction.
   string ToString() const {
     return strings::Substitute("op[state=$0, write_op=$1]",
-                               state, write_op->ToString());
+                               state,
+                               KUDU_REDACT(write_op->ToString()));
   }
 };
 
@@ -285,7 +290,7 @@ WriteRpc::WriteRpc(const scoped_refptr<Batcher>& batcher,
     // until after we sent it, the RPC callback could fire before we got a chance
     // to change its state to 'sent'.
     op->state = InFlightOp::kRequestSent;
-    VLOG(4) << ++ctr << ". Encoded row " << op->write_op->ToString();
+    VLOG(4) << ++ctr << ". Encoded row " << op->ToString();
   }
 
   if (VLOG_IS_ON(3)) {
@@ -520,7 +525,7 @@ Status Batcher::Add(KuduWriteOperation* write_op) {
   op->state = InFlightOp::kLookingUpTablet;
 
   AddInFlightOp(op.get());
-  VLOG(3) << "Looking up tablet for " << op->write_op->ToString();
+  VLOG(3) << "Looking up tablet for " << op->ToString();
   // Increment our reference count for the outstanding callback.
   //
   // deadline_ is set in FlushAsync(), after all Add() calls are done, so
@@ -586,14 +591,14 @@ void Batcher::TabletLookupFinished(InFlightOp* op, const Status& s) {
   std::unique_lock<simple_spinlock> l(lock_);
 
   if (IsAbortedUnlocked()) {
-    VLOG(1) << "Aborted batch: TabletLookupFinished for " << op->write_op->ToString();
+    VLOG(1) << "Aborted batch: TabletLookupFinished for " << op->ToString();
     MarkInFlightOpFailedUnlocked(op, Status::Aborted("Batch aborted"));
     // 'op' is deleted by above function.
     return;
   }
 
   if (VLOG_IS_ON(3)) {
-    VLOG(3) << "TabletLookupFinished for " << op->write_op->ToString()
+    VLOG(3) << "TabletLookupFinished for " << op->ToString()
             << ": " << s.ToString();
     if (s.ok()) {
       VLOG(3) << "Result: tablet_id = " << op->tablet->tablet_id();

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 9095a4b..3ca0aba 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1183,11 +1183,10 @@ struct CloseCallback {
 } // anonymous namespace
 
 string KuduScanner::ToString() const {
-  return Substitute("$0: $1",
-                    data_->table_->name(),
-                    data_->configuration()
-                    .spec()
-                    .ToString(*data_->table_->schema().schema_));
+  return KUDU_DISABLE_REDACTION(Substitute(
+      "$0: $1",
+      data_->table_->name(),
+      data_->configuration().spec().ToString(*data_->table_->schema().schema_)));
 }
 
 Status KuduScanner::Open() {
@@ -1200,13 +1199,13 @@ Status KuduScanner::Open() {
 
   if (data_->configuration().spec().CanShortCircuit() ||
       !data_->partition_pruner_.HasMorePartitionKeyRanges()) {
-    VLOG(2) << "Short circuiting scan " << ToString();
+    VLOG(2) << "Short circuiting scan " << data_->DebugString();
     data_->open_ = true;
     data_->short_circuit_ = true;
     return Status::OK();
   }
 
-  VLOG(2) << "Beginning scan " << ToString();
+  VLOG(2) << "Beginning " << data_->DebugString();
 
   MonoTime deadline = MonoTime::Now() + data_->configuration().timeout();
   set<string> blacklist;
@@ -1224,7 +1223,7 @@ Status KuduScanner::KeepAlive() {
 void KuduScanner::Close() {
   if (!data_->open_) return;
 
-  VLOG(2) << "Ending scan " << ToString();
+  VLOG(2) << "Ending " << data_->DebugString();
 
   // Close the scanner on the server-side, if necessary.
   //
@@ -1277,7 +1276,7 @@ Status KuduScanner::NextBatch(KuduScanBatch* batch) {
 
   if (data_->data_in_open_) {
     // We have data from a previous scan.
-    VLOG(2) << "Extracting data from scan " << ToString();
+    VLOG(2) << "Extracting data from " << data_->DebugString();
     data_->data_in_open_ = false;
     return batch->data_->Reset(&data_->controller_,
                                data_->configuration().projection(),
@@ -1285,7 +1284,7 @@ Status KuduScanner::NextBatch(KuduScanBatch* batch) {
                                 make_gscoped_ptr(data_->last_response_.release_data()));
   } else if (data_->last_response_.has_more_results()) {
     // More data is available in this tablet.
-    VLOG(2) << "Continuing scan " << ToString();
+    VLOG(2) << "Continuing " << data_->DebugString();
 
     MonoTime batch_deadline = MonoTime::Now() + data_->configuration().timeout();
     data_->PrepareRequest(KuduScanner::Data::CONTINUE);
@@ -1313,12 +1312,13 @@ Status KuduScanner::NextBatch(KuduScanBatch* batch) {
       Status s = data_->HandleError(result, batch_deadline, &blacklist);
       if (!s.ok()) {
         LOG(WARNING) << "Scan at tablet server " << data_->ts_->ToString() << " of tablet "
-                     << ToString() << " failed: " << result.status.ToString();
+                     << data_->DebugString() << " failed: " << result.status.ToString();
         return s;
       }
 
       if (data_->configuration().is_fault_tolerant()) {
-        LOG(WARNING) << "Attempting to retry scan of tablet " << ToString() << " elsewhere.";
+        LOG(WARNING) << "Attempting to retry scan of tablet " << data_->DebugString()
+                     << " elsewhere.";
         return data_->ReopenCurrentTablet(batch_deadline, &blacklist);
       }
 
@@ -1334,7 +1334,7 @@ Status KuduScanner::NextBatch(KuduScanBatch* batch) {
     // More data may be available in other tablets.
     // No need to close the current tablet; we scanned all the data so the
     // server closed it for us.
-    VLOG(2) << "Scanning next tablet " << ToString();
+    VLOG(2) << "Scanning next tablet " << data_->DebugString();
     data_->last_primary_key_.clear();
     MonoTime deadline = MonoTime::Now() + data_->configuration().timeout();
     set<string> blacklist;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 4030638..2cf5dbc 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -1923,6 +1923,9 @@ class KUDU_EXPORT KuduScanner {
   KuduSchema GetProjectionSchema() const;
 
   /// @return String representation of this scan.
+  ///
+  /// @internal This method must not be used in log messages because it contains
+  ///   sensitive predicate values. Use Scanner::Data::DebugString instead.
   std::string ToString() const;
 
  private:
@@ -2112,9 +2115,6 @@ class KUDU_EXPORT KuduScanTokenBuilder {
   /// @return Operation result status.
   Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
 
-  /// @return String representation of this scan.
-  std::string ToString() const;
-
  private:
   class KUDU_NO_EXPORT Data;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/meta_cache.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/meta_cache.h b/src/kudu/client/meta_cache.h
index 18a4e48..8bb69ce 100644
--- a/src/kudu/client/meta_cache.h
+++ b/src/kudu/client/meta_cache.h
@@ -320,6 +320,11 @@ class MetaCacheEntry {
   // Returns true if this meta cache entry is stale.
   bool stale() const;
 
+  // Returns a formatted string representation of the metacache suitable for
+  // debug printing.
+  //
+  // This string will not be redacted, since table partitions are considered
+  // metadata.
   std::string DebugString(const KuduTable* table) const;
 
  private:

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/scanner-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/scanner-internal.h b/src/kudu/client/scanner-internal.h
index 550711a..4292f7e 100644
--- a/src/kudu/client/scanner-internal.h
+++ b/src/kudu/client/scanner-internal.h
@@ -215,6 +215,17 @@ class KuduScanner::Data {
   // The scanner's cumulative resource metrics since the scan was started.
   ResourceMetrics resource_metrics_;
 
+  // Returns a text description of the scan suitable for debug printing.
+  //
+  // This method will not return sensitive predicate information, so it's
+  // suitable for use in client-side logging (as opposed to Scanner::ToString).
+  std::string DebugString() const {
+    return strings::Substitute("Scanner { table: $0, projection: $1, scan_spec: $2 }",
+                               table_->name(),
+                               configuration_.projection()->ToString(),
+                               configuration_.spec().ToString(*table_->schema().schema_));
+  }
+
  private:
   // Analyze the response of the last Scan RPC made by this scanner.
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/session-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/session-internal.cc b/src/kudu/client/session-internal.cc
index b97fb3d..19666e8 100644
--- a/src/kudu/client/session-internal.cc
+++ b/src/kudu/client/session-internal.cc
@@ -25,6 +25,7 @@
 #include "kudu/client/shared_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/messenger.h"
+#include "kudu/util/logging.h"
 
 namespace kudu {
 
@@ -323,10 +324,8 @@ Status KuduSession::Data::ApplyWriteOp(KuduWriteOperation* write_op) {
     return Status::InvalidArgument("NULL operation");
   }
   if (PREDICT_FALSE(!write_op->row().IsKeySet())) {
-    Status status = Status::IllegalState(
-        "Key not specified", write_op->ToString());
-    error_collector_->AddError(
-        gscoped_ptr<KuduError>(new KuduError(write_op, status)));
+    Status status = Status::IllegalState("Key not specified", KUDU_REDACT(write_op->ToString()));
+    error_collector_->AddError(gscoped_ptr<KuduError>(new KuduError(write_op, status)));
     return status;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/client/write_op.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/write_op.h b/src/kudu/client/write_op.h
index 6513eb7..b4433e5 100644
--- a/src/kudu/client/write_op.h
+++ b/src/kudu/client/write_op.h
@@ -75,6 +75,9 @@ class KUDU_EXPORT KuduWriteOperation {
   KuduPartialRow* mutable_row() { return &row_; }
 
   /// @return String representation of the operation.
+  ///
+  /// @internal this method does *NOT* redact row values. The
+  ///   caller must handle redaction themselves, if necessary.
   virtual std::string ToString() const = 0;
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/column_predicate-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/column_predicate-test.cc b/src/kudu/common/column_predicate-test.cc
index fcafcff..5c13cfa 100644
--- a/src/kudu/common/column_predicate-test.cc
+++ b/src/kudu/common/column_predicate-test.cc
@@ -25,6 +25,8 @@
 #include "kudu/common/types.h"
 #include "kudu/util/test_util.h"
 
+DECLARE_bool(log_redact_user_data);
+
 namespace kudu {
 
 class TestColumnPredicate : public KuduTest {
@@ -1002,4 +1004,12 @@ TEST_F(TestColumnPredicate, TestSelectivity) {
                                   ColumnPredicate::Equality(column_s, &one_s)),
             0);
 }
+
+TEST_F(TestColumnPredicate, TestRedaction) {
+  FLAGS_log_redact_user_data = true;
+  ColumnSchema column_i32("a", INT32, true);
+  int32_t one_32 = 1;
+  ASSERT_EQ("`a` = <redacted>", ColumnPredicate::Equality(column_i32, &one_32).ToString());
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partial_row.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partial_row.cc b/src/kudu/common/partial_row.cc
index a3e54d6..d3957c2 100644
--- a/src/kudu/common/partial_row.cc
+++ b/src/kudu/common/partial_row.cc
@@ -27,6 +27,7 @@
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/bitmap.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/memory/overwrite.h"
 #include "kudu/util/status.h"
 
@@ -665,6 +666,8 @@ bool KuduPartialRow::IsKeySet() const {
 
 
 std::string KuduPartialRow::ToString() const {
+  ScopedDisableRedaction no_redaction;
+
   ContiguousRow row(schema_, row_data_);
   std::string ret;
   bool first = true;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partial_row.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/partial_row.h b/src/kudu/common/partial_row.h
index 22c71dc..96f4d4b 100644
--- a/src/kudu/common/partial_row.h
+++ b/src/kudu/common/partial_row.h
@@ -458,6 +458,8 @@ class KUDU_EXPORT KuduPartialRow {
   bool AllColumnsSet() const;
 
   /// @return String representation for the partial row.
+  ///
+  /// @internal NOTE: this is not redacted.
   std::string ToString() const;
 
   /// @return The schema object for the partial row.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partition-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition-test.cc b/src/kudu/common/partition-test.cc
index 2afb5d6..5b2464c 100644
--- a/src/kudu/common/partition-test.cc
+++ b/src/kudu/common/partition-test.cc
@@ -15,13 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <stdint.h>
+
 #include <algorithm>
-#include <boost/optional.hpp>
 #include <iterator>
-#include <stdint.h>
 #include <utility>
 #include <vector>
 
+#include <boost/optional.hpp>
+#include <gflags/gflags.h>
+
 #include "kudu/common/common.pb.h"
 #include "kudu/common/partial_row.h"
 #include "kudu/common/partition.h"
@@ -38,6 +41,8 @@ using std::pair;
 using std::string;
 using std::vector;
 
+DECLARE_bool(log_redact_user_data);
+
 namespace kudu {
 
 namespace {
@@ -116,10 +121,12 @@ void CheckCreateRangePartitions(const vector<pair<optional<string>, optional<str
 
 } // namespace
 
+class PartitionTest : public KuduTest {};
+
 // Tests that missing values are correctly filled in with minimums when creating
 // range partition keys, and that completely missing keys are encoded as the
 // logical minimum and logical maximum for lower and upper bounds, respectively.
-TEST(PartitionTest, TestCompoundRangeKeyEncoding) {
+TEST_F(PartitionTest, TestCompoundRangeKeyEncoding) {
 
   // CREATE TABLE t (c1 STRING, c2 STRING, c3 STRING),
   // PRIMARY KEY (c1, c2, c3)
@@ -184,7 +191,7 @@ TEST(PartitionTest, TestCompoundRangeKeyEncoding) {
             partition_schema.PartitionDebugString(partitions[3], schema));
 }
 
-TEST(PartitionTest, TestPartitionKeyEncoding) {
+TEST_F(PartitionTest, TestPartitionKeyEncoding) {
   // CREATE TABLE t (a INT32, b VARCHAR, c VARCHAR, PRIMARY KEY (a, b, c))
   // PARITITION BY [HASH BUCKET (a, b), HASH BUCKET (c), RANGE (a, b, c)];
   Schema schema({ ColumnSchema("a", INT32),
@@ -273,9 +280,25 @@ TEST(PartitionTest, TestPartitionKeyEncoding) {
     EXPECT_EQ(expected, partition_schema.PartitionKeyDebugString(row));
     EXPECT_EQ(expected, partition_schema.PartitionKeyDebugString(key, schema));
   }
+
+  {
+    // Check that row values are redacted when the log_redact_user_data flag is set.
+    FLAGS_log_redact_user_data = true;
+    string key;
+    KuduPartialRow row(&schema);
+    ASSERT_OK(row.SetInt32("a", 1));
+    ASSERT_OK(row.SetStringCopy("b", "b"));
+    ASSERT_OK(row.SetStringCopy("c", "c"));
+    ASSERT_OK(partition_schema.EncodeKey(row, &key));
+
+    string expected =
+      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));
+  }
 }
 
-TEST(PartitionTest, TestCreateRangePartitions) {
+TEST_F(PartitionTest, TestCreateRangePartitions) {
   {
     // Splits:
     // { a: "1" }
@@ -382,7 +405,7 @@ TEST(PartitionTest, TestCreateRangePartitions) {
   }
 }
 
-TEST(PartitionTest, TestCreateHashBucketPartitions) {
+TEST_F(PartitionTest, TestCreateHashBucketPartitions) {
   // CREATE TABLE t (a VARCHAR PRIMARY KEY),
   // PARITITION BY [HASH BUCKET (a)];
   Schema schema({ ColumnSchema("a", STRING) }, { ColumnId(0) }, 1);
@@ -430,7 +453,12 @@ TEST(PartitionTest, TestCreateHashBucketPartitions) {
             partition_schema.PartitionDebugString(partitions[2], schema));
 }
 
-TEST(PartitionTest, TestCreatePartitions) {
+TEST_F(PartitionTest, TestCreatePartitions) {
+  // Explicitly enable redaction. It should have no effect on the subsequent
+  // partition pretty printing tests, as partitions are metadata and thus not
+  // redacted.
+  FLAGS_log_redact_user_data = true;
+
   // CREATE TABLE t (a VARCHAR, b VARCHAR, c VARCHAR, PRIMARY KEY (a, b, c))
   // PARITITION BY [HASH BUCKET (a), HASH BUCKET (b), RANGE (a, b, c)];
   Schema schema({ ColumnSchema("a", STRING),
@@ -618,7 +646,7 @@ TEST(PartitionTest, TestCreatePartitions) {
             partition_schema.PartitionDebugString(partitions[11], schema));
 }
 
-TEST(PartitionTest, TestIncrementRangePartitionBounds) {
+TEST_F(PartitionTest, TestIncrementRangePartitionBounds) {
   // CREATE TABLE t (a INT8, b INT8, c INT8, PRIMARY KEY (a, b, c))
   // PARITITION BY RANGE (a, b, c);
   Schema schema({ ColumnSchema("c1", INT8),
@@ -711,7 +739,7 @@ TEST(PartitionTest, TestIncrementRangePartitionBounds) {
               s.ToString());
 }
 
-TEST(PartitionTest, TestIncrementRangePartitionStringBounds) {
+TEST_F(PartitionTest, TestIncrementRangePartitionStringBounds) {
   // CREATE TABLE t (a STRING, b STRING, PRIMARY KEY (a, b))
   // PARITITION BY RANGE (a, b, c);
   Schema schema({ ColumnSchema("c1", STRING),

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partition.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition.cc b/src/kudu/common/partition.cc
index 4dc63e9..fcd23b1 100644
--- a/src/kudu/common/partition.cc
+++ b/src/kudu/common/partition.cc
@@ -30,6 +30,7 @@
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/hash_util.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/url-coding.h"
 
 using std::pair;
@@ -318,7 +319,8 @@ Status PartitionSchema::SplitRangeBounds(const Schema& schema,
     for (; split != splits.end() && (upper.empty() || *split <= upper); split++) {
       if (!lower.empty() && *split < lower) {
         return Status::InvalidArgument("split out of bounds", RangeKeyDebugString(*split, schema));
-      } else if (lower == *split || upper == *split) {
+      }
+      if (lower == *split || upper == *split) {
         return Status::InvalidArgument("split matches lower or upper bound",
                                        RangeKeyDebugString(*split, schema));
       }
@@ -582,6 +584,9 @@ string ColumnIdsToColumnNames(const Schema& schema,
 
 string PartitionSchema::PartitionDebugString(const Partition& partition,
                                              const Schema& schema) const {
+  // Partitions are considered metadata, so don't redact them.
+  ScopedDisableRedaction no_redaction;
+
   vector<string> components;
   if (partition.hash_buckets_.size() != hash_bucket_schemas_.size()) {
     return "<hash-partition-error>";
@@ -675,6 +680,9 @@ string PartitionSchema::PartitionKeyDebugString(Slice key, const Schema& schema)
 
 string PartitionSchema::RangePartitionDebugString(const KuduPartialRow& lower_bound,
                                                   const KuduPartialRow& upper_bound) const {
+  // Partitions are considered metadata, so don't redact them.
+  ScopedDisableRedaction no_redaction;
+
   bool lower_unbounded = IsRangePartitionKeyEmpty(lower_bound);
   bool upper_unbounded = IsRangePartitionKeyEmpty(upper_bound);
   if (lower_unbounded && upper_unbounded) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partition.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition.h b/src/kudu/common/partition.h
index 60e15e8..efc90a2 100644
--- a/src/kudu/common/partition.h
+++ b/src/kudu/common/partition.h
@@ -125,6 +125,15 @@ class Partition {
 // When creating the initial set of partitions during table creation, we deal
 // with this by "carrying through" absolute-start or absolute-ends into lower
 // significance components.
+//
+// Notes on redaction:
+//
+// For the purposes of redaction, Kudu considers partitions and partition
+// schemas to be metadata - not sensitive data which needs to be redacted from
+// log files. However, the partition keys of individual rows _are_ considered
+// sensitive, so we redact them from log messages and error messages. Thus,
+// methods which format partitions and partition schemas will never redact, but
+// the methods which format individual partition keys do redact.
 class PartitionSchema {
  public:
 
@@ -165,6 +174,9 @@ class PartitionSchema {
                               bool* contains) const WARN_UNUSED_RESULT;
 
   // Returns a text description of the partition suitable for debug printing.
+  //
+  // Partitions are considered metadata, so no redaction will happen on the hash
+  // and range bound values.
   std::string PartitionDebugString(const Partition& partition, const Schema& schema) const;
 
   // Returns a text description of a partition key suitable for debug printing.
@@ -174,18 +186,19 @@ class PartitionSchema {
 
   // Returns a text description of the range partition with the provided
   // inclusive lower bound and exclusive upper bound.
+  //
+  // Range partitions are considered metadata, so no redaction will happen on
+  // the row values.
   std::string RangePartitionDebugString(const KuduPartialRow& lower_bound,
                                         const KuduPartialRow& upper_bound) const;
   std::string RangePartitionDebugString(Slice lower_bound,
                                         Slice upper_bound,
                                         const Schema& schema) const;
 
-  // Returns a text description of the encoded range key suitable for debug printing.
-  std::string RangeKeyDebugString(Slice range_key, const Schema& schema) const;
-  std::string RangeKeyDebugString(const KuduPartialRow& key) const;
-  std::string RangeKeyDebugString(const ConstContiguousRow& key) const;
-
   // Returns a text description of this partition schema suitable for debug printing.
+  //
+  // The partition schema is considered metadata, so partition bound information
+  // is not redacted from the returned string.
   std::string DebugString(const Schema& schema) const;
 
   // Returns a text description of this partition schema suitable for display in the web UI.
@@ -199,6 +212,8 @@ class PartitionSchema {
   // Returns header and entry HTML cells for the partition schema for the master
   // table web UI. This is an abstraction leak, but it's better than leaking the
   // internals of partitions to the master path handlers.
+  //
+  // Partitions are considered metadata, so no redaction will be done.
   std::string PartitionTableHeader(const Schema& schema) const;
   std::string PartitionTableEntry(const Schema& schema, const Partition& partition) const;
 
@@ -207,10 +222,16 @@ class PartitionSchema {
 
   // Transforms an exclusive lower bound range partition key into an inclusive
   // lower bound range partition key.
+  //
+  // The provided partial row is considered metadata, so error messages may
+  // contain unredacted row data.
   Status MakeLowerBoundRangePartitionKeyInclusive(KuduPartialRow* row) const;
 
   // Transforms an inclusive upper bound range partition key into an exclusive
   // upper bound range partition key.
+  //
+  // The provided partial row is considered metadata, so error messages may
+  // contain unredacted row data.
   Status MakeUpperBoundRangePartitionKeyExclusive(KuduPartialRow* row) const;
 
  private:
@@ -228,6 +249,11 @@ class PartitionSchema {
     uint32_t seed;
   };
 
+  // Returns a text description of the encoded range key suitable for debug printing.
+  std::string RangeKeyDebugString(Slice range_key, const Schema& schema) const;
+  std::string RangeKeyDebugString(const KuduPartialRow& key) const;
+  std::string RangeKeyDebugString(const ConstContiguousRow& key) const;
+
   // Encodes the specified columns of a row into lexicographic sort-order
   // preserving format.
   static Status EncodeColumns(const KuduPartialRow& row,
@@ -277,8 +303,11 @@ class PartitionSchema {
   void AppendRangeDebugStringComponentsOrMin(const KuduPartialRow& row,
                                              std::vector<std::string>* components) const;
 
-  /// Returns the stringified hash and range schema componenets of the partition
-  /// schema.
+  // Returns the stringified hash and range schema componenets of the partition
+  // schema.
+  //
+  // Partition schemas are considered metadata, so no redaction will happen on
+  // the hash and range bound values.
   std::vector<std::string> DebugStringComponents(const Schema& schema) const;
 
   // Encode the provided row into a range key. The row must not include values

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partition_pruner-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition_pruner-test.cc b/src/kudu/common/partition_pruner-test.cc
index 0a0e8e9..701d4a1 100644
--- a/src/kudu/common/partition_pruner-test.cc
+++ b/src/kudu/common/partition_pruner-test.cc
@@ -55,13 +55,13 @@ void CheckPrunedPartitions(const Schema& schema,
   SCOPED_TRACE(strings::Substitute("schema: $0", schema.ToString()));
   SCOPED_TRACE(strings::Substitute("partition schema: $0", partition_schema.DebugString(schema)));
   SCOPED_TRACE(strings::Substitute("partition pruner: $0",
-                                    pruner.ToString(schema, partition_schema)));
+                                   pruner.ToString(schema, partition_schema)));
   SCOPED_TRACE(strings::Substitute("scan spec: $0", spec.ToString(schema)));
 
   int pruned_partitions = count_if(partitions.begin(), partitions.end(),
-                                    [&] (const Partition& partition) {
-                                      return pruner.ShouldPrune(partition);
-                                    });
+                                   [&] (const Partition& partition) {
+                                     return pruner.ShouldPrune(partition);
+                                   });
   ASSERT_EQ(remaining_tablets, partitions.size() - pruned_partitions);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/partition_pruner.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition_pruner.h b/src/kudu/common/partition_pruner.h
index 5b2fb26..632c8dc 100644
--- a/src/kudu/common/partition_pruner.h
+++ b/src/kudu/common/partition_pruner.h
@@ -60,6 +60,8 @@ class PartitionPruner {
   // Returns true if the provided partition should be pruned.
   bool ShouldPrune(const Partition& partition) const;
 
+  // Returns a text description of this partition pruner suitable for debug
+  // printing.
   std::string ToString(const Schema& schema, const PartitionSchema& partition_schema) const;
 
  private:

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/row_operations.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_operations.cc b/src/kudu/common/row_operations.cc
index 4c8adc6..388225a 100644
--- a/src/kudu/common/row_operations.cc
+++ b/src/kudu/common/row_operations.cc
@@ -23,6 +23,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/bitmap.h"
 #include "kudu/util/faststring.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/safe_math.h"
 #include "kudu/util/slice.h"
 
@@ -32,6 +33,9 @@ using strings::Substitute;
 namespace kudu {
 
 string DecodedRowOperation::ToString(const Schema& schema) const {
+  // A note on redaction: We redact row operations, since they contain sensitive
+  // row data. Range partition operations are not redacted, since range
+  // partitions are considered to be metadata.
   switch (type) {
     case RowOperationsPB::UNKNOWN:
       return "UNKNOWN";
@@ -45,15 +49,17 @@ string DecodedRowOperation::ToString(const Schema& schema) const {
                         schema.DebugRowKey(ConstContiguousRow(&schema, row_data)),
                         changelist.ToString(schema));
     case RowOperationsPB::SPLIT_ROW:
-      return Substitute("SPLIT_ROW $0", split_row->ToString());
+      return Substitute("SPLIT_ROW $0", KUDU_DISABLE_REDACTION(split_row->ToString()));
     case RowOperationsPB::RANGE_LOWER_BOUND:
-      return Substitute("RANGE_LOWER_BOUND $0", split_row->ToString());
+      return Substitute("RANGE_LOWER_BOUND $0", KUDU_DISABLE_REDACTION(split_row->ToString()));
     case RowOperationsPB::RANGE_UPPER_BOUND:
-      return Substitute("RANGE_UPPER_BOUND $0", split_row->ToString());
+      return Substitute("RANGE_UPPER_BOUND $0", KUDU_DISABLE_REDACTION(split_row->ToString()));
     case RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND:
-      return Substitute("EXCLUSIVE_RANGE_LOWER_BOUND $0", split_row->ToString());
+      return Substitute("EXCLUSIVE_RANGE_LOWER_BOUND $0",
+                        KUDU_DISABLE_REDACTION(split_row->ToString()));
     case RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND:
-      return Substitute("INCLUSIVE_RANGE_UPPER_BOUND $0", split_row->ToString());
+      return Substitute("INCLUSIVE_RANGE_UPPER_BOUND $0",
+                        KUDU_DISABLE_REDACTION(split_row->ToString()));
   }
   return "UNKNOWN";
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/row_operations.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_operations.h b/src/kudu/common/row_operations.h
index 3bce09b..00defaa 100644
--- a/src/kudu/common/row_operations.h
+++ b/src/kudu/common/row_operations.h
@@ -66,6 +66,7 @@ struct DecodedRowOperation {
   // For SPLIT_ROW, the partial row to split on.
   std::shared_ptr<KuduPartialRow> split_row;
 
+  // Stringifies, including redaction when appropriate.
   std::string ToString(const Schema& schema) const;
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/scan_spec-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/scan_spec-test.cc b/src/kudu/common/scan_spec-test.cc
index 795c49f..bf40be7 100644
--- a/src/kudu/common/scan_spec-test.cc
+++ b/src/kudu/common/scan_spec-test.cc
@@ -31,6 +31,8 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
+DECLARE_bool(log_redact_user_data);
+
 namespace kudu {
 
 class TestScanSpec : public KuduTest {
@@ -363,6 +365,13 @@ TEST_F(CompositeIntKeysTest, TestInListPushdownWithRange) {
             "PK < (int8 a=101, int8 b=-128, int8 c=-128) AND "
             "`b` IN (50, 100)",
             spec.ToString(schema_));
+
+  // Test redaction.
+  FLAGS_log_redact_user_data = true;
+  EXPECT_EQ("PK >= (int8 a=<redacted>, int8 b=<redacted>, int8 c=<redacted>) AND "
+            "PK < (int8 a=<redacted>, int8 b=<redacted>, int8 c=<redacted>) AND "
+            "`b` IN (<redacted>, <redacted>)",
+            spec.ToString(schema_));
 }
 
 // Tests that a scan spec without primary key bounds will not have predicates

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/schema-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema-test.cc b/src/kudu/common/schema-test.cc
index 0f59764..ed8f24e 100644
--- a/src/kudu/common/schema-test.cc
+++ b/src/kudu/common/schema-test.cc
@@ -28,6 +28,7 @@
 #include "kudu/util/hexdump.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
 
 namespace kudu {
 namespace tablet {
@@ -53,10 +54,10 @@ static Status CopyRowToArena(const Slice &row,
   return Status::OK();
 }
 
-
+class TestSchema : public KuduTest {};
 
 // Test basic functionality of Schema definition
-TEST(TestSchema, TestSchema) {
+TEST_F(TestSchema, TestSchema) {
   Schema empty_schema;
   ASSERT_GT(empty_schema.memory_footprint_excluding_this(), 0);
 
@@ -85,7 +86,7 @@ TEST(TestSchema, TestSchema) {
   EXPECT_EQ("uint32 NULLABLE", schema.column(1).TypeToString());
 }
 
-TEST(TestSchema, TestSwap) {
+TEST_F(TestSchema, TestSwap) {
   Schema schema1({ ColumnSchema("col1", STRING),
                    ColumnSchema("col2", STRING),
                    ColumnSchema("col3", UINT32) },
@@ -100,7 +101,7 @@ TEST(TestSchema, TestSwap) {
   ASSERT_EQ(2, schema2.num_key_columns());
 }
 
-TEST(TestSchema, TestReset) {
+TEST_F(TestSchema, TestReset) {
   Schema schema;
   ASSERT_FALSE(schema.initialized());
 
@@ -118,7 +119,7 @@ TEST(TestSchema, TestReset) {
 
 // Test for KUDU-943, a bug where we suspected that Variant didn't behave
 // correctly with empty strings.
-TEST(TestSchema, TestEmptyVariant) {
+TEST_F(TestSchema, TestEmptyVariant) {
   Slice empty_val("");
   Slice nonempty_val("test");
 
@@ -130,7 +131,7 @@ TEST(TestSchema, TestEmptyVariant) {
   ASSERT_EQ("test", (static_cast<const Slice*>(v.value()))->ToString());
 }
 
-TEST(TestSchema, TestProjectSubset) {
+TEST_F(TestSchema, TestProjectSubset) {
   Schema schema1({ ColumnSchema("col1", STRING),
                    ColumnSchema("col2", STRING),
                    ColumnSchema("col3", UINT32) },
@@ -156,7 +157,7 @@ TEST(TestSchema, TestProjectSubset) {
 
 // Test projection when the type of the projected column
 // doesn't match the original type.
-TEST(TestSchema, TestProjectTypeMismatch) {
+TEST_F(TestSchema, TestProjectTypeMismatch) {
   Schema schema1({ ColumnSchema("key", STRING),
                    ColumnSchema("val", UINT32) },
                  1);
@@ -170,7 +171,7 @@ TEST(TestSchema, TestProjectTypeMismatch) {
 
 // Test projection when the some columns in the projection
 // are not present in the base schema
-TEST(TestSchema, TestProjectMissingColumn) {
+TEST_F(TestSchema, TestProjectMissingColumn) {
   Schema schema1({ ColumnSchema("key", STRING), ColumnSchema("val", UINT32) }, 1);
   Schema schema2({ ColumnSchema("val", UINT32), ColumnSchema("non_present", STRING) }, 0);
   Schema schema3({ ColumnSchema("val", UINT32), ColumnSchema("non_present", UINT32, true) }, 0);
@@ -209,7 +210,7 @@ TEST(TestSchema, TestProjectMissingColumn) {
 // Test projection mapping using IDs.
 // This simulate a column rename ('val' -> 'val_renamed')
 // and a new column added ('non_present')
-TEST(TestSchema, TestProjectRename) {
+TEST_F(TestSchema, TestProjectRename) {
   SchemaBuilder builder;
   ASSERT_OK(builder.AddKeyColumn("key", STRING));
   ASSERT_OK(builder.AddColumn("val", UINT32));
@@ -237,7 +238,7 @@ TEST(TestSchema, TestProjectRename) {
 
 
 // Test that the schema can be used to compare and stringify rows.
-TEST(TestSchema, TestRowOperations) {
+TEST_F(TestSchema, TestRowOperations) {
   Schema schema({ ColumnSchema("col1", STRING),
                   ColumnSchema("col2", STRING),
                   ColumnSchema("col3", UINT32),
@@ -306,7 +307,7 @@ TEST(TestKeyEncoder, TestKeyEncoder) {
   }
 }
 
-TEST(TestSchema, TestDecodeKeys_CompoundStringKey) {
+TEST_F(TestSchema, TestDecodeKeys_CompoundStringKey) {
   Schema schema({ ColumnSchema("col1", STRING),
                   ColumnSchema("col2", STRING),
                   ColumnSchema("col3", STRING) },
@@ -327,7 +328,7 @@ TEST(TestSchema, TestDecodeKeys_CompoundStringKey) {
 
 // Test that appropriate statuses are returned when trying to decode an invalid
 // encoded key.
-TEST(TestSchema, TestDecodeKeys_InvalidKeys) {
+TEST_F(TestSchema, TestDecodeKeys_InvalidKeys) {
   Schema schema({ ColumnSchema("col1", STRING),
                   ColumnSchema("col2", UINT32),
                   ColumnSchema("col3", STRING) },
@@ -344,7 +345,7 @@ TEST(TestSchema, TestDecodeKeys_InvalidKeys) {
             schema.DebugEncodedRowKey(Slice("foo\x00\x00\xff\xff", 7), Schema::START_KEY));
 }
 
-TEST(TestSchema, TestCreateProjection) {
+TEST_F(TestSchema, TestCreateProjection) {
   Schema schema({ ColumnSchema("col1", STRING),
                   ColumnSchema("col2", STRING),
                   ColumnSchema("col3", STRING),

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/types-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/types-test.cc b/src/kudu/common/types-test.cc
index ff93301..5e9f835 100644
--- a/src/kudu/common/types-test.cc
+++ b/src/kudu/common/types-test.cc
@@ -23,6 +23,7 @@
 
 #include "kudu/common/types.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/test_util.h"
 
 using std::get;
 using std::make_tuple;
@@ -33,7 +34,9 @@ using std::vector;
 
 namespace kudu {
 
-TEST(TestTypes, TestTimestampPrinting) {
+class TestTypes : public KuduTest {};
+
+TEST_F(TestTypes, TestTimestampPrinting) {
   const TypeInfo* info = GetTypeInfo(UNIXTIME_MICROS);
 
   // Test the minimum value
@@ -86,7 +89,7 @@ namespace {
   }
 } // anonymous namespace
 
-TEST(TestTypes, TestAreConsecutiveInteger) {
+TEST_F(TestTypes, TestAreConsecutiveInteger) {
   vector<tuple<int64_t, int64_t, bool>> test_cases {
     make_tuple(0, 0, false),
     make_tuple(0, 1, true),
@@ -104,7 +107,7 @@ TEST(TestTypes, TestAreConsecutiveInteger) {
   TestAreConsecutive(INT64, test_cases);
 }
 
-TEST(TestTypes, TestAreConsecutiveDouble) {
+TEST_F(TestTypes, TestAreConsecutiveDouble) {
   vector<tuple<double, double, bool>> test_cases {
     make_tuple(0.0, 1.0, false),
     make_tuple(0.0, 0.1, false),
@@ -116,7 +119,7 @@ TEST(TestTypes, TestAreConsecutiveDouble) {
   TestAreConsecutive(DOUBLE, test_cases);
 }
 
-TEST(TestTypes, TestAreConsecutiveString) {
+TEST_F(TestTypes, TestAreConsecutiveString) {
   vector<tuple<Slice, Slice, bool>> test_cases {
     make_tuple("abc", "abc", false),
     make_tuple("abc", Slice("abc\0", 4), true),

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/common/types.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/types.cc b/src/kudu/common/types.cc
index ed20081..df7e89f 100644
--- a/src/kudu/common/types.cc
+++ b/src/kudu/common/types.cc
@@ -21,6 +21,7 @@
 #include <unordered_map>
 
 #include "kudu/gutil/singleton.h"
+#include "kudu/util/logging.h"
 
 using std::shared_ptr;
 using std::unordered_map;
@@ -41,7 +42,11 @@ TypeInfo::TypeInfo(TypeTraitsClass t)
 }
 
 void TypeInfo::AppendDebugStringForValue(const void *ptr, string *str) const {
-  append_func_(ptr, str);
+  if (KUDU_SHOULD_REDACT()) {
+    str->append(kRedactionMessage);
+  } else {
+    append_func_(ptr, str);
+  }
 }
 
 int TypeInfo::Compare(const void *lhs, const void *rhs) const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/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 1497ad2..8afff37 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -628,6 +628,9 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
   // rely on forcefully cutting power to a machine or equivalent.
   argv.push_back("--never_fsync");
 
+  // Disable log redaction.
+  argv.push_back("--log_redact_user_data=false");
+
   // Enable metrics logging.
   argv.push_back("--metrics_log_interval_ms=1000");
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/server/webserver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver.cc b/src/kudu/server/webserver.cc
index c658ea2..ceca077 100644
--- a/src/kudu/server/webserver.cc
+++ b/src/kudu/server/webserver.cc
@@ -42,6 +42,7 @@
 #include "kudu/util/env.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/url-coding.h"
 #include "kudu/util/version_info.h"
@@ -268,6 +269,10 @@ int Webserver::BeginRequestCallbackStatic(struct sq_connection* connection) {
 
 int Webserver::BeginRequestCallback(struct sq_connection* connection,
                                     struct sq_request_info* request_info) {
+  // Redaction is disabled from the web UI. This affects operations like default
+  // value and scan predicate pretty printing.
+  ScopedDisableRedaction disable_redaction;
+
   PathHandler* handler;
   {
     shared_lock<RWMutex> l(lock_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/tablet/transactions/write_transaction.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/write_transaction.cc b/src/kudu/tablet/transactions/write_transaction.cc
index d9c7347..a56fc19 100644
--- a/src/kudu/tablet/transactions/write_transaction.cc
+++ b/src/kudu/tablet/transactions/write_transaction.cc
@@ -354,8 +354,6 @@ string WriteTransactionState::ToString() const {
   }
 
   // Stringify the actual row operations (eg INSERT/UPDATE/etc)
-  // NOTE: we'll eventually need to gate this by some flag if we want to avoid
-  // user data escaping into the log. See KUDU-387.
   string row_ops_str = "[";
   {
     std::lock_guard<simple_spinlock> l(txn_state_lock_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/tools/tool_action_local_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index 7e0cd22..ec4e8b5 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -416,7 +416,7 @@ Status DumpTabletMeta(FsManager* fs_manager,
 
   cout << Indent(indent) << "Partition: "
        << meta->partition_schema().PartitionDebugString(meta->partition(),
-                                                       meta->schema())
+                                                        meta->schema())
        << endl;
   cout << Indent(indent) << "Table name: " << meta->table_name()
        << " Table id: " << meta->table_id() << endl;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/tools/tool_main.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_main.cc b/src/kudu/tools/tool_main.cc
index 59d7f11..2d0533b 100644
--- a/src/kudu/tools/tool_main.cc
+++ b/src/kudu/tools/tool_main.cc
@@ -35,14 +35,14 @@
 #include "kudu/util/status.h"
 
 DECLARE_bool(help);
-DECLARE_bool(helpshort);
-DECLARE_string(helpon);
-DECLARE_string(helpmatch);
 DECLARE_bool(helppackage);
+DECLARE_bool(helpshort);
 DECLARE_bool(helpxml);
+DECLARE_bool(log_redact_user_data);
+DECLARE_string(helpmatch);
+DECLARE_string(helpon);
 
 using std::cerr;
-using std::cout;
 using std::deque;
 using std::endl;
 using std::string;
@@ -224,6 +224,8 @@ static bool ParseCommandLineFlags(int* argc, char*** argv) {
 int main(int argc, char** argv) {
   bool show_help = ParseCommandLineFlags(&argc, &argv);
   FLAGS_logtostderr = true;
+  // Disable redaction so that user data printed to the console will be shown in full.
+  FLAGS_log_redact_user_data = false;
   kudu::InitGoogleLoggingSafe(argv[0]);
   return kudu::tools::RunTool(argc, argv, show_help);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/util/flag_tags-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flag_tags-test.cc b/src/kudu/util/flag_tags-test.cc
index 3f6849e..89b99a9 100644
--- a/src/kudu/util/flag_tags-test.cc
+++ b/src/kudu/util/flag_tags-test.cc
@@ -25,6 +25,7 @@
 #include "kudu/util/logging_test_util.h"
 #include "kudu/util/test_util.h"
 
+DECLARE_bool(log_redact_user_data);
 DECLARE_bool(never_fsync);
 
 DEFINE_int32(flag_with_no_tags, 0, "test flag that has no tags");
@@ -50,9 +51,10 @@ namespace kudu {
 class FlagTagsTest : public KuduTest {
  public:
   FlagTagsTest() {
-    // Set to true via KuduTest, but explicitly unset here as this test deals
-    // with unsafe flags.
+    // Set by KuduTest, and set back to the default value here as this test
+    // deals with unsafe and experimental flags.
     FLAGS_never_fsync = false;
+    FLAGS_log_redact_user_data = true;
   }
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/util/flags-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags-test.cc b/src/kudu/util/flags-test.cc
index 1bbfc38..ecf16d4 100644
--- a/src/kudu/util/flags-test.cc
+++ b/src/kudu/util/flags-test.cc
@@ -34,15 +34,17 @@ DEFINE_string(test_default_ff, "default",
 DEFINE_string(test_default_explicit, "default",
              "Check if we track explicitly set defaults");
 DECLARE_bool(never_fsync);
+DECLARE_bool(log_redact_user_data);
 
 namespace kudu {
 
 class FlagsTest : public KuduTest {
  public:
   FlagsTest() {
-    // Set to true via KuduTest, but explicitly unset here as this test parses
-    // command line flags and fails if an unsafe flag is set.
+    // Set by KuduTest, and set back to the default value here as this test
+    // deals with unsafe and experimental flags.
     FLAGS_never_fsync = false;
+    FLAGS_log_redact_user_data = true;
   }
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/util/logging-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging-test.cc b/src/kudu/util/logging-test.cc
index 08fc27d..b3eb458 100644
--- a/src/kudu/util/logging-test.cc
+++ b/src/kudu/util/logging-test.cc
@@ -160,4 +160,39 @@ TEST(LoggingTest, TestAsyncLogger) {
   ASSERT_GT(async.app_threads_blocked_count_for_tests(), 0);
 }
 
+// Basic test that the redaction utilities work as expected.
+TEST(LoggingTest, TestRedactionBasic) {
+  ASSERT_STREQ("<redacted>", KUDU_REDACT("hello"));
+  {
+    ScopedDisableRedaction no_redaction;
+    ASSERT_STREQ("hello", KUDU_REDACT("hello"));
+  }
+  ASSERT_STREQ("hello", KUDU_DISABLE_REDACTION(KUDU_REDACT("hello")));
+}
+
+// Typically, ToString() methods apply to some complex object with a bunch
+// of fields, some of which are user data (need redaction) and others of which
+// are not. This shows an example of a such a function, which will behave
+// differently based on whether the calling scope has explicitly disabled
+// redaction.
+string SomeComplexStringify(const string& public_data, const string& private_data) {
+  return strings::Substitute("public=$0, private=$1",
+                             public_data,
+                             KUDU_REDACT(private_data));
+}
+
+TEST(LoggingTest, TestRedactionIllustrateUsage) {
+  // By default, the private data will be redacted.
+  ASSERT_EQ("public=abc, private=<redacted>", SomeComplexStringify("abc", "def"));
+
+  // We can wrap the expression in KUDU_DISABLE_REDACTION(...) to evaluate it
+  // with redaction temporarily disabled.
+  ASSERT_EQ("public=abc, private=def", KUDU_DISABLE_REDACTION(SomeComplexStringify("abc", "def")));
+
+  // Or we can execute an entire scope with redaction disabled.
+  KUDU_DISABLE_REDACTION({
+    ASSERT_EQ("public=abc, private=def", SomeComplexStringify("abc", "def"));
+  });
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/util/logging.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.cc b/src/kudu/util/logging.cc
index 1cc0dda..4438380 100644
--- a/src/kudu/util/logging.cc
+++ b/src/kudu/util/logging.cc
@@ -62,6 +62,11 @@ DEFINE_int32(max_log_files, 10,
 TAG_FLAG(max_log_files, runtime);
 TAG_FLAG(max_log_files, experimental);
 
+DEFINE_bool(log_redact_user_data, true,
+    "Whether log and error messages will have row data redacted.");
+TAG_FLAG(log_redact_user_data, runtime);
+TAG_FLAG(log_redact_user_data, experimental);
+
 #define PROJ_NAME "kudu"
 
 bool logging_initialized = false;
@@ -74,6 +79,9 @@ using base::SpinLockHolder;
 
 namespace kudu {
 
+__thread bool tls_redact_user_data = true;
+const char* const kRedactionMessage = "<redacted>";
+
 namespace {
 
 class SimpleSink : public google::LogSink {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/util/logging.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.h b/src/kudu/util/logging.h
index 5f8769c..9bdc370 100644
--- a/src/kudu/util/logging.h
+++ b/src/kudu/util/logging.h
@@ -18,6 +18,7 @@
 #define KUDU_UTIL_LOGGING_H
 
 #include <string>
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/gutil/atomicops.h"
@@ -28,6 +29,76 @@
 #include "kudu/util/status.h"
 
 ////////////////////////////////////////////////////////////////////////////////
+// Redaction support
+////////////////////////////////////////////////////////////////////////////////
+
+// Disable redaction of user data while evaluating the expression 'expr'.
+// This may be used inline as an expression, such as:
+//
+//   LOG(INFO) << KUDU_DISABLE_REDACTION(schema.DebugRow(my_row));
+//
+// or with a block:
+//
+//  KUDU_DISABLE_REDACTION({
+//    LOG(INFO) << schema.DebugRow(my_row);
+//  });
+//
+// Redaction should be disabled in the following cases:
+//
+// 1) Outputting strings to a "secure" endpoint (for example an authenticated and authorized
+//    web UI)
+//
+// 2) Using methods like schema.DebugRow(...) when the parameter is not in fact a user-provided
+//    row, but instead some piece of metadata such as a partition boundary.
+#define KUDU_DISABLE_REDACTION(expr) ([&]() {        \
+      kudu::ScopedDisableRedaction s;                \
+      return (expr);                                 \
+    })()
+
+// Evaluates to 'true' if the caller should redact any user data in the current scope.
+// Most callers should instead use KUDU_REDACT(...) defined below, but this can be useful
+// to short-circuit expensive logic.
+#define KUDU_SHOULD_REDACT() (FLAGS_log_redact_user_data && kudu::tls_redact_user_data)
+
+// Either evaluate and return 'expr', or return the string "redacted", depending on whether
+// redaction is enabled in the current scope.
+#define KUDU_REDACT(expr) \
+  (KUDU_SHOULD_REDACT() ? kudu::kRedactionMessage : (expr))
+
+
+////////////////////////////////////////
+// Redaction implementation details follow.
+////////////////////////////////////////
+DECLARE_bool(log_redact_user_data);
+
+namespace kudu {
+
+// Flag which allows redaction to be enabled or disabled for a thread context.
+// Defaults to enabling redaction, since it's the safer default with respect to
+// leaking user data, and it's easier to identify when data is over-redacted
+// than vice-versa.
+extern __thread bool tls_redact_user_data;
+
+// Redacted log messages are replaced with this constant.
+extern const char* const kRedactionMessage;
+
+class ScopedDisableRedaction {
+ public:
+  ScopedDisableRedaction()
+      : old_val_(tls_redact_user_data) {
+    tls_redact_user_data = false;
+  }
+
+  ~ScopedDisableRedaction() {
+    tls_redact_user_data = old_val_;
+  }
+ private:
+  bool old_val_;
+};
+
+} // namespace kudu
+
+////////////////////////////////////////////////////////////////////////////////
 // Throttled logging support
 ////////////////////////////////////////////////////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1179bbdb/src/kudu/util/test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index 855330e..3a08ef8 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -33,6 +33,7 @@
 #include "kudu/util/random.h"
 #include "kudu/util/spinlock_profiling.h"
 
+DECLARE_bool(log_redact_user_data);
 DECLARE_bool(never_fsync);
 
 DEFINE_string(test_leave_files, "on_failure",
@@ -69,6 +70,9 @@ KuduTest::KuduTest()
   // Disabling fsync() speeds up tests dramatically, and it's safe to do as no
   // tests rely on cutting power to a machine or equivalent.
   FLAGS_never_fsync = true;
+
+  // Disable log redaction.
+  FLAGS_log_redact_user_data = false;
 }
 
 KuduTest::~KuduTest() {