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 2017/03/01 18:41:48 UTC

[2/2] kudu git commit: Combine redaction flag for log and flags into one.

Combine redaction flag for log and flags into one.

This adds a new flag --redact to replace --redact_sensitive_flags
and --log_redact_user_data. By default, --redact is enabled with
'all', which means both log and flag redaction is on.

Change-Id: Ia49ae439824351df6c87291b0828173c69ab31b0
Reviewed-on: http://gerrit.cloudera.org:8080/6112
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: 0efc1e26a9a00feff35b5fd27a19558fb8758054
Parents: 82882f6
Author: hahao <ha...@cloudera.com>
Authored: Tue Feb 28 18:00:10 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Wed Mar 1 18:34:29 2017 +0000

----------------------------------------------------------------------
 src/kudu/common/column_predicate-test.cc        |  4 +-
 src/kudu/common/partition-test.cc               | 12 ++-
 src/kudu/common/scan_spec-test.cc               |  4 +-
 src/kudu/common/types-test.cc                   |  6 +-
 .../integration-tests/external_mini_cluster.cc  |  2 +-
 src/kudu/server/default-path-handlers.cc        |  2 +-
 src/kudu/tools/tool_main.cc                     |  3 +-
 src/kudu/util/flag_tags-test.cc                 |  1 -
 src/kudu/util/flag_tags.h                       |  2 +-
 src/kudu/util/flags-test.cc                     |  1 -
 src/kudu/util/flags.cc                          | 80 +++++++++++++++++---
 src/kudu/util/flags.h                           |  4 +-
 src/kudu/util/jsonwriter-test.cc                |  4 +-
 src/kudu/util/logging.cc                        |  7 +-
 src/kudu/util/logging.h                         | 12 ++-
 src/kudu/util/pb_util-test.cc                   |  6 +-
 src/kudu/util/test_util.cc                      |  2 +-
 17 files changed, 100 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/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 d8a4ada..d5d471c 100644
--- a/src/kudu/common/column_predicate-test.cc
+++ b/src/kudu/common/column_predicate-test.cc
@@ -27,8 +27,6 @@
 #include "kudu/common/types.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(log_redact_user_data);
