You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2020/01/14 19:41:33 UTC

[kudu] 01/03: [tools] Support running the master and tablet server via the kudu binary

This is an automated email from the ASF dual-hosted git repository.

granthenke pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 4f82a46da97d9987139aee26614e904118bce4a9
Author: Grant Henke <gr...@apache.org>
AuthorDate: Sun Feb 17 14:48:15 2019 -0600

    [tools] Support running the master and tablet server via the kudu binary
    
    Adds commands to the kudu tools to run the master
    and tablet server:
       - `kudu master run ...`
       - `kudu tserver run ...`
    
    This means we can ship a single binary in the kudu
    docker image or potentially the kudu-binary jar
    reducing the size by approximately 66%. Though
    follow up changes may be needed in the kudu-binary
    case.
    
    The behavior is the same as running the kudu-master
    and kudu-tserver binaries. The logging initialization was
    updated to ensure that the logging filenames are
    the same as when running via the dedicated binaries.
    A few logging fixes are included to ensure this capability
    is possible.
    
    This patch also contains a few related fixes for default
    flags ensuring that they can be overwritten when
    necessary. These changes also ensure flags are not
    incorrectly reported as non-default.
    
    Change-Id: I3717cbac930b3506a76f7a51388c64afbcbb480e
    Reviewed-on: http://gerrit.cloudera.org:8080/12517
    Tested-by: Grant Henke <gr...@apache.org>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/master/CMakeLists.txt                     |  1 +
 src/kudu/master/master-test.cc                     |  2 +
 src/kudu/master/master_main.cc                     | 73 +-----------------
 .../master/{master_main.cc => master_runner.cc}    | 89 ++++++++--------------
 src/kudu/master/master_runner.h                    | 31 ++++++++
 src/kudu/server/diagnostics_log.cc                 |  4 +-
 src/kudu/server/diagnostics_log.h                  |  3 +-
 src/kudu/server/server_base.cc                     |  4 +-
 src/kudu/tools/kudu-tool-test.cc                   | 12 +--
 src/kudu/tools/tool_action.cc                      | 27 ++++++-
 src/kudu/tools/tool_action.h                       | 15 ++++
 src/kudu/tools/tool_action_master.cc               | 40 +++++++++-
 src/kudu/tools/tool_action_tserver.cc              | 41 +++++++++-
 src/kudu/tools/tool_main.cc                        |  8 +-
 src/kudu/tserver/CMakeLists.txt                    |  1 +
 src/kudu/tserver/tablet_server.h                   |  1 +
 src/kudu/tserver/tablet_server_main.cc             | 55 +------------
 ...blet_server_main.cc => tablet_server_runner.cc} | 52 +++++--------
 src/kudu/tserver/tablet_server_runner.h            | 31 ++++++++
 src/kudu/util/flag_tags-test.cc                    | 16 ++--
 src/kudu/util/flags-test.cc                        |  9 +--
 src/kudu/util/flags.cc                             | 28 ++++---
 src/kudu/util/flags.h                              |  7 +-
 src/kudu/util/logging.cc                           | 10 ++-
 src/kudu/util/logging.h                            |  1 +
 src/kudu/util/minidump.cc                          | 12 ++-
 src/kudu/util/rolling_log-test.cc                  |  6 +-
 src/kudu/util/rolling_log.cc                       | 10 ++-
 src/kudu/util/rolling_log.h                        |  6 +-
 29 files changed, 316 insertions(+), 279 deletions(-)