-
 namespace kudu {
 
 class TestColumnPredicate : public KuduTest {
@@ -1086,7 +1084,7 @@ TEST_F(TestColumnPredicate, TestSelectivity) {
 }
 
 TEST_F(TestColumnPredicate, TestRedaction) {
-  FLAGS_log_redact_user_data = true;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
   ColumnSchema column_i32("a", INT32, true);
   int32_t one_32 = 1;
   ASSERT_EQ("`a` = <redacted>", ColumnPredicate::Equality(column_i32, &one_32).ToString());

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/common/partition-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition-test.cc b/src/kudu/common/partition-test.cc
index 30bd138..077ba71 100644
--- a/src/kudu/common/partition-test.cc
+++ b/src/kudu/common/partition-test.cc
@@ -41,8 +41,6 @@ using std::pair;
 using std::string;
 using std::vector;
 
-DECLARE_bool(log_redact_user_data);
-
 namespace kudu {
 
 namespace {
@@ -282,8 +280,8 @@ TEST_F(PartitionTest, TestPartitionKeyEncoding) {
   }
 
   {
-    // Check that row values are redacted when the log_redact_user_data flag is set.
-    FLAGS_log_redact_user_data = true;
+    // Check that row values are redacted when the log redact flag is set.
+    ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
     string key;
     KuduPartialRow row(&schema);
     ASSERT_OK(row.SetInt32("a", 1));
@@ -296,8 +294,8 @@ TEST_F(PartitionTest, TestPartitionKeyEncoding) {
     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.
+    // Check that row values are redacted from error messages when
+    // --redact is set with 'log'.
 
     EXPECT_EQ("<hash-decode-error>",
               partition_schema.PartitionKeyDebugString(string("\0\1\0\1", 4), schema));
@@ -469,7 +467,7 @@ 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;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
 
   // 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)];

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/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 7798854..be218a6 100644
--- a/src/kudu/common/scan_spec-test.cc
+++ b/src/kudu/common/scan_spec-test.cc
@@ -31,8 +31,6 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(log_redact_user_data);
-
 namespace kudu {
 
 class TestScanSpec : public KuduTest {
@@ -371,7 +369,7 @@ TEST_F(CompositeIntKeysTest, TestInListPushdownWithRange) {
             spec.ToString(schema_));
 
   // Test redaction.
-  FLAGS_log_redact_user_data = true;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
   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>)",

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/common/types-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/types-test.cc b/src/kudu/common/types-test.cc
index c297444..c01d227 100644
--- a/src/kudu/common/types-test.cc
+++ b/src/kudu/common/types-test.cc
@@ -34,8 +34,6 @@ using std::string;
 using std::tuple;
 using std::vector;
 
-DECLARE_bool(log_redact_user_data);
-
 namespace kudu {
 
 class TestTypes : public KuduTest {};
@@ -78,9 +76,9 @@ TEST_F(TestTypes, TestTimestampPrinting) {
   result = "";
 
   {
-    // Check that row values are redacted when the log_redact_user_data flag is set.
+    // Check that row values are redacted when --redact is set with 'log'.
     google::FlagSaver flag_saver;
-    FLAGS_log_redact_user_data = true;
+    ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
     time = 0;
     info->AppendDebugStringForValue(&time, &result);
     ASSERT_EQ("<redacted>", result);

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/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 6dc84ae..617e5e9 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -641,7 +641,7 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
   argv.push_back("--enable_minidumps=false");
 
   // Disable log redaction.
-  argv.push_back("--log_redact_user_data=false");
+  argv.push_back("--redact=flag");
 
   // Enable metrics logging.
   argv.push_back("--metrics_log_interval_ms=1000");

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/server/default-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/default-path-handlers.cc b/src/kudu/server/default-path-handlers.cc
index 1ff7b19..064d6ea 100644
--- a/src/kudu/server/default-path-handlers.cc
+++ b/src/kudu/server/default-path-handlers.cc
@@ -112,7 +112,7 @@ static void LogsHandler(const Webserver::WebRequest& req, std::ostringstream* ou
 }
 
 // Registered to handle "/flags", and prints out all command-line flags and their values
-// If --redact_sensitive_flags is true, the values of flags tagged as sensitive will
+// If --redact is set with 'flag', the values of flags tagged as sensitive will
 // be redacted.
 static void FlagsHandler(const Webserver::WebRequest& req, std::ostringstream* output) {
   bool as_text = (req.parsed_args.find("raw") != req.parsed_args.end());

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/tools/tool_main.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_main.cc b/src/kudu/tools/tool_main.cc
index a9dd0e7..dc90db6 100644
--- a/src/kudu/tools/tool_main.cc
+++ b/src/kudu/tools/tool_main.cc
@@ -38,7 +38,6 @@ DECLARE_bool(help);
 DECLARE_bool(helppackage);
 DECLARE_bool(helpshort);
 DECLARE_bool(helpxml);
-DECLARE_bool(log_redact_user_data);
 DECLARE_string(helpmatch);
 DECLARE_string(helpon);
 
@@ -225,7 +224,7 @@ int main(int argc, char** argv) {
   // Disable redaction by default so that user data printed to the console will be shown
   // in full.
   CHECK_NE("",  google::SetCommandLineOptionWithMode(
-      "log_redact_user_data", "false", google::SET_FLAGS_DEFAULT));
+      "redact", "", google::SET_FLAGS_DEFAULT));
 
   FLAGS_logtostderr = true;
   bool show_help = ParseCommandLineFlags(&argc, &argv);

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/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 36ea773..0cf850a 100644
--- a/src/kudu/util/flag_tags-test.cc
+++ b/src/kudu/util/flag_tags-test.cc
@@ -27,7 +27,6 @@
 #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");

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/flag_tags.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/flag_tags.h b/src/kudu/util/flag_tags.h
index 95cf7ea..ddfdab1 100644
--- a/src/kudu/util/flag_tags.h
+++ b/src/kudu/util/flag_tags.h
@@ -82,7 +82,7 @@
 //
 // - "sensitive":
 //         The values of these flags are considered sensitive and will be redacted
-//         if --redact_sensitive_flags is true.
+//         if --redact is set with 'flag'.
 //
 // A given flag may have zero or more tags associated with it. The system does
 // not make any attempt to check integrity of the tags - for example, it allows

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/flags-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags-test.cc b/src/kudu/util/flags-test.cc
index c33179d..e7a90ef 100644
--- a/src/kudu/util/flags-test.cc
+++ b/src/kudu/util/flags-test.cc
@@ -39,7 +39,6 @@ DEFINE_bool(test_sensitive_flag, false, "a sensitive flag");
 TAG_FLAG(test_sensitive_flag, sensitive);
 
 DECLARE_bool(never_fsync);
-DECLARE_bool(log_redact_user_data);
 
 namespace kudu {
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/flags.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags.cc b/src/kudu/util/flags.cc
index 76786db..875f958 100644
--- a/src/kudu/util/flags.cc
+++ b/src/kudu/util/flags.cc
@@ -30,22 +30,27 @@
 #include <gperftools/heap-profiler.h>
 
 #include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/os-util.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/string_case.h"
 #include "kudu/util/url-coding.h"
 #include "kudu/util/version_info.h"
 
 using google::CommandLineFlagInfo;
+
 using std::cout;
 using std::endl;
 using std::string;
 using std::stringstream;
 using std::unordered_set;
 
+using strings::Substitute;
+
 // Because every binary initializes its flags here, we use it as a convenient place
 // to offer some global flags as well.
 DEFINE_bool(dump_metrics_json, false,
@@ -91,8 +96,8 @@ static bool ValidateUmask(const char* /*flagname*/, const string& value) {
   }
   return true;
 }
-static bool dummy = google::RegisterFlagValidator(&FLAGS_umask, &ValidateUmask);
 
+DEFINE_validator(umask, &ValidateUmask);
 
 DEFINE_bool(unlock_experimental_flags, false,
             "Unlock flags marked as 'experimental'. These flags are not guaranteed to "
@@ -108,12 +113,65 @@ DEFINE_bool(unlock_unsafe_flags, false,
 TAG_FLAG(unlock_unsafe_flags, advanced);
 TAG_FLAG(unlock_unsafe_flags, stable);
 
-DEFINE_bool(redact_sensitive_flags, true,
-            "Sensitive flags marked as 'sensitive'. The value of these flags will "
-            "be redacted when logging the configuration at daemon startup.");
-TAG_FLAG(redact_sensitive_flags, advanced);
-TAG_FLAG(redact_sensitive_flags, evolving);
+DEFINE_string(redact, "all",
+              "Comma-separated list of redactions. Supported redactions are 'flag', "
+              "'log' and 'all'. If 'flag' is specified, configuration flags which may "
+              "include sensitive data will be redacted whenever server configuration "
+              "is emitted. If 'log' is specified, row data will be redacted from log "
+              "and error messages. If 'all' is specified, all of above will be redacted.");
+TAG_FLAG(redact, advanced);
+TAG_FLAG(redact, evolving);
+
+static bool ValidateRedact(const char* /*flagname*/, const string& value) {
+  // Empty value is valid.
+  if (value.empty()) {
+    kudu::g_should_redact_log = false;
+    kudu::g_should_redact_flag = false;
+    return true;
+  }
+
+  // Flag value is case insensitive
+  string redact_flags;
+  kudu::ToUpperCase(value, &redact_flags);
+  // "ALL" is valid, "ALL, LOG" is not valid
+  if (redact_flags.compare("ALL") == 0) {
+    kudu::g_should_redact_log = true;
+    kudu::g_should_redact_flag = true;
+    return true;
+  }
+
+  // If use specific flag value, it can only be "FLAG"
+  // or "LOG".
+  vector<string> enabled_redact_types = strings::Split(redact_flags, ",",
+                                                       strings::SkipEmpty());
+  vector<string>::const_iterator iter;
+  bool is_valid = true;
+  bool enabled_redact_log = false;
+  bool enabled_redact_flag = false;
+  for (const auto& t : enabled_redact_types) {
+    if (t.compare("LOG") == 0) {
+      enabled_redact_log = true;
+    } else {
+      if (t.compare("FLAG") == 0) {
+        enabled_redact_flag = true;
+      } else {
+        is_valid = false;
+      }
+    }
+  }
+  if (!is_valid) {
+    LOG(ERROR) << Substitute("Invalid redaction type: $0. Available types are 'flag', "
+                             "'log', 'all'.", value);
+  } else {
+    // If the value of --redact flag is valid, set
+    // g_should_redact_log and g_should_redact_flag accordingly.
+    kudu::g_should_redact_log = enabled_redact_log;
+    kudu::g_should_redact_flag = enabled_redact_flag;
+  }
+  return is_valid;
+}
 
+DEFINE_validator(redact, &ValidateRedact);
 // Tag a bunch of the flags that we inherit from glog/gflags.
 
 //------------------------------------------------------------
@@ -359,14 +417,14 @@ void CheckFlagsAllowed() {
   }
 }
 
-// Redact the flag tagged as 'sensitive', if --redact_sensitive_flags
-// is true. Otherwise, return its value as-is.
+// Redact the flag tagged as 'sensitive', if --redact is set
+// with 'flag'. Otherwise, return its value as-is.
 string CheckFlagAndRedact(const CommandLineFlagInfo& flag) {
   string retval;
   unordered_set<string> tags;
   GetFlagTags(flag.name, &tags);
 
-  if (ContainsKey(tags, "sensitive") && FLAGS_redact_sensitive_flags) {
+  if (ContainsKey(tags, "sensitive") && g_should_redact_flag) {
     retval += kRedactionMessage;
   } else {
     retval += flag.current_value;
@@ -457,8 +515,8 @@ string GetNonDefaultFlags(const GFlagsMap& default_flags) {
           args << '\n';
         }
 
-        // Redact the flags tagged as sensitive, if --redact_sensitive_flags
-        // is true.
+        // Redact the flags tagged as sensitive, if --redact is set
+        // with 'flag'.
         string flagValue = CheckFlagAndRedact(flag);
         args << "--" << flag.name << '=' << flagValue;
       }

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/flags.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags.h b/src/kudu/util/flags.h
index 269ad94..aa6bd94 100644
--- a/src/kudu/util/flags.h
+++ b/src/kudu/util/flags.h
@@ -50,7 +50,7 @@ int ParseCommandLineFlags(int* argc, char*** argv, bool remove_flags);
 // google::ParseCommandLineNonHelpFlags().
 void HandleCommonFlags();
 
-// Stick the flags into a string. If --redact_sensitive_flags is true,
+// Stick the flags into a string. If --redact is set with 'flag',
 // the values of flags tagged as sensitive will be redacted. Otherwise,
 // the values will be written to the string as-is.
 std::string CommandlineFlagsIntoString();
@@ -59,7 +59,7 @@ typedef std::unordered_map<std::string, google::CommandLineFlagInfo> GFlagsMap;
 
 // Get all the flags different from their defaults. The output is a nicely
 // formatted string with --flag=value pairs per line. Redact any flags that
-// are tagged as sensitive, if --redact_sensitive_flags is true.
+// are tagged as sensitive, if --redact is set with 'flag'.
 std::string GetNonDefaultFlags(const GFlagsMap& default_flags);
 
 GFlagsMap GetFlagsMap();

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/jsonwriter-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/jsonwriter-test.cc b/src/kudu/util/jsonwriter-test.cc
index 7f458fa..08d54de 100644
--- a/src/kudu/util/jsonwriter-test.cc
+++ b/src/kudu/util/jsonwriter-test.cc
@@ -35,7 +35,7 @@ TEST_F(TestJsonWriter, TestPBEmpty) {
 }
 
 TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
-  FLAGS_log_redact_user_data = true;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
   TestAllTypes pb;
   pb.set_optional_int32(1);
   pb.set_optional_int64(2);
@@ -93,7 +93,7 @@ TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
 }
 
 TEST_F(TestJsonWriter, TestPBRepeatedPrimitives) {
-  FLAGS_log_redact_user_data = true;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
   TestAllTypes pb;
   for (int i = 0; i <= 3; i++) {
     pb.add_repeated_int32(i);

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/logging.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.cc b/src/kudu/util/logging.cc
index 321504c..f45285d 100644
--- a/src/kudu/util/logging.cc
+++ b/src/kudu/util/logging.cc
@@ -64,11 +64,6 @@ 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;
@@ -82,6 +77,8 @@ using base::SpinLockHolder;
 namespace kudu {
 
 __thread bool tls_redact_user_data = true;
+bool g_should_redact_log;
+bool g_should_redact_flag;
 const char* const kRedactionMessage = "<redacted>";
 
 namespace {

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/logging.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.h b/src/kudu/util/logging.h
index e0b5dff..1640793 100644
--- a/src/kudu/util/logging.h
+++ b/src/kudu/util/logging.h
@@ -25,6 +25,7 @@
 #include "kudu/gutil/dynamic_annotations.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/walltime.h"
+#include "kudu/util/flags.h"
 #include "kudu/util/logging_callback.h"
 #include "kudu/util/status.h"
 
@@ -58,12 +59,12 @@
 // 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)
+#define KUDU_SHOULD_REDACT() (kudu::g_should_redact_log && 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))
+  (KUDU_SHOULD_REDACT() ? kRedactionMessage : (expr))
 
 // Like the above, but with the additional condition that redaction will only
 // be performed if 'cond' must be true.
@@ -73,7 +74,6 @@
 ////////////////////////////////////////
 // Redaction implementation details follow.
 ////////////////////////////////////////
-DECLARE_bool(log_redact_user_data);
 
 namespace kudu {
 
@@ -86,6 +86,12 @@ extern __thread bool tls_redact_user_data;
 // Redacted log messages are replaced with this constant.
 extern const char* const kRedactionMessage;
 
+// Flag for checking if log redaction is enabled or disabled.
+extern bool g_should_redact_log;
+
+// Flag for checking if flag redaction is enabled or disabled.
+extern bool g_should_redact_flag;
+
 class ScopedDisableRedaction {
  public:
   ScopedDisableRedaction()

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/pb_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util-test.cc b/src/kudu/util/pb_util-test.cc
index 94acd21..a4b63a8 100644
--- a/src/kudu/util/pb_util-test.cc
+++ b/src/kudu/util/pb_util-test.cc
@@ -36,8 +36,6 @@
 #include "kudu/util/status.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(log_redact_user_data);
-
 namespace kudu {
 namespace pb_util {
 
@@ -585,7 +583,7 @@ TEST_F(TestPBUtil, TestOverwriteExistingPB) {
 }
 
 TEST_F(TestPBUtil, TestRedaction) {
-  FLAGS_log_redact_user_data = true;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
   TestSecurePrintingPB pb;
 
   pb.set_insecure1("public 1");
@@ -605,7 +603,7 @@ TEST_F(TestPBUtil, TestRedaction) {
   }
 
   // If we disable redaction, we should see the private fields.
-  FLAGS_log_redact_user_data = false;
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", ""));
   ASSERT_STR_CONTAINS(SecureDebugString(pb), "private");
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/0efc1e26/src/kudu/util/test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index edc2c59..1fa8c8b 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -70,7 +70,7 @@ KuduTest::KuduTest()
     // tests rely on cutting power to a machine or equivalent.
     {"never_fsync", "true"},
     // Disable log redaction.
-    {"log_redact_user_data", "false"},
+    {"redact", "flag"},
     // Reduce default RSA key length for faster tests.
     {"ipki_server_key_size", "1024"},
     {"ipki_ca_key_size", "1024"}