diff --git a/src/kudu/master/CMakeLists.txt b/src/kudu/master/CMakeLists.txt
index 39092fd..d4e27be 100644
--- a/src/kudu/master/CMakeLists.txt
+++ b/src/kudu/master/CMakeLists.txt
@@ -41,6 +41,7 @@ set(MASTER_SRCS
   master_cert_authority.cc
   master_options.cc
   master_path_handlers.cc
+  master_runner.cc
   master_service.cc
   mini_master.cc
   placement_policy.cc
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 331e502..dfb13cd 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -116,6 +116,7 @@ DECLARE_int32(master_inject_latency_on_tablet_lookups_ms);
 DECLARE_int64(live_row_count_for_testing);
 DECLARE_int64(on_disk_size_for_testing);
 DECLARE_string(location_mapping_cmd);
+DECLARE_string(log_filename);
 
 namespace kudu {
 namespace master {
@@ -913,6 +914,7 @@ TEST_F(MasterTest, TestDumpStacksOnRpcQueueOverflow) {
   // Use a new log directory so that the tserver and master don't share the
   // same one. This allows us to isolate the diagnostics log from the master.
   FLAGS_log_dir = JoinPathSegments(GetTestDataDirectory(), "master-logs");
+  FLAGS_log_filename = "kudu-master";
   Status s = env_->CreateDir(FLAGS_log_dir);
   ASSERT_TRUE(s.ok() || s.IsAlreadyPresent()) << s.ToString();
   mini_master_->Shutdown();
diff --git a/src/kudu/master/master_main.cc b/src/kudu/master/master_main.cc
index 7066925..613bdb8 100644
--- a/src/kudu/master/master_main.cc
+++ b/src/kudu/master/master_main.cc
@@ -16,95 +16,28 @@
 // under the License.
 
 #include <iostream>
-#include <string>
 
-#include <gflags/gflags.h>
 #include <glog/logging.h>
 
-#include "kudu/gutil/strings/substitute.h"
-#include "kudu/master/master.h"
-#include "kudu/util/flag_validators.h"
+#include "kudu/master/master_runner.h"
 #include "kudu/util/flags.h"
 #include "kudu/util/init.h"
 #include "kudu/util/logging.h"
-#include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
-#include "kudu/util/version_info.h"
-
-using kudu::master::Master;
-
-DECLARE_bool(evict_failed_followers);
-DECLARE_int32(webserver_port);
-DECLARE_string(rpc_bind_addresses);
-
-DECLARE_bool(hive_metastore_sasl_enabled);
-DECLARE_string(keytab_file);
 
 namespace kudu {
 namespace master {
 
-namespace {
-// Validates that if the HMS is configured with SASL enabled, the server has a
-// keytab available. This is located in master.cc because the HMS module (where
-// -hive_metastore_sasl_enabled is defined) doesn't link to the server module
-// (where --keytab_file is defined), and vice-versa. The master module is the
-// first module which links to both.
-// Note: this check only needs to be run on a server. E.g. tools that run with
-// the HMS don't need to pass in a keytab.
-bool ValidateHiveMetastoreSaslEnabled() {
-  if (FLAGS_hive_metastore_sasl_enabled &&
-      FLAGS_keytab_file.empty()) {
-    LOG(ERROR) << "When the Hive Metastore has SASL enabled "
-                  "(--hive_metastore_sasl_enabled), Kudu must be configured with "
-                  "a keytab (--keytab_file).";
-    return false;
-  }
-  return true;
-}
-GROUP_FLAG_VALIDATOR(hive_metastore_sasl_enabled, ValidateHiveMetastoreSaslEnabled);
-} // anonymous namespace
-
 static int MasterMain(int argc, char** argv) {
   RETURN_MAIN_NOT_OK(InitKudu(), "InitKudu() failed", 1);
-
-  // Reset some default values before parsing gflags.
-  FLAGS_rpc_bind_addresses = strings::Substitute("0.0.0.0:$0",
-                                                 Master::kDefaultPort);
-  FLAGS_webserver_port = Master::kDefaultWebPort;
-
-  // A multi-node Master leader should not evict failed Master followers
-  // because there is no-one to assign replacement servers in order to maintain
-  // the desired replication factor. (It's not turtles all the way down!)
-  FLAGS_evict_failed_followers = false;
-
-  // Setting the default value of the 'force_block_cache_capacity' flag to
-  // 'false' makes the corresponding group validator enforce proper settings
-  // for the memory limit and the cfile cache capacity.
-  CHECK_NE("", SetCommandLineOptionWithMode("force_block_cache_capacity",
-        "false", gflags::SET_FLAGS_DEFAULT));
-
-  GFlagsMap default_flags = GetFlagsMap();
-
+  SetMasterFlagDefaults();
   ParseCommandLineFlags(&argc, &argv, true);
   if (argc != 1) {
     std::cerr << "usage: " << argv[0] << std::endl;
     return 2;
   }
-  std::string nondefault_flags = GetNonDefaultFlags(default_flags);
   InitGoogleLoggingSafe(argv[0]);
-
-  LOG(INFO) << "Master server non-default flags:\n"
-            << nondefault_flags << '\n'
-            << "Master server version:\n"
-            << VersionInfo::GetAllVersionInfo();
-
-  Master server({});
-  RETURN_MAIN_NOT_OK(server.Init(), "Init() failed", 3);
-  RETURN_MAIN_NOT_OK(server.Start(), "Start() failed", 4);
-
-  while (true) {
-    SleepFor(MonoDelta::FromSeconds(60));
-  }
+  RETURN_MAIN_NOT_OK(RunMasterServer(), "RunMasterServer() failed", 3);
 
   return 0;
 }
diff --git a/src/kudu/master/master_main.cc b/src/kudu/master/master_runner.cc
similarity index 50%
copy from src/kudu/master/master_main.cc
copy to src/kudu/master/master_runner.cc
index 7066925..07cec26 100644
--- a/src/kudu/master/master_main.cc
+++ b/src/kudu/master/master_runner.cc
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/master/master_runner.h"
+
 #include <iostream>
 #include <string>
 
@@ -23,95 +25,64 @@
 
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.h"
-#include "kudu/util/flag_validators.h"
 #include "kudu/util/flags.h"
-#include "kudu/util/init.h"
-#include "kudu/util/logging.h"
 #include "kudu/util/monotime.h"
-#include "kudu/util/status.h"
 #include "kudu/util/version_info.h"
 
 using kudu::master::Master;
+using std::string;
 
 DECLARE_bool(evict_failed_followers);
-DECLARE_int32(webserver_port);
-DECLARE_string(rpc_bind_addresses);
-
-DECLARE_bool(hive_metastore_sasl_enabled);
-DECLARE_string(keytab_file);
 
 namespace kudu {
 namespace master {
 
-namespace {
-// Validates that if the HMS is configured with SASL enabled, the server has a
-// keytab available. This is located in master.cc because the HMS module (where
-// -hive_metastore_sasl_enabled is defined) doesn't link to the server module
-// (where --keytab_file is defined), and vice-versa. The master module is the
-// first module which links to both.
-// Note: this check only needs to be run on a server. E.g. tools that run with
-// the HMS don't need to pass in a keytab.
-bool ValidateHiveMetastoreSaslEnabled() {
-  if (FLAGS_hive_metastore_sasl_enabled &&
-      FLAGS_keytab_file.empty()) {
-    LOG(ERROR) << "When the Hive Metastore has SASL enabled "
-                  "(--hive_metastore_sasl_enabled), Kudu must be configured with "
-                  "a keytab (--keytab_file).";
-    return false;
-  }
-  return true;
-}
-GROUP_FLAG_VALIDATOR(hive_metastore_sasl_enabled, ValidateHiveMetastoreSaslEnabled);
-} // anonymous namespace
-
-static int MasterMain(int argc, char** argv) {
-  RETURN_MAIN_NOT_OK(InitKudu(), "InitKudu() failed", 1);
-
+void SetMasterFlagDefaults() {
   // Reset some default values before parsing gflags.
-  FLAGS_rpc_bind_addresses = strings::Substitute("0.0.0.0:$0",
-                                                 Master::kDefaultPort);
-  FLAGS_webserver_port = Master::kDefaultWebPort;
-
-  // A multi-node Master leader should not evict failed Master followers
-  // because there is no-one to assign replacement servers in order to maintain
-  // the desired replication factor. (It's not turtles all the way down!)
-  FLAGS_evict_failed_followers = false;
+  CHECK_NE("", google::SetCommandLineOptionWithMode("rpc_bind_addresses",
+                                                    strings::Substitute(
+                                                        "0.0.0.0:$0",
+                                                        Master::kDefaultPort).c_str(),
+                                                    google::FlagSettingMode::SET_FLAGS_DEFAULT));
+  CHECK_NE("", google::SetCommandLineOptionWithMode("webserver_port",
+                                                    std::to_string(
+                                                        Master::kDefaultWebPort).c_str(),
+                                                    google::FlagSettingMode::SET_FLAGS_DEFAULT));
 
   // Setting the default value of the 'force_block_cache_capacity' flag to
   // 'false' makes the corresponding group validator enforce proper settings
   // for the memory limit and the cfile cache capacity.
   CHECK_NE("", SetCommandLineOptionWithMode("force_block_cache_capacity",
-        "false", gflags::SET_FLAGS_DEFAULT));
+                                            "false",
+                                            gflags::SET_FLAGS_DEFAULT));
 
-  GFlagsMap default_flags = GetFlagsMap();
-
-  ParseCommandLineFlags(&argc, &argv, true);
-  if (argc != 1) {
-    std::cerr << "usage: " << argv[0] << std::endl;
-    return 2;
-  }
-  std::string nondefault_flags = GetNonDefaultFlags(default_flags);
-  InitGoogleLoggingSafe(argv[0]);
+  // A multi-node Master leader should not evict failed Master followers
+  // because there is no-one to assign replacement servers in order to maintain
+  // the desired replication factor. (It's not turtles all the way down!)
+  CHECK_NE("", SetCommandLineOptionWithMode("evict_failed_followers",
+                                            "false",
+                                            gflags::SET_FLAGS_DEFAULT));
+  // SET_FLAGS_DEFAULT won't reset the flag value if it has previously been
+  // set, instead it will only change the default. Because we want to ensure
+  // evict_failed_followers is always false, we explicitly set the flag.
+  FLAGS_evict_failed_followers = false;
+}
 
+Status RunMasterServer() {
+  string nondefault_flags = GetNonDefaultFlags();
   LOG(INFO) << "Master server non-default flags:\n"
             << nondefault_flags << '\n'
             << "Master server version:\n"
             << VersionInfo::GetAllVersionInfo();
 
   Master server({});
-  RETURN_MAIN_NOT_OK(server.Init(), "Init() failed", 3);
-  RETURN_MAIN_NOT_OK(server.Start(), "Start() failed", 4);
+  RETURN_NOT_OK(server.Init());
+  RETURN_NOT_OK(server.Start());
 
   while (true) {
     SleepFor(MonoDelta::FromSeconds(60));
   }
-
-  return 0;
 }
 
 } // namespace master
 } // namespace kudu
-
-int main(int argc, char** argv) {
-  return kudu::master::MasterMain(argc, argv);
-}
diff --git a/src/kudu/master/master_runner.h b/src/kudu/master/master_runner.h
new file mode 100644
index 0000000..a3fc7cf
--- /dev/null
+++ b/src/kudu/master/master_runner.h
@@ -0,0 +1,31 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace master {
+
+// Adjusts the default values of some gflags for use with the master server.
+void SetMasterFlagDefaults();
+
+// Initialize and start a master server. Will run until interrupted.
+Status RunMasterServer();
+
+} // namespace master
+} // namespace kudu
diff --git a/src/kudu/server/diagnostics_log.cc b/src/kudu/server/diagnostics_log.cc
index 415b5fa..2b11540 100644
--- a/src/kudu/server/diagnostics_log.cc
+++ b/src/kudu/server/diagnostics_log.cc
@@ -110,8 +110,10 @@ class DiagnosticsLog::SymbolSet {
 };
 
 DiagnosticsLog::DiagnosticsLog(string log_dir,
+                               string program_name,
                                MetricRegistry* metric_registry) :
     log_dir_(std::move(log_dir)),
+    program_name_(std::move(program_name)),
     metric_registry_(metric_registry),
     wake_(&lock_),
     metrics_log_interval_(MonoDelta::FromSeconds(60)),
@@ -134,7 +136,7 @@ void DiagnosticsLog::DumpStacksNow(std::string reason) {
 
 
 Status DiagnosticsLog::Start() {
-  unique_ptr<RollingLog> l(new RollingLog(Env::Default(), log_dir_, "diagnostics"));
+  unique_ptr<RollingLog> l(new RollingLog(Env::Default(), log_dir_, program_name_, "diagnostics"));
   RETURN_NOT_OK_PREPEND(l->Open(), "unable to open diagnostics log");
   log_ = std::move(l);
   Status s = Thread::Create("server", "diag-logger",
diff --git a/src/kudu/server/diagnostics_log.h b/src/kudu/server/diagnostics_log.h
index 5b8cfab..3841007 100644
--- a/src/kudu/server/diagnostics_log.h
+++ b/src/kudu/server/diagnostics_log.h
@@ -39,7 +39,7 @@ namespace server {
 
 class DiagnosticsLog {
  public:
-  DiagnosticsLog(std::string log_dir, MetricRegistry* metric_registry);
+  DiagnosticsLog(std::string log_dir, std::string program_name, MetricRegistry* metric_registry);
   ~DiagnosticsLog();
 
   void SetMetricsLogInterval(MonoDelta interval);
@@ -68,6 +68,7 @@ class DiagnosticsLog {
   MonoTime ComputeNextWakeup(DiagnosticsLog::WakeupType type) const;
 
   const std::string log_dir_;
+  const std::string program_name_;
   const MetricRegistry* metric_registry_;
 
   scoped_refptr<Thread> thread_;
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index dcca690..c089eb5 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -225,6 +225,7 @@ DECLARE_bool(use_hybrid_clock);
 DECLARE_int32(dns_resolver_max_threads_num);
 DECLARE_uint32(dns_resolver_cache_capacity_mb);
 DECLARE_uint32(dns_resolver_cache_ttl_sec);
+DECLARE_string(log_filename);
 
 METRIC_DECLARE_gauge_size(merged_entities_count_of_server);
 
@@ -691,7 +692,8 @@ Status ServerBase::StartMetricsLogging() {
     LOG(INFO) << "Not starting metrics log since no log directory was specified.";
     return Status::OK();
   }
-  unique_ptr<DiagnosticsLog> l(new DiagnosticsLog(FLAGS_log_dir, metric_registry_.get()));
+  unique_ptr<DiagnosticsLog> l(new DiagnosticsLog(FLAGS_log_dir, FLAGS_log_filename,
+      metric_registry_.get()));
   l->SetMetricsLogInterval(MonoDelta::FromMilliseconds(options_.metrics_log_interval_ms));
   RETURN_NOT_OK(l->Start());
   diag_log_ = std::move(l);
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 88684f8..0c47957 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -327,11 +327,11 @@ class ToolTest : public KuduTest {
     ASSERT_TRUE(stdout.empty());
     ASSERT_FALSE(stderr.empty());
 
-    // If it was an invalid command, the usage string is on the third line.
-    int usage_idx = 1;
+    // If it was an invalid command, the usage string is on the second line.
+    int usage_idx = 0;
     if (!expected_status.ok()) {
-      ASSERT_EQ(expected_status.ToString(), stderr[1]);
-      usage_idx = 2;
+      ASSERT_EQ(expected_status.ToString(), stderr[0]);
+      usage_idx = 1;
     }
     ASSERT_EQ(0, stderr[usage_idx].find("Usage: "));
 
@@ -354,8 +354,8 @@ class ToolTest : public KuduTest {
     vector<string> err_lines;
     RunTool(arg_str, nullptr, nullptr, nullptr, /* stderr_lines = */ &err_lines);
     ASSERT_GE(err_lines.size(), 3) << err_lines;
-    ASSERT_EQ(expected_status.ToString(), err_lines[1]);
-    ASSERT_STR_MATCHES(err_lines[3], "Usage: kudu.*");
+    ASSERT_EQ(expected_status.ToString(), err_lines[0]);
+    ASSERT_STR_MATCHES(err_lines[2], "Usage: kudu.*");
   }
 
   void RunFsCheck(const string& arg_str,
diff --git a/src/kudu/tools/tool_action.cc b/src/kudu/tools/tool_action.cc
index 5ed62fd..b1ed9ab 100644
--- a/src/kudu/tools/tool_action.cc
+++ b/src/kudu/tools/tool_action.cc
@@ -33,6 +33,8 @@
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/url-coding.h"
 
 using std::pair;
@@ -213,6 +215,12 @@ ActionBuilder& ActionBuilder::ExtraDescription(const string& extra_description)
   return *this;
 }
 
+ActionBuilder& ActionBuilder::ProgramName(const string& program_name) {
+  CHECK(!program_name_.is_initialized());
+  program_name_ = program_name;
+  return *this;
+}
+
 ActionBuilder& ActionBuilder::AddRequiredParameter(
     const ActionArgsDescriptor::Arg& arg) {
   args_.required.push_back(arg);
@@ -246,6 +254,7 @@ unique_ptr<Action> ActionBuilder::Build() {
   action->name_ = name_;
   action->description_ = description_;
   action->extra_description_ = extra_description_;
+  action->program_name_ = program_name_;
   action->runner_ = runner_;
   action->args_ = args_;
   return action;
@@ -255,6 +264,18 @@ Status Action::Run(const vector<Mode*>& chain,
                    const unordered_map<string, string>& required_args,
                    const vector<string>& variadic_args) const {
   SetOptionalParameterDefaultValues();
+
+  // If `program_name_` is defined,  initialize the logging as if the
+  // program binary name was `program_name_`, otherwise fallback to the
+  // default behavior of using argv0.
+  if (program_name_) {
+    CHECK_NE("", google::SetCommandLineOptionWithMode("log_filename",
+        program_name_->c_str(),
+        google::FlagSettingMode::SET_FLAGS_DEFAULT));
+  }
+  kudu::InitGoogleLoggingSafe(program_name_.get_value_or(gflags::GetArgv0()).c_str());
+  kudu::ValidateFlags();
+
   return runner_({ chain, this, required_args, variadic_args });
 }
 
@@ -396,9 +417,9 @@ string Action::BuildHelpXML(const vector<Mode*>& chain) const {
 void Action::SetOptionalParameterDefaultValues() const {
   for (const auto& param : args_.optional) {
     if (param.default_value) {
-      google::SetCommandLineOptionWithMode(param.name.c_str(),
-                                           param.default_value->c_str(),
-                                           google::FlagSettingMode::SET_FLAGS_DEFAULT);
+      CHECK_NE("", google::SetCommandLineOptionWithMode(param.name.c_str(),
+          param.default_value->c_str(),
+          google::FlagSettingMode::SET_FLAGS_DEFAULT));
     }
   }
 }
diff --git a/src/kudu/tools/tool_action.h b/src/kudu/tools/tool_action.h
index 5b7aace..b8fea5a 100644
--- a/src/kudu/tools/tool_action.h
+++ b/src/kudu/tools/tool_action.h
@@ -204,6 +204,13 @@ class ActionBuilder {
   // action's help following Description().
   ActionBuilder& ExtraDescription(const std::string& extra_description);
 
+  // Sets the program name to use when running this action. If unset,
+  // argv0 is used.
+  //
+  // This will ensure logging is initialized as if the program binary name was
+  // the passed `program_name`.
+  ActionBuilder& ProgramName(const std::string& program_name);
+
   // Add a new required parameter to this builder.
   //
   // This parameter will be parsed as a positional argument following the name
@@ -247,6 +254,8 @@ class ActionBuilder {
 
   boost::optional<std::string> extra_description_;
 
+  boost::optional<std::string> program_name_;
+
   ActionRunner runner_;
 
   ActionArgsDescriptor args_;
@@ -284,6 +293,10 @@ class Action {
     return extra_description_;
   }
 
+  const boost::optional<std::string>& program_name() const {
+    return program_name_;
+  }
+
   const ActionArgsDescriptor& args() const { return args_; }
 
  private:
@@ -301,6 +314,8 @@ class Action {
 
   boost::optional<std::string> extra_description_;
 
+  boost::optional<std::string> program_name_;
+
   ActionRunner runner_;
 
   ActionArgsDescriptor args_;
diff --git a/src/kudu/tools/tool_action_master.cc b/src/kudu/tools/tool_action_master.cc
index dff62e8..8ed5db1 100644
--- a/src/kudu/tools/tool_action_master.cc
+++ b/src/kudu/tools/tool_action_master.cc
@@ -28,7 +28,7 @@
 
 #include <boost/algorithm/string/predicate.hpp>
 #include <boost/optional/optional.hpp>
-#include <gflags/gflags_declare.h>
+#include <gflags/gflags.h>
 #include <glog/logging.h>
 
 #include "kudu/common/common.pb.h"
@@ -43,9 +43,11 @@
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
+#include "kudu/master/master_runner.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
+#include "kudu/util/init.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 
@@ -87,6 +89,18 @@ Status MasterGetFlags(const RunnerContext& context) {
   return PrintServerFlags(address, Master::kDefaultPort);
 }
 
+Status MasterRun(const RunnerContext& context) {
+  RETURN_NOT_OK(InitKudu());
+
+  // Enable redaction by default. Unlike most tools, we don't want user data
+  // printed to the console/log to be shown by default.
+  CHECK_NE("", google::SetCommandLineOptionWithMode("redact",
+      "all", google::FlagSettingMode::SET_FLAGS_DEFAULT));
+
+  master::SetMasterFlagDefaults();
+  return master::RunMasterServer();
+}
+
 Status MasterSetFlag(const RunnerContext& context) {
   const string& address = FindOrDie(context.required_args, kMasterAddressArg);
   const string& flag = FindOrDie(context.required_args, kFlagArg);
@@ -343,6 +357,30 @@ unique_ptr<Mode> BuildMasterMode() {
     builder.AddAction(std::move(get_flags));
   }
   {
+    unique_ptr<Action> run =
+        ActionBuilder("run", &MasterRun)
+        .ProgramName("kudu-master")
+        .Description("Runs a Kudu Master")
+        .ExtraDescription("Note: The master server is started in this process and "
+                          "runs until interrupted.\n\n"
+                          "The most common configuration flags are described below. "
+                          "For all the configuration options pass --helpfull or see "
+                          "https://kudu.apache.org/docs/configuration_reference.html"
+                          "#kudu-master_supported")
+        .AddOptionalParameter("master_addresses")
+        // Even though fs_wal_dir is required, we don't want it to be positional argument.
+        // This allows it to be passed as a standard flag.
+        .AddOptionalParameter("fs_wal_dir")
+        .AddOptionalParameter("fs_data_dirs")
+        .AddOptionalParameter("fs_metadata_dir")
+        .AddOptionalParameter("log_dir")
+        // Unlike most tools we don't log to stderr by default to match the
+        // kudu-master binary as closely as possible.
+        .AddOptionalParameter("logtostderr", string("false"))
+        .Build();
+    builder.AddAction(std::move(run));
+  }
+  {
     unique_ptr<Action> set_flag =
         ActionBuilder("set_flag", &MasterSetFlag)
         .Description("Change a gflag value on a Kudu Master")
diff --git a/src/kudu/tools/tool_action_tserver.cc b/src/kudu/tools/tool_action_tserver.cc
index 337272a..a89a566 100644
--- a/src/kudu/tools/tool_action_tserver.cc
+++ b/src/kudu/tools/tool_action_tserver.cc
@@ -24,7 +24,7 @@
 
 #include <boost/algorithm/string/predicate.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
 
 #include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
@@ -38,6 +38,8 @@
 #include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/tserver/tablet_server.h"
+#include "kudu/tserver/tablet_server_runner.h"
+#include "kudu/util/init.h"
 #include "kudu/util/status.h"
 
 DEFINE_bool(allow_missing_tserver, false, "If true, performs the action on the "
@@ -59,6 +61,7 @@ using master::ListTabletServersRequestPB;
 using master::ListTabletServersResponsePB;
 using master::MasterServiceProxy;
 using master::TServerStateChangePB;
+using tserver::TabletServer;
 
 namespace tools {
 namespace {
@@ -77,6 +80,18 @@ Status TServerGetFlags(const RunnerContext& context) {
   return PrintServerFlags(address, tserver::TabletServer::kDefaultPort);
 }
 
+Status TServerRun(const RunnerContext& context) {
+  RETURN_NOT_OK(InitKudu());
+
+  // Enable redaction by default. Unlike most tools, we don't want user data
+  // printed to the console/log to be shown by default.
+  CHECK_NE("", google::SetCommandLineOptionWithMode("redact",
+      "all", google::FlagSettingMode::SET_FLAGS_DEFAULT));
+
+  tserver::SetTabletServerFlagDefaults();
+  return tserver::RunTabletServer();
+}
+
 Status TServerSetFlag(const RunnerContext& context) {
   const string& address = FindOrDie(context.required_args, kTServerAddressArg);
   const string& flag = FindOrDie(context.required_args, kFlagArg);
@@ -229,6 +244,29 @@ unique_ptr<Mode> BuildTServerMode() {
       .AddOptionalParameter("flag_tags")
       .Build();
 
+  unique_ptr<Action> run =
+      ActionBuilder("run", &TServerRun)
+      .ProgramName("kudu-tserver")
+      .Description("Runs a Kudu Tablet Server")
+      .ExtraDescription("Note: The tablet server is started in this process and "
+                        "runs until interrupted.\n\n"
+                        "The most common configuration flags are described below. "
+                        "For all the configuration options pass --helpfull or see "
+                        "https://kudu.apache.org/docs/configuration_reference.html"
+                        "#kudu-tserver_supported")
+      .AddOptionalParameter("tserver_master_addrs")
+      // Even though fs_wal_dir is required, we don't want it to be positional argument.
+      .AddOptionalParameter("fs_wal_dir")
+      .AddOptionalParameter("fs_data_dirs")
+      .AddOptionalParameter("fs_metadata_dir")
+      .AddOptionalParameter("block_cache_capacity_mb")
+      .AddOptionalParameter("memory_limit_hard_bytes")
+      .AddOptionalParameter("log_dir")
+      // Unlike most tools we don't log to stderr by default to match the
+      // kudu-tserver binary as closely as possible.
+      .AddOptionalParameter("logtostderr", string("false"))
+      .Build();
+
   unique_ptr<Action> set_flag =
       ActionBuilder("set_flag", &TServerSetFlag)
       .Description("Change a gflag value on a Kudu Tablet Server")
@@ -291,6 +329,7 @@ unique_ptr<Mode> BuildTServerMode() {
       .Description("Operate on a Kudu Tablet Server")
       .AddAction(std::move(dump_memtrackers))
       .AddAction(std::move(get_flags))
+      .AddAction(std::move(run))
       .AddAction(std::move(set_flag))
       .AddAction(std::move(status))
       .AddAction(std::move(timestamp))
diff --git a/src/kudu/tools/tool_main.cc b/src/kudu/tools/tool_main.cc
index 55c5f26..edd190c 100644
--- a/src/kudu/tools/tool_main.cc
+++ b/src/kudu/tools/tool_main.cc
@@ -26,7 +26,6 @@
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/gutil/map-util.h"
@@ -34,7 +33,6 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tools/tool_action.h"
 #include "kudu/util/flags.h"
-#include "kudu/util/logging.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/status.h"
 
@@ -252,14 +250,16 @@ int main(int argc, char** argv) {
   CHECK_NE("",  google::SetCommandLineOptionWithMode(
       "redact", "", google::SET_FLAGS_DEFAULT));
 
+  // Set logtostderr by default given these are command line tools.
+  CHECK_NE("",  google::SetCommandLineOptionWithMode(
+      "logtostderr", "true", google::SET_FLAGS_DEFAULT));
+
   // Hide the regular gflags help unless --helpfull is used.
   //
   // Inspired by https://github.com/gflags/gflags/issues/43#issuecomment-168280647.
   gflags::ParseCommandLineNonHelpFlags(&argc, &argv, true);
 
-  FLAGS_logtostderr = true;
   const char* prog_name = argv[0];
-  kudu::InitGoogleLoggingSafe(prog_name);
   bool show_help = ParseCommandLineFlags(prog_name);
 
   return kudu::tools::RunTool(argc, argv, show_help);
diff --git a/src/kudu/tserver/CMakeLists.txt b/src/kudu/tserver/CMakeLists.txt
index bcee01a..ae74010 100644
--- a/src/kudu/tserver/CMakeLists.txt
+++ b/src/kudu/tserver/CMakeLists.txt
@@ -112,6 +112,7 @@ set(TSERVER_SRCS
   tablet_copy_source_session.cc
   tablet_server.cc
   tablet_server_options.cc
+  tablet_server_runner.cc
   tablet_service.cc
   ts_tablet_manager.cc
   tserver_path_handlers.cc
diff --git a/src/kudu/tserver/tablet_server.h b/src/kudu/tserver/tablet_server.h
index 5d802c8..bce7773 100644
--- a/src/kudu/tserver/tablet_server.h
+++ b/src/kudu/tserver/tablet_server.h
@@ -45,6 +45,7 @@ class TabletServer : public kserver::KuduServer {
   // this constant as well.
   static const uint16_t kDefaultPort = 7050;
   static const uint16_t kDefaultWebPort = 8050;
+  static const uint16_t kDefaultNumServiceThreads = 20;
 
   explicit TabletServer(const TabletServerOptions& opts);
   ~TabletServer();
diff --git a/src/kudu/tserver/tablet_server_main.cc b/src/kudu/tserver/tablet_server_main.cc
index c01770a..87de630 100644
--- a/src/kudu/tserver/tablet_server_main.cc
+++ b/src/kudu/tserver/tablet_server_main.cc
@@ -16,77 +16,28 @@
 // under the License.
 
 #include <iostream>
-#include <string>
 
-#include <gflags/gflags.h>
 #include <glog/logging.h>
 
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/strings/substitute.h"
-#include "kudu/tserver/tablet_server.h"
-#include "kudu/util/fault_injection.h"
-#include "kudu/util/flag_tags.h"
+#include "kudu/tserver/tablet_server_runner.h"
 #include "kudu/util/flags.h"
 #include "kudu/util/init.h"
 #include "kudu/util/logging.h"
-#include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
-#include "kudu/util/version_info.h"
-
-using kudu::tserver::TabletServer;
-
-DECLARE_int32(rpc_num_service_threads);
-DECLARE_int32(webserver_port);
-DECLARE_string(rpc_bind_addresses);
-
-DEFINE_double(fault_before_start, 0.0,
-              "Fake fault flag that always causes a crash on startup. "
-              "Used to test the test infrastructure. Should never be set outside of tests.");
-TAG_FLAG(fault_before_start, hidden);
-TAG_FLAG(fault_before_start, unsafe);
 
 namespace kudu {
 namespace tserver {
 
 static int TabletServerMain(int argc, char** argv) {
   RETURN_MAIN_NOT_OK(InitKudu(), "InitKudu() failed", 1);
-
-  // Reset some default values before parsing gflags.
-  FLAGS_rpc_bind_addresses = strings::Substitute("0.0.0.0:$0",
-                                                 TabletServer::kDefaultPort);
-  FLAGS_rpc_num_service_threads = 20;
-  FLAGS_webserver_port = TabletServer::kDefaultWebPort;
-
-  // Setting the default value of the 'force_block_cache_capacity' flag to
-  // 'false' makes the corresponding group validator enforce proper settings
-  // for the memory limit and the cfile cache capacity.
-  CHECK_NE("", SetCommandLineOptionWithMode("force_block_cache_capacity",
-        "false", gflags::SET_FLAGS_DEFAULT));
-
-  GFlagsMap default_flags = GetFlagsMap();
-
+  SetTabletServerFlagDefaults();
   ParseCommandLineFlags(&argc, &argv, true);
   if (argc != 1) {
     std::cerr << "usage: " << argv[0] << std::endl;
     return 2;
   }
-  std::string nondefault_flags = GetNonDefaultFlags(default_flags);
   InitGoogleLoggingSafe(argv[0]);
-
-  LOG(INFO) << "Tablet server non-default flags:\n"
-            << nondefault_flags << '\n'
-            << "Tablet server version:\n"
-            << VersionInfo::GetAllVersionInfo();
-
-  TabletServer server({});
-  RETURN_MAIN_NOT_OK(server.Init(), "Init() failed", 3);
-  MAYBE_FAULT(FLAGS_fault_before_start);
-  RETURN_MAIN_NOT_OK(server.Start(), "Start() failed", 4);
-
-  while (true) {
-    SleepFor(MonoDelta::FromSeconds(60));
-  }
-
+  RETURN_MAIN_NOT_OK(RunTabletServer(), "RunTabletServer() failed", 3);
   return 0;
 }
 
diff --git a/src/kudu/tserver/tablet_server_main.cc b/src/kudu/tserver/tablet_server_runner.cc
similarity index 67%
copy from src/kudu/tserver/tablet_server_main.cc
copy to src/kudu/tserver/tablet_server_runner.cc
index c01770a..feb7b80 100644
--- a/src/kudu/tserver/tablet_server_main.cc
+++ b/src/kudu/tserver/tablet_server_runner.cc
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/tserver/tablet_server_runner.h"
+
 #include <iostream>
 #include <string>
 
@@ -27,17 +29,11 @@
 #include "kudu/util/fault_injection.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/flags.h"
-#include "kudu/util/init.h"
-#include "kudu/util/logging.h"
 #include "kudu/util/monotime.h"
-#include "kudu/util/status.h"
 #include "kudu/util/version_info.h"
 
 using kudu::tserver::TabletServer;
-
-DECLARE_int32(rpc_num_service_threads);
-DECLARE_int32(webserver_port);
-DECLARE_string(rpc_bind_addresses);
+using std::string;
 
 DEFINE_double(fault_before_start, 0.0,
               "Fake fault flag that always causes a crash on startup. "
@@ -48,51 +44,41 @@ TAG_FLAG(fault_before_start, unsafe);
 namespace kudu {
 namespace tserver {
 
-static int TabletServerMain(int argc, char** argv) {
-  RETURN_MAIN_NOT_OK(InitKudu(), "InitKudu() failed", 1);
-
+void SetTabletServerFlagDefaults() {
   // Reset some default values before parsing gflags.
-  FLAGS_rpc_bind_addresses = strings::Substitute("0.0.0.0:$0",
-                                                 TabletServer::kDefaultPort);
-  FLAGS_rpc_num_service_threads = 20;
-  FLAGS_webserver_port = TabletServer::kDefaultWebPort;
+  CHECK_NE("", google::SetCommandLineOptionWithMode("rpc_bind_addresses",
+      strings::Substitute("0.0.0.0:$0", TabletServer::kDefaultPort).c_str(),
+      google::FlagSettingMode::SET_FLAGS_DEFAULT));
+  CHECK_NE("", google::SetCommandLineOptionWithMode("rpc_num_service_threads",
+      std::to_string(TabletServer::kDefaultNumServiceThreads).c_str(),
+      google::FlagSettingMode::SET_FLAGS_DEFAULT));
+  CHECK_NE("", google::SetCommandLineOptionWithMode("webserver_port",
+      std::to_string(TabletServer::kDefaultWebPort).c_str(),
+      google::FlagSettingMode::SET_FLAGS_DEFAULT));
 
   // Setting the default value of the 'force_block_cache_capacity' flag to
   // 'false' makes the corresponding group validator enforce proper settings
   // for the memory limit and the cfile cache capacity.
   CHECK_NE("", SetCommandLineOptionWithMode("force_block_cache_capacity",
-        "false", gflags::SET_FLAGS_DEFAULT));
-
-  GFlagsMap default_flags = GetFlagsMap();
-
-  ParseCommandLineFlags(&argc, &argv, true);
-  if (argc != 1) {
-    std::cerr << "usage: " << argv[0] << std::endl;
-    return 2;
-  }
-  std::string nondefault_flags = GetNonDefaultFlags(default_flags);
-  InitGoogleLoggingSafe(argv[0]);
+                                            "false", gflags::SET_FLAGS_DEFAULT));
+}
 
+Status RunTabletServer() {
+  string nondefault_flags = GetNonDefaultFlags();
   LOG(INFO) << "Tablet server non-default flags:\n"
             << nondefault_flags << '\n'
             << "Tablet server version:\n"
             << VersionInfo::GetAllVersionInfo();
 
   TabletServer server({});
-  RETURN_MAIN_NOT_OK(server.Init(), "Init() failed", 3);
+  RETURN_NOT_OK(server.Init());
   MAYBE_FAULT(FLAGS_fault_before_start);
-  RETURN_MAIN_NOT_OK(server.Start(), "Start() failed", 4);
+  RETURN_NOT_OK(server.Start());
 
   while (true) {
     SleepFor(MonoDelta::FromSeconds(60));
   }
-
-  return 0;
 }
 
 } // namespace tserver
 } // namespace kudu
-
-int main(int argc, char** argv) {
-  return kudu::tserver::TabletServerMain(argc, argv);
-}
diff --git a/src/kudu/tserver/tablet_server_runner.h b/src/kudu/tserver/tablet_server_runner.h
new file mode 100644
index 0000000..3b6264e
--- /dev/null
+++ b/src/kudu/tserver/tablet_server_runner.h
@@ -0,0 +1,31 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace tserver {
+
+// Adjusts the default values of some gflags for use with the tablet server.
+void SetTabletServerFlagDefaults();
+
+// Initialize and start a tablet server. Will run until interrupted.
+Status RunTabletServer();
+
+} // namespace tserver
+} // namespace kudu
diff --git a/src/kudu/util/flag_tags-test.cc b/src/kudu/util/flag_tags-test.cc
index 4626d0d..7b7c8a5 100644
--- a/src/kudu/util/flag_tags-test.cc
+++ b/src/kudu/util/flag_tags-test.cc
@@ -15,18 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/util/flag_tags.h"
+
 #include <string>
 #include <unordered_set>
 #include <vector>
 
-#include <gtest/gtest.h>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
+#include <gtest/gtest.h>
 
-#include "kudu/gutil/map-util.h"
 #include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/util/flag_tags.h"
 #include "kudu/util/flags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/logging_test_util.h"
@@ -83,7 +83,7 @@ TEST_F(FlagTagsTest, TestUnlockFlags) {
   {
     gflags::FlagSaver s;
     gflags::SetCommandLineOption("test_unsafe_flag", "true");
-    ASSERT_DEATH({ HandleCommonFlags(); },
+    ASSERT_DEATH({ ValidateFlags(); },
                  "Flag --test_unsafe_flag is unsafe and unsupported.*"
                  "Use --unlock_unsafe_flags to proceed");
   }
@@ -95,7 +95,7 @@ TEST_F(FlagTagsTest, TestUnlockFlags) {
     gflags::FlagSaver s;
     gflags::SetCommandLineOption("test_unsafe_flag", "true");
     gflags::SetCommandLineOption("unlock_unsafe_flags", "true");
-    HandleCommonFlags();
+    ValidateFlags();
     ASSERT_EQ(1, sink.logged_msgs().size());
     ASSERT_STR_CONTAINS(sink.logged_msgs()[0], "Enabled unsafe flag: --test_unsafe_flag");
   }
@@ -104,7 +104,7 @@ TEST_F(FlagTagsTest, TestUnlockFlags) {
   {
     gflags::FlagSaver s;
     gflags::SetCommandLineOption("test_experimental_flag", "true");
-    ASSERT_DEATH({ HandleCommonFlags(); },
+    ASSERT_DEATH({ ValidateFlags(); },
                  "Flag --test_experimental_flag is experimental and unsupported.*"
                  "Use --unlock_experimental_flags to proceed");
   }
@@ -116,7 +116,7 @@ TEST_F(FlagTagsTest, TestUnlockFlags) {
     gflags::FlagSaver s;
     gflags::SetCommandLineOption("test_experimental_flag", "true");
     gflags::SetCommandLineOption("unlock_experimental_flags", "true");
-    HandleCommonFlags();
+    ValidateFlags();
     ASSERT_EQ(1, sink.logged_msgs().size());
     ASSERT_STR_CONTAINS(sink.logged_msgs()[0],
                         "Enabled experimental flag: --test_experimental_flag");
diff --git a/src/kudu/util/flags-test.cc b/src/kudu/util/flags-test.cc
index 9ebc178..675ef6f 100644
--- a/src/kudu/util/flags-test.cc
+++ b/src/kudu/util/flags-test.cc
@@ -15,18 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/util/flags.h"
+
 #include <string>
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/env.h"
-#include "kudu/util/flags.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/slice.h"
@@ -53,9 +53,6 @@ namespace kudu {
 class FlagsTest : public KuduTest {};
 
 TEST_F(FlagsTest, TestNonDefaultFlags) {
-  // Memorize the default flags
-  GFlagsMap default_flags = GetFlagsMap();
-
   std::string flagfile_path(GetTestPath("test_nondefault_flags"));
   std::string flagfile_contents = "--test_nondefault_ff=nondefault\n"
                                   "--test_default_ff=default";
@@ -92,7 +89,7 @@ TEST_F(FlagsTest, TestNonDefaultFlags) {
   // a redacted value.
   FLAGS_test_sensitive_flag = true;
   kudu::g_should_redact = kudu::RedactContext::LOG;
-  std::string result = GetNonDefaultFlags(default_flags);
+  std::string result = GetNonDefaultFlags();
 
   for (const auto& expected : expected_flags) {
     ASSERT_STR_CONTAINS(result, expected);
diff --git a/src/kudu/util/flags.cc b/src/kudu/util/flags.cc
index b966a44..b4eaf7e 100644
--- a/src/kudu/util/flags.cc
+++ b/src/kudu/util/flags.cc
@@ -17,21 +17,20 @@
 
 #include "kudu/util/flags.h"
 
+#include <sys/stat.h>
+#include <unistd.h> // IWYU pragma: keep
 
 #include <cstdlib>
 #include <functional>
 #include <iostream>
+#include <map>
 #include <string>
 #include <unordered_set>
 #include <utility>
 #include <vector>
 
-#include <sys/stat.h>
-#include <unistd.h> // IWYU pragma: keep
-
 #include <boost/algorithm/string/predicate.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #ifdef TCMALLOC_ENABLED
 #include <gperftools/heap-profiler.h>
@@ -481,6 +480,7 @@ int ParseCommandLineFlags(int* argc, char*** argv, bool remove_flags) {
 
   int ret = google::ParseCommandLineNonHelpFlags(argc, argv, remove_flags);
   HandleCommonFlags();
+  ValidateFlags();
   return ret;
 }
 
@@ -497,8 +497,6 @@ void HandleCommonFlags() {
   }
 
   google::HandleCommandLineHelpFlags();
-  CheckFlagsAllowed();
-  RunCustomValidators();
 
   if (FLAGS_disable_core_dumps) {
     DisableCoreDumps();
@@ -527,6 +525,11 @@ void HandleCommonFlags() {
 #endif
 }
 
+void ValidateFlags() {
+  CheckFlagsAllowed();
+  RunCustomValidators();
+}
+
 string CommandlineFlagsIntoString(EscapeMode mode) {
   string ret_value;
   vector<CommandLineFlagInfo> flags;
@@ -546,20 +549,15 @@ string CommandlineFlagsIntoString(EscapeMode mode) {
   return ret_value;
 }
 
-string GetNonDefaultFlags(const GFlagsMap& default_flags) {
+string GetNonDefaultFlags() {
   ostringstream args;
   vector<CommandLineFlagInfo> flags;
   GetAllFlags(&flags);
   for (const auto& flag : flags) {
     if (!flag.is_default) {
-      // This only means that the flag has been rewritten. It doesn't
-      // mean that this has been done in the command line, or even
-      // that it's truly different from the default value.
-      // Next, we try to check both.
-      const auto& default_flag = default_flags.find(flag.name);
-      // it's very unlikely, but still possible that we don't have the flag in defaults
-      if (default_flag == default_flags.end() ||
-          flag.current_value != default_flag->second.current_value) {
+      // This only means that the flag has been rewritten.
+      // We need to check that the value is different from the default value.
+      if (flag.current_value != flag.default_value) {
         if (!args.str().empty()) {
           args << '\n';
         }
diff --git a/src/kudu/util/flags.h b/src/kudu/util/flags.h
index 83cb152..033a57f 100644
--- a/src/kudu/util/flags.h
+++ b/src/kudu/util/flags.h
@@ -54,6 +54,11 @@ int ParseCommandLineFlags(int* argc, char*** argv, bool remove_flags);
 // google::ParseCommandLineNonHelpFlags().
 void HandleCommonFlags();
 
+// Verifies that the flags are allowed to be set and valid.
+// Should be called after logging is initialized. Otherwise
+// logging will write to stderr.
+void ValidateFlags();
+
 enum class EscapeMode {
   HTML,
   NONE
@@ -70,7 +75,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 redaction is enabled.
-std::string GetNonDefaultFlags(const GFlagsMap& default_flags);
+std::string GetNonDefaultFlags();
 
 GFlagsMap GetFlagsMap();
 
diff --git a/src/kudu/util/logging.cc b/src/kudu/util/logging.cc
index cfda466..36850c3 100644
--- a/src/kudu/util/logging.cc
+++ b/src/kudu/util/logging.cc
@@ -14,6 +14,7 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+
 #include "kudu/util/logging.h"
 
 #include <unistd.h>
@@ -23,10 +24,12 @@
 #include <cstdlib>
 #include <ctime>
 #include <fstream>
+#include <initializer_list>
 #include <mutex>
 #include <utility>
 
 #include <boost/uuid/random_generator.hpp>
+#include <boost/uuid/uuid.hpp>
 #include <boost/uuid/uuid_io.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
@@ -217,13 +220,12 @@ void InitGoogleLoggingSafe(const char* arg) {
     }
   }
 
-  // This forces our logging to use /tmp rather than looking for a
+  // This forces our logging to default to /tmp rather than looking for a
   // temporary directory if none is specified. This is done so that we
   // can reliably construct the log file name without duplicating the
   // complex logic that glog uses to guess at a temporary dir.
-  if (FLAGS_log_dir.empty()) {
-    FLAGS_log_dir = "/tmp";
-  }
+  CHECK_NE("", google::SetCommandLineOptionWithMode("log_dir",
+     "/tmp", google::FlagSettingMode::SET_FLAGS_DEFAULT));
 
   if (!FLAGS_logtostderr) {
     // Verify that a log file can be created in log_dir by creating a tmp file.
diff --git a/src/kudu/util/logging.h b/src/kudu/util/logging.h
index 0d3492a..f8b03b5 100644
--- a/src/kudu/util/logging.h
+++ b/src/kudu/util/logging.h
@@ -25,6 +25,7 @@
 #include "kudu/gutil/atomicops.h"
 #include "kudu/gutil/dynamic_annotations.h"
 #include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/logging_callback.h"
 #include "kudu/util/status.h"
diff --git a/src/kudu/util/minidump.cc b/src/kudu/util/minidump.cc
index d02dc5d..9a301f4 100644
--- a/src/kudu/util/minidump.cc
+++ b/src/kudu/util/minidump.cc
@@ -57,6 +57,7 @@ static constexpr bool kMinidumpPlatformSupported = false;
 #endif // defined(__linux__)
 
 DECLARE_string(log_dir);
+DECLARE_string(log_filename);
 
 DEFINE_bool(enable_minidumps, kMinidumpPlatformSupported,
             "Whether to enable minidump generation upon process crash or SIGUSR1. "
@@ -208,11 +209,14 @@ Status MinidumpExceptionHandler::InitMinidumpExceptionHandler() {
   RETURN_NOT_OK_PREPEND(CreateDirIfMissing(env, minidump_dir_),
                         "Error creating top-level minidump directory");
 
-  // Add the executable name to the path where minidumps will be written. This makes
-  // identification easier and prevents name collisions between the files.
+  // Add the program_name to the path where minidumps will be written.
+  // This makes identification easier and prevents name collisions between the files.
   // This is also consistent with how Impala organizes its minidump files.
-  const char* exe_name = gflags::ProgramInvocationShortName();
-  minidump_dir_ = JoinPathSegments(minidump_dir_, exe_name);
+  // The log_filename flag will be used if non-empty, otherwise the executable name
+  // will be used.
+  const char* program_name = FLAGS_log_filename.empty() ? gflags::ProgramInvocationShortName() :
+          FLAGS_log_filename.c_str();
+  minidump_dir_ = JoinPathSegments(minidump_dir_, program_name);
 
   // Create the directory if it is not there. The minidump doesn't get written if there is
   // no directory.
diff --git a/src/kudu/util/rolling_log-test.cc b/src/kudu/util/rolling_log-test.cc
index f4f8186..9327ac7 100644
--- a/src/kudu/util/rolling_log-test.cc
+++ b/src/kudu/util/rolling_log-test.cc
@@ -78,7 +78,7 @@ class RollingLogTest : public KuduTest {
 
 // Test with compression off.
 TEST_F(RollingLogTest, TestLog) {
-  RollingLog log(env_, log_dir_, "mylog");
+  RollingLog log(env_, log_dir_, "rolling_log-test", "mylog");
   log.SetCompressionEnabled(false);
   log.SetRollThresholdBytes(100);
 
@@ -107,7 +107,7 @@ TEST_F(RollingLogTest, TestLog) {
 
 // Test with compression on.
 TEST_F(RollingLogTest, TestCompression) {
-  RollingLog log(env_, log_dir_, "mylog");
+  RollingLog log(env_, log_dir_, "rolling_log-test", "mylog");
   ASSERT_OK(log.Open());
 
   StringPiece data = "Hello world\n";
@@ -130,7 +130,7 @@ TEST_F(RollingLogTest, TestCompression) {
 }
 
 TEST_F(RollingLogTest, TestFileCountLimit) {
-  RollingLog log(env_, log_dir_, "mylog");
+  RollingLog log(env_, log_dir_, "rolling_log-test", "mylog");
   ASSERT_OK(log.Open());
   log.SetRollThresholdBytes(100);
   log.SetMaxNumSegments(3);
diff --git a/src/kudu/util/rolling_log.cc b/src/kudu/util/rolling_log.cc
index 610aa9e..9f35e0b 100644
--- a/src/kudu/util/rolling_log.cc
+++ b/src/kudu/util/rolling_log.cc
@@ -24,10 +24,10 @@
 #include <memory>
 #include <ostream>
 #include <string>
+#include <type_traits>
 #include <utility>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <zlib.h>
 
@@ -53,9 +53,11 @@ DECLARE_int32(max_log_files);
 
 namespace kudu {
 
-RollingLog::RollingLog(Env* env, string log_dir, string log_name)
+RollingLog::RollingLog(Env* env, string log_dir, string program_name, string log_name)
     : env_(env),
       log_dir_(std::move(log_dir)),
+      program_name_(program_name.empty() ?
+        google::ProgramInvocationShortName() : std::move(program_name)),
       log_name_(std::move(log_name)),
       roll_threshold_bytes_(kDefaultRollThresholdBytes),
       max_num_segments_(FLAGS_max_log_files),
@@ -121,7 +123,7 @@ string FormattedTimestamp() {
 
 string RollingLog::GetLogFileName(int sequence) const {
   return Substitute("$0.$1.$2.$3.$4.$5.$6",
-                    google::ProgramInvocationShortName(),
+                    program_name_,
                     HostnameOrUnknown(),
                     UsernameOrUnknown(),
                     log_name_,
@@ -132,7 +134,7 @@ string RollingLog::GetLogFileName(int sequence) const {
 
 string RollingLog::GetLogFilePattern() const {
   return Substitute("$0.$1.$2.$3.$4.$5.$6",
-                    google::ProgramInvocationShortName(),
+                    program_name_,
                     HostnameOrUnknown(),
                     UsernameOrUnknown(),
                     log_name_,
diff --git a/src/kudu/util/rolling_log.h b/src/kudu/util/rolling_log.h
index 0bb6755..17ca8e9 100644
--- a/src/kudu/util/rolling_log.h
+++ b/src/kudu/util/rolling_log.h
@@ -41,7 +41,8 @@ class WritableFile;
 //
 // <log_dir>/<program-name>.<hostname>.<user-name>.<log-name>.<timestamp>.<sequence>.<pid>
 //   log_dir:      the log_dir specified in the constructor
-//   program-name: argv[0], as determined by google::ProgramInvocationShortName()
+//   program-name: the name of the program specified in the constructor. If unset, defaults
+//                 to argv[0], as determined by google::ProgramInvocationShortName().
 //   hostname:     the local machine hostname
 //   user-name:    the current user name
 //   log-name:     the log_name specified in the constructor
@@ -55,7 +56,7 @@ class WritableFile;
 // This class is not thread-safe and must be externally synchronized.
 class RollingLog {
  public:
-  RollingLog(Env* env, std::string log_dir, std::string log_name);
+  RollingLog(Env* env, std::string log_dir, std::string program_name, std::string log_name);
 
   ~RollingLog();
 
@@ -111,6 +112,7 @@ class RollingLog {
 
   Env* const env_;
   const std::string log_dir_;
+  const std::string program_name_;
   const std::string log_name_;
 
   int64_t roll_threshold_bytes_;