You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2020/11/30 18:16:47 UTC

[kudu] branch master updated: [CLI] introduce connection negotiation timeout

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 802057f  [CLI] introduce connection negotiation timeout
802057f is described below

commit 802057f3f243a6fa92c90b4da4c229307761762f
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Sun Nov 22 20:10:51 2020 -0800

    [CLI] introduce connection negotiation timeout
    
    Since the connection negotiation timeout for Kudu client is now
    configurable (see KUDU-2966 and changelist 62ec1e474), it makes sense
    to allow using the new functionality in the kudu CLI tools.
    
    This changelist introduces a new command line option/flag
    --negotiation_timeout_ms to control the client-side connection
    negotiation timeout in the kudu CLI tools.  The default value for the
    new flag is set to 3000 milliseconds to match the default connection
    negotiation timeout on the server.  With larger and heavy loaded
    clusters it might be a good idea to the negotiation timeout if
    connection negotiation timeouts happen time to time.  Also, it could
    help in cases where the RPC authentication is required and KDC is not
    fast enough.  Keep in mind that the total RPC timeout includes the
    connection negotiation time, so in general it makes sense to bump
    --timeout_ms along with --negotiation_timeout_ms by the same delta.
    
    In addition, with this changeliest, all RPC-related kudu CLI tools
    except for `kudu local_replica copy_from_remote` now have command line
    arguments to control the RPC and the connection negotiation timeouts:
      * timeout_ms
      * negotiation_timeout_ms
    
    In addition, this patch also unifies the way how KuduClient instances
    are created in the kudu CLI code.
    
    I'm planning to take care of the RPC-related parameters of the
    `kudu local_replica copy_from_remote` tool in a separate patch.
    
    Change-Id: I5627517fd1da4f4d7472f1f0af810d741efe3390
    Reviewed-on: http://gerrit.cloudera.org:8080/16770
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 src/kudu/integration-tests/replace_tablet-itest.cc |   7 +-
 src/kudu/server/server_base.cc                     |   2 +-
 src/kudu/tools/ksck_checksum.cc                    |   5 +-
 src/kudu/tools/ksck_remote.cc                      |  22 ++-
 src/kudu/tools/kudu-tool-test.cc                   | 159 +++++++++++++++++++--
 src/kudu/tools/rebalancer_tool.cc                  |   8 +-
 src/kudu/tools/rebalancer_tool.h                   |   2 +-
 src/kudu/tools/tool_action.h                       |   5 +-
 src/kudu/tools/tool_action_cluster.cc              |   8 +-
 src/kudu/tools/tool_action_common.cc               | 119 +++++++++++++--
 src/kudu/tools/tool_action_common.h                |  61 +++++++-
 src/kudu/tools/tool_action_hms.cc                  | 128 ++++++++---------
 src/kudu/tools/tool_action_local_replica.cc        |  68 ++++-----
 src/kudu/tools/tool_action_master.cc               |  42 ++----
 src/kudu/tools/tool_action_perf.cc                 |  97 +++++++------
 src/kudu/tools/tool_action_remote_replica.cc       |  21 +--
 src/kudu/tools/tool_action_table.cc                |  79 ++++------
 src/kudu/tools/tool_action_tablet.cc               |  29 ++--
 src/kudu/tools/tool_action_tserver.cc              |  39 ++---
 src/kudu/tools/tool_replica_util.cc                |   9 +-
 20 files changed, 545 insertions(+), 365 deletions(-)

diff --git a/src/kudu/integration-tests/replace_tablet-itest.cc b/src/kudu/integration-tests/replace_tablet-itest.cc
index 8f8ad0e..3f86607 100644
--- a/src/kudu/integration-tests/replace_tablet-itest.cc
+++ b/src/kudu/integration-tests/replace_tablet-itest.cc
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <functional>
 #include <memory>
 #include <ostream>
 #include <string>
@@ -32,6 +33,7 @@
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
+#include "kudu/rpc/response_callback.h"
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/tserver/tserver.pb.h"
@@ -101,7 +103,10 @@ TEST_F(ReplaceTabletITest, DISABLED_ReplaceTabletsWhileWriting) {
   for (const auto& hp : cluster_->master_rpc_addrs()) {
     master_addrs.emplace_back(hp.ToString());
   }
-  ASSERT_OK(proxy.Init(master_addrs, MonoDelta::FromSeconds(10)));
+  const auto timeout = MonoDelta::FromSeconds(10);
+  ASSERT_OK(proxy.Init(master_addrs,
+                       timeout /* rpc_timeout */,
+                       timeout /* connection_negotiation_timeout */));
 
   TestWorkload workload(cluster_.get());
   workload.set_num_replicas(kNumTabletServers);
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 3ccede8..9dd988f 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -101,7 +101,7 @@ DEFINE_int32(max_negotiation_threads, 50, "Maximum number of connection negotiat
 TAG_FLAG(max_negotiation_threads, advanced);
 
 DEFINE_int64(rpc_negotiation_timeout_ms, 3000,
-             "Timeout for negotiating an RPC connection.");
+             "Timeout for negotiating an RPC connection, in milliseconds");
 TAG_FLAG(rpc_negotiation_timeout_ms, advanced);
 
 DEFINE_bool(webserver_enabled, true, "Whether to enable the web server on this daemon. "
diff --git a/src/kudu/tools/ksck_checksum.cc b/src/kudu/tools/ksck_checksum.cc
index 20df7aa..69272b2 100644
--- a/src/kudu/tools/ksck_checksum.cc
+++ b/src/kudu/tools/ksck_checksum.cc
@@ -37,9 +37,9 @@
 #include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/rpc/messenger.h"
 #include "kudu/rpc/periodic.h"
 #include "kudu/tools/ksck.h"
+#include "kudu/tools/tool_action_common.h"
 #include "kudu/util/fault_injection.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/scoped_cleanup.h"
@@ -359,8 +359,7 @@ bool KsckChecksumManager::ReserveSlotsToChecksumUnlocked(
 
 Status KsckChecksumManager::RunChecksumsAsync() {
   if (!messenger_) {
-    rpc::MessengerBuilder builder("timestamp update");
-    RETURN_NOT_OK(builder.Build(&messenger_));
+    RETURN_NOT_OK(BuildMessenger("timestamp update", &messenger_));
   }
   timestamp_update_timer_ = rpc::PeriodicTimer::Create(
       messenger_,
diff --git a/src/kudu/tools/ksck_remote.cc b/src/kudu/tools/ksck_remote.cc
index 18fe7b2..34792bc 100644
--- a/src/kudu/tools/ksck_remote.cc
+++ b/src/kudu/tools/ksck_remote.cc
@@ -47,7 +47,6 @@
 #include "kudu/master/master.pb.h"
 #include "kudu/master/sys_catalog.h"
 #include "kudu/rebalance/cluster_status.h"
-#include "kudu/rpc/messenger.h"
 #include "kudu/rpc/response_callback.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/server/server_base.pb.h"
@@ -70,8 +69,8 @@
 #include "kudu/util/threadpool.h"
 #include "kudu/util/version_info.pb.h"
 
-DECLARE_int64(timeout_ms); // defined in tool_action_common
 DECLARE_int32(fetch_info_concurrency);
+DECLARE_int64(timeout_ms); // defined in tool_action_common
 
 DEFINE_bool(checksum_cache_blocks, false, "Should the checksum scanners cache the read blocks.");
 DEFINE_bool(quiescing_info, true,
@@ -79,7 +78,6 @@ DEFINE_bool(quiescing_info, true,
             "e.g. number of tablet leaders per server, the number of active scanners "
             "per server.");
 
-using kudu::client::KuduClientBuilder;
 using kudu::client::KuduScanToken;
 using kudu::client::KuduScanTokenBuilder;
 using kudu::client::KuduSchema;
@@ -90,7 +88,6 @@ using kudu::master::ListTabletServersRequestPB;
 using kudu::master::ListTabletServersResponsePB;
 using kudu::master::TServerStatePB;
 using kudu::rpc::Messenger;
-using kudu::rpc::MessengerBuilder;
 using kudu::rpc::RpcController;
 using kudu::server::GenericServiceProxy;
 using kudu::server::GetFlagsRequestPB;
@@ -108,6 +105,7 @@ namespace tools {
 static const std::string kMessengerName = "ksck";
 
 namespace {
+
 MonoDelta GetDefaultTimeout() {
   return MonoDelta::FromMilliseconds(FLAGS_timeout_ms);
 }
@@ -523,25 +521,23 @@ RemoteKsckCluster::RemoteKsckCluster(std::vector<std::string> master_addresses,
 }
 
 Status RemoteKsckCluster::Connect() {
-  KuduClientBuilder builder;
-  builder.default_rpc_timeout(GetDefaultTimeout());
-  builder.master_server_addrs(master_addresses_);
-  ReplicaController::SetVisibility(&builder, ReplicaController::Visibility::ALL);
-  return builder.Build(&client_);
+  return CreateKuduClient(master_addresses_,
+                          &client_,
+                          true /* can_see_all_replicas */);
 }
 
 Status RemoteKsckCluster::Build(const vector<string>& master_addresses,
                                shared_ptr<KsckCluster>* cluster) {
   CHECK(!master_addresses.empty());
   shared_ptr<Messenger> messenger;
-  MessengerBuilder builder(kMessengerName);
-  RETURN_NOT_OK(builder.Build(&messenger));
-  auto* cl = new RemoteKsckCluster(master_addresses, messenger);
+  RETURN_NOT_OK(BuildMessenger(kMessengerName, &messenger));
+  shared_ptr<RemoteKsckCluster> cl(new RemoteKsckCluster(
+      master_addresses, messenger));
   for (const auto& master : cl->masters()) {
     RETURN_NOT_OK_PREPEND(master->Init(),
                           Substitute("unable to initialize master at $0", master->address()));
   }
-  cluster->reset(cl);
+  *cluster = std::move(cl);
   return Status::OK();
 }
 
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 31f25ef..7c9aa04 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -141,6 +141,8 @@ DECLARE_bool(fs_data_dirs_consider_available_space);
 DECLARE_bool(hive_metastore_sasl_enabled);
 DECLARE_bool(show_values);
 DECLARE_bool(show_attributes);
+DECLARE_int32(catalog_manager_inject_latency_load_ca_info_ms);
+DECLARE_int32(rpc_negotiation_inject_delay_ms);
 DECLARE_string(block_manager);
 DECLARE_string(hive_metastore_uris);
 
@@ -343,6 +345,23 @@ class ToolTest : public KuduTest {
     }
   }
 
+  // Run each of the provided 'arg_str_subcommands' for the specified 'arg_str'
+  // command with '--help' command-line flag and make sure the result usage
+  // message contains information on command line flags to control RPC and
+  // connection negotiation timeouts. This targets various CLI tools issuing
+  // RPCs to a Kudu cluster (either to masters or tablet servers).
+  void RunTestHelpRpcFlags(const string& arg_str,
+                           const vector<string>& arg_str_subcommands) const {
+    static const vector<string> kTimeoutRegexes = {
+      "-negotiation_timeout_ms \\(Timeout for negotiating an RPC connection",
+      "-timeout_ms \\(RPC timeout in milliseconds\\)",
+    };
+    for (auto& cmd_str : arg_str_subcommands) {
+      const string arg = arg_str + " " + cmd_str + " --help";
+      NO_FATALS(RunTestHelp(arg, kTimeoutRegexes));
+    }
+  }
+
   // Run tool without a required positional argument, expecting error message
   void RunActionMissingRequiredArg(const string& arg_str, const string& required_arg,
                                    bool variadic = false) const {
@@ -984,11 +1003,13 @@ TEST_F(ToolTest, TestTopLevelHelp) {
 
 TEST_F(ToolTest, TestModeHelp) {
   {
+    const string kCmd = "cluster";
     const vector<string> kClusterModeRegexes = {
         "ksck.*Check the health of a Kudu cluster",
         "rebalance.*Move tablet replicas between tablet servers",
     };
-    NO_FATALS(RunTestHelp("cluster", kClusterModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kClusterModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd, {"ksck", "rebalance"}));
   }
   {
     const vector<string> kDiagnoseModeRegexes = {
@@ -1019,6 +1040,7 @@ TEST_F(ToolTest, TestModeHelp) {
 
   }
   {
+    const string kCmd = "hms";
     const vector<string> kHmsModeRegexes = {
         "check.*Check metadata consistency",
         "downgrade.*Downgrade the metadata",
@@ -1026,9 +1048,11 @@ TEST_F(ToolTest, TestModeHelp) {
         "list.*List the Kudu table HMS entries",
         "precheck.*Check that the Kudu cluster is prepared",
     };
-    NO_FATALS(RunTestHelp("hms", kHmsModeRegexes));
-    NO_FATALS(RunTestHelp("hms not_a_mode", kHmsModeRegexes,
+    NO_FATALS(RunTestHelp(kCmd, kHmsModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd + " not_a_mode", kHmsModeRegexes,
                           Status::InvalidArgument("unknown command 'not_a_mode'")));
+    NO_FATALS(RunTestHelpRpcFlags(
+        kCmd, {"check", "downgrade", "fix", "list", "precheck"}));
   }
   {
     const vector<string> kLocalReplicaModeRegexes = {
@@ -1072,6 +1096,7 @@ TEST_F(ToolTest, TestModeHelp) {
                           kLocalReplicaCopyFromRemoteRegexes));
   }
   {
+    const string kCmd = "master";
     const vector<string> kMasterModeRegexes = {
         "authz_cache.*Operate on the authz caches of the Kudu Masters",
         "dump_memtrackers.*Dump the memtrackers",
@@ -1082,13 +1107,23 @@ TEST_F(ToolTest, TestModeHelp) {
         "timestamp.*Get the current timestamp",
         "list.*List masters in a Kudu cluster",
     };
-    NO_FATALS(RunTestHelp("master", kMasterModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kMasterModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd,
+        { "dump_memtrackers",
+          "get_flags",
+          "set_flag",
+          "status",
+          "timestamp",
+          "list",
+        }));
   }
   {
+    const string kSubCmd = "master authz_cache";
     const vector<string> kMasterAuthzCacheModeRegexes = {
         "refresh.*Refresh the authorization policies",
     };
-    NO_FATALS(RunTestHelp("master authz_cache", kMasterAuthzCacheModeRegexes));
+    NO_FATALS(RunTestHelp(kSubCmd, kMasterAuthzCacheModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kSubCmd, {"refresh"}));
   }
   {
     const vector<string> kPbcModeRegexes = {
@@ -1098,14 +1133,17 @@ TEST_F(ToolTest, TestModeHelp) {
     NO_FATALS(RunTestHelp("pbc", kPbcModeRegexes));
   }
   {
+    const string kCmd = "perf";
     const vector<string> kPerfRegexes = {
         "loadgen.*Run load generation with optional scan afterwards",
         "table_scan.*Show row count and scanning time cost of tablets in a table",
         "tablet_scan.*Show row count of a local tablet",
     };
-    NO_FATALS(RunTestHelp("perf", kPerfRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kPerfRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd, {"loadgen", "table_scan"}));
   }
   {
+    const string kCmd = "remote_replica";
     const vector<string> kRemoteReplicaModeRegexes = {
         "check.*Check if all tablet replicas",
         "copy.*Copy a tablet replica from one Kudu Tablet Server",
@@ -1114,9 +1152,18 @@ TEST_F(ToolTest, TestModeHelp) {
         "list.*List all tablet replicas",
         "unsafe_change_config.*Force the specified replica to adopt",
     };
-    NO_FATALS(RunTestHelp("remote_replica", kRemoteReplicaModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kRemoteReplicaModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd,
+        { "check",
+          "copy",
+          "delete",
+          "dump",
+          "list",
+          "unsafe_change_config",
+        }));
   }
   {
+    const string kCmd = "table";
     const vector<string> kTableModeRegexes = {
         "add_range_partition.*Add a range partition for table",
         "column_remove_default.*Remove write_default value for a column",
@@ -1140,24 +1187,59 @@ TEST_F(ToolTest, TestModeHelp) {
         "set_extra_config.*Change a extra configuration value on a table",
         "statistics.*Get table statistics",
     };
-    NO_FATALS(RunTestHelp("table", kTableModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kTableModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd,
+        { "add_range_partition",
+          "column_remove_default",
+          "column_set_block_size",
+          "column_set_compression",
+          "column_set_default",
+          "column_set_encoding",
+          "column_set_comment",
+          "copy",
+          "create",
+          "delete_column",
+          "delete",
+          "describe",
+          "drop_range_partition",
+          "get_extra_configs",
+          "list",
+          "locate_row",
+          "rename_column",
+          "rename_table",
+          "scan",
+          "set_extra_config",
+          "statistics",
+        }));
   }
   {
+    const string kCmd = "tablet";
     const vector<string> kTabletModeRegexes = {
         "change_config.*Change.*Raft configuration",
         "leader_step_down.*Change.*tablet's leader",
         "unsafe_replace_tablet.*Replace a tablet with an empty one",
     };
-    NO_FATALS(RunTestHelp("tablet", kTabletModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kTabletModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd,
+        { "leader_step_down",
+          "unsafe_replace_tablet",
+        }));
   }
   {
+    const string kSubCmd = "tablet change_config";
     const vector<string> kChangeConfigModeRegexes = {
         "add_replica.*Add a new replica",
         "change_replica_type.*Change the type of an existing replica",
         "move_replica.*Move a tablet replica",
         "remove_replica.*Remove an existing replica",
     };
-    NO_FATALS(RunTestHelp("tablet change_config", kChangeConfigModeRegexes));
+    NO_FATALS(RunTestHelp(kSubCmd, kChangeConfigModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kSubCmd,
+        { "add_replica",
+          "change_replica_type",
+          "move_replica",
+          "remove_replica",
+        }));
   }
   {
     const vector<string> kTestModeRegexes = {
@@ -1166,6 +1248,7 @@ TEST_F(ToolTest, TestModeHelp) {
     NO_FATALS(RunTestHelp("test", kTestModeRegexes));
   }
   {
+    const string kCmd = "tserver";
     const vector<string> kTServerModeRegexes = {
         "dump_memtrackers.*Dump the memtrackers",
         "get_flags.*Get the gflags",
@@ -1177,22 +1260,37 @@ TEST_F(ToolTest, TestModeHelp) {
         "timestamp.*Get the current timestamp",
         "list.*List tablet servers",
     };
-    NO_FATALS(RunTestHelp("tserver", kTServerModeRegexes));
+    NO_FATALS(RunTestHelp(kCmd, kTServerModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kCmd,
+        { "dump_memtrackers",
+          "get_flags",
+          "set_flag",
+          "status",
+          "timestamp",
+          "list",
+        }));
   }
   {
+    const string kSubCmd = "tserver state";
     const vector<string> kTServerSetStateModeRegexes = {
         "enter_maintenance.*Begin maintenance on the Tablet Server",
         "exit_maintenance.*End maintenance of the Tablet Server",
     };
-    NO_FATALS(RunTestHelp("tserver state", kTServerSetStateModeRegexes));
+    NO_FATALS(RunTestHelp(kSubCmd, kTServerSetStateModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kSubCmd,
+        { "enter_maintenance",
+          "exit_maintenance",
+        }));
   }
   {
+    const string kSubCmd = "tserver quiesce";
     const vector<string> kTServerQuiesceModeRegexes = {
         "status.*Output information about the quiescing state",
         "start.*Start quiescing the given Tablet Server",
         "stop.*Stop quiescing a Tablet Server",
     };
-    NO_FATALS(RunTestHelp("tserver quiesce", kTServerQuiesceModeRegexes));
+    NO_FATALS(RunTestHelp(kSubCmd, kTServerQuiesceModeRegexes));
+    NO_FATALS(RunTestHelpRpcFlags(kSubCmd, {"status", "start", "stop"}));
   }
   {
     const vector<string> kWalModeRegexes = {
@@ -5954,5 +6052,40 @@ TEST_F(ToolTest, TestGetTableStatisticsNotSupported) {
   ASSERT_STR_CONTAINS(stdout, "live row count: N/A");
 }
 
+// Run one of the cluster-targeted subcommands verifying the functionality of
+// the --connection_negotiation_timeout command-line option (the RPC
+// connection negotiation timeout).
+TEST_F(ToolTest, ConnectionNegotiationTimeoutOption) {
+  static constexpr const char* const kPattern =
+      "cluster ksck $0 --negotiation_timeout_ms=$1 --timeout_ms=$2";
+
+  FLAGS_rpc_negotiation_inject_delay_ms = 200;
+  NO_FATALS(StartMiniCluster());
+  const auto rpc_addr = mini_cluster_->mini_master()->bound_rpc_addr().ToString();
+
+  {
+    string msg;
+    // Set the RPC timeout to be a bit longer than the connection negotiation
+    // timeout, but not too much: otherwise, there would be many RPC retries.
+    auto s = RunActionStderrString(Substitute(kPattern, rpc_addr, 10, 11), &msg);
+    ASSERT_TRUE(s.IsRuntimeError());
+    ASSERT_STR_CONTAINS(msg, Substitute(
+        "Timed out: Client connection negotiation failed: client connection to "
+        "$0: received 0 of 4 requested bytes", rpc_addr));
+  }
+
+  {
+    string msg;
+    // Set the RPC timeout to be a bit longer than the connection negotiation
+    // timeout, but not too much: otherwise, there would be many RPC retries.
+    auto s = RunActionStderrString(Substitute(kPattern, rpc_addr, 2, 1), &msg);
+    ASSERT_TRUE(s.IsRuntimeError());
+    ASSERT_STR_CONTAINS(msg,
+        "RPC timeout set by --timeout_ms should be not less than connection "
+        "negotiation timeout set by --negotiation_timeout_ms; "
+        "current settings are 1 and 2 correspondingly");
+  }
+}
+
 } // namespace tools
 } // namespace kudu
diff --git a/src/kudu/tools/rebalancer_tool.cc b/src/kudu/tools/rebalancer_tool.cc
index b0a9525..824f531 100644
--- a/src/kudu/tools/rebalancer_tool.cc
+++ b/src/kudu/tools/rebalancer_tool.cc
@@ -20,6 +20,7 @@
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <functional>
 #include <iostream>
 #include <iterator>
 #include <map>
@@ -36,7 +37,6 @@
 #include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 
-#include "kudu/client/client.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/gutil/basictypes.h"
@@ -49,6 +49,7 @@
 #include "kudu/rebalance/placement_policy_util.h"
 #include "kudu/rebalance/rebalance_algo.h"
 #include "kudu/rebalance/rebalancer.h"
+#include "kudu/rpc/response_callback.h"
 #include "kudu/tools/ksck.h"
 #include "kudu/tools/ksck_remote.h"
 #include "kudu/tools/ksck_results.h"
@@ -57,7 +58,6 @@
 #include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 
-using kudu::client::KuduClientBuilder;
 using kudu::cluster_summary::ServerHealth;
 using kudu::cluster_summary::ServerHealthSummary;
 using kudu::cluster_summary::TableSummary;
@@ -669,9 +669,7 @@ Status RebalancerTool::BaseRunner::Init(vector<string> master_addresses) {
   DCHECK(master_addresses_.empty());
   DCHECK(client_.get() == nullptr);
   master_addresses_ = std::move(master_addresses);
-  return KuduClientBuilder()
-      .master_server_addrs(master_addresses_)
-      .Build(&client_);
+  return CreateKuduClient(master_addresses_, &client_);
 }
 
 Status RebalancerTool::BaseRunner::GetNextMoves(bool* has_moves) {
diff --git a/src/kudu/tools/rebalancer_tool.h b/src/kudu/tools/rebalancer_tool.h
index b07ff6e..94e6aed 100644
--- a/src/kudu/tools/rebalancer_tool.h
+++ b/src/kudu/tools/rebalancer_tool.h
@@ -33,7 +33,7 @@
 #include "kudu/client/shared_ptr.h" // IWYU pragma: keep
 #include "kudu/rebalance/rebalance_algo.h"
 #include "kudu/rebalance/rebalancer.h"
-#include "kudu/util/monotime.h"
+#include "kudu/util/monotime.h"     // IWYU pragma: keep
 #include "kudu/util/status.h"
 
 namespace kudu {
diff --git a/src/kudu/tools/tool_action.h b/src/kudu/tools/tool_action.h
index b8fea5a..e7a7cef 100644
--- a/src/kudu/tools/tool_action.h
+++ b/src/kudu/tools/tool_action.h
@@ -193,6 +193,7 @@ class ActionBuilder {
   // action runner. The name uniquely identifies the action amongst its
   // siblings in the tree.
   ActionBuilder(std::string name, ActionRunner runner);
+  virtual ~ActionBuilder() = default;
 
   // Sets the description of this action (e.g. "Format a new Kudu filesystem"),
   // to be used when printing the parent mode's help and the action's help.
@@ -245,9 +246,9 @@ class ActionBuilder {
                                       boost::optional<std::string> description = boost::none);
 
   // Creates an action using builder state.
-  std::unique_ptr<Action> Build();
+  virtual std::unique_ptr<Action> Build();
 
- private:
+ protected:
   const std::string name_;
 
   std::string description_;
diff --git a/src/kudu/tools/tool_action_cluster.cc b/src/kudu/tools/tool_action_cluster.cc
index 66fd411..cf7c4aa 100644
--- a/src/kudu/tools/tool_action_cluster.cc
+++ b/src/kudu/tools/tool_action_cluster.cc
@@ -375,10 +375,9 @@ unique_ptr<Mode> BuildClusterMode() {
         "to output detailed information on cluster status even if no "
         "inconsistency is found in metadata.";
 
-    unique_ptr<Action> ksck = ActionBuilder("ksck", &RunKsck)
+    unique_ptr<Action> ksck = ClusterActionBuilder("ksck", &RunKsck)
         .Description(desc)
         .ExtraDescription(extra_desc)
-        .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
         .AddOptionalParameter("checksum_cache_blocks")
         .AddOptionalParameter("checksum_scan")
         .AddOptionalParameter("checksum_scan_concurrency")
@@ -406,13 +405,12 @@ unique_ptr<Mode> BuildClusterMode() {
         "balance the count of replicas per table on each tablet server, "
         "and after that attempting to balance the total number of replicas "
         "per tablet server.";
-    unique_ptr<Action> rebalance = ActionBuilder("rebalance", &RunRebalance)
+    unique_ptr<Action> rebalance = ClusterActionBuilder("rebalance", &RunRebalance)
         .Description(desc)
         .ExtraDescription(extra_desc)
-        .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-        .AddOptionalParameter("disable_policy_fixer")
         .AddOptionalParameter("disable_cross_location_rebalancing")
         .AddOptionalParameter("disable_intra_location_rebalancing")
+        .AddOptionalParameter("disable_policy_fixer")
         .AddOptionalParameter("fetch_info_concurrency")
         .AddOptionalParameter("ignored_tservers")
         .AddOptionalParameter("load_imbalance_threshold")
diff --git a/src/kudu/tools/tool_action_common.cc b/src/kudu/tools/tool_action_common.cc
index 314db9c..b45fba5 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -42,6 +42,7 @@
 #include "kudu/client/client-internal.h"  // IWYU pragma: keep
 #include "kudu/client/client.h"
 #include "kudu/client/master_proxy_rpc.h"
+#include "kudu/client/replica_controller-internal.h"
 #include "kudu/client/shared_ptr.h" // IWYU pragma: keep
 #include "kudu/common/common.pb.h"
 #include "kudu/common/row_operations.h"
@@ -76,6 +77,7 @@
 #include "kudu/tserver/tserver_service.proxy.h" // IWYU pragma: keep
 #include "kudu/util/async_util.h"
 #include "kudu/util/env.h"
+#include "kudu/util/flag_validators.h"
 #include "kudu/util/jsonwriter.h"
 #include "kudu/util/mem_tracker.pb.h"
 #include "kudu/util/memory/arena.h"
@@ -140,9 +142,27 @@ static bool ValidateNumThreads(const char* flag_name, int32_t flag_value) {
 }
 DEFINE_validator(num_threads, &ValidateNumThreads);
 
+DEFINE_int64(negotiation_timeout_ms, 3000,
+             "Timeout for negotiating an RPC connection to a Kudu server, "
+             "in milliseconds");
+
+bool ValidateTimeoutSettings() {
+  if (FLAGS_timeout_ms < FLAGS_negotiation_timeout_ms) {
+    LOG(ERROR) << strings::Substitute(
+        "RPC timeout set by --timeout_ms should be not less than connection "
+        "negotiation timeout set by --negotiation_timeout_ms; "
+        "current settings are $0 and $1 correspondingly",
+        FLAGS_timeout_ms, FLAGS_negotiation_timeout_ms);
+    return false;
+  }
+  return true;
+}
+GROUP_FLAG_VALIDATOR(timeout_flags, ValidateTimeoutSettings);
+
 using kudu::client::KuduClient;
 using kudu::client::KuduClientBuilder;
 using kudu::client::internal::AsyncLeaderMasterRpc;
+using kudu::client::internal::ReplicaController;
 using kudu::consensus::ConsensusServiceProxy; // NOLINT
 using kudu::consensus::ReplicateMsg;
 using kudu::log::LogEntryPB;
@@ -200,6 +220,16 @@ const char* const kTabletIdsCsvArg = "tablet_ids";
 const char* const kTabletIdsCsvArgDesc =
     "Comma-separated list of Tablet Identifiers";
 
+const char* const kMasterAddressArg = "master_address";
+const char* const kMasterAddressDesc = "Address of a Kudu Master of form "
+    "'hostname:port'. Port may be omitted if the Master is bound to the "
+    "default port.";
+
+const char* const kTServerAddressArg = "tserver_address";
+const char* const kTServerAddressDesc = "Address of a Kudu Tablet Server of "
+    "form 'hostname:port'. Port may be omitted if the Tablet Server is bound "
+    "to the default port.";
+
 namespace {
 
 enum PrintEntryType {
@@ -356,17 +386,52 @@ Status GetServerFlags(const string& address,
 
 } // anonymous namespace
 
+RpcActionBuilder::RpcActionBuilder(std::string name, ActionRunner runner)
+    : ActionBuilder(std::move(name), std::move(runner)) {
+}
+
+unique_ptr<Action> RpcActionBuilder::Build() {
+  AddOptionalParameter("negotiation_timeout_ms");
+  AddOptionalParameter("timeout_ms");
+  return ActionBuilder::Build();
+}
+
+ClusterActionBuilder::ClusterActionBuilder(std::string name, ActionRunner runner)
+    : RpcActionBuilder(std::move(name), std::move(runner)) {
+  AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc });
+}
+
+MasterActionBuilder::MasterActionBuilder(std::string name, ActionRunner runner)
+    : RpcActionBuilder(std::move(name), std::move(runner)) {
+  AddRequiredParameter({ kMasterAddressArg, kMasterAddressDesc });
+}
+
+TServerActionBuilder::TServerActionBuilder(std::string name, ActionRunner runner)
+    : RpcActionBuilder(std::move(name), std::move(runner)) {
+  AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc });
+}
+
+Status BuildMessenger(std::string name, shared_ptr<Messenger>* messenger) {
+  shared_ptr<Messenger> m;
+  MessengerBuilder b(std::move(name));
+  b.set_rpc_negotiation_timeout_ms(FLAGS_negotiation_timeout_ms);
+  auto s = b.Build(&m);
+  if (s.ok()) {
+    *messenger = std::move(m);
+  }
+  return s;
+}
+
 template<class ProxyClass>
 Status BuildProxy(const string& address,
                   uint16_t default_port,
                   unique_ptr<ProxyClass>* proxy) {
   HostPort hp;
   RETURN_NOT_OK(hp.ParseString(address, default_port));
-  shared_ptr<Messenger> messenger;
-  RETURN_NOT_OK(MessengerBuilder("tool").Build(&messenger));
-
   vector<Sockaddr> resolved;
   RETURN_NOT_OK(hp.ResolveAddresses(&resolved));
+  shared_ptr<Messenger> messenger;
+  RETURN_NOT_OK(BuildMessenger("tool", &messenger));
 
   proxy->reset(new ProxyClass(messenger, resolved[0], hp.host()));
   return Status::OK();
@@ -507,14 +572,31 @@ bool MatchesAnyPattern(const vector<string>& patterns, const string& str) {
   return false;
 }
 
+Status CreateKuduClient(const vector<string>& master_addresses,
+                        client::sp::shared_ptr<KuduClient>* client,
+                        bool can_see_all_replicas) {
+  auto rpc_timeout = MonoDelta::FromMilliseconds(FLAGS_timeout_ms);
+  auto negotiation_timeout = MonoDelta::FromMilliseconds(
+      FLAGS_negotiation_timeout_ms);
+  KuduClientBuilder b;
+  if (can_see_all_replicas) {
+    ReplicaController::SetVisibility(&b, ReplicaController::Visibility::ALL);
+  }
+  return b
+      .master_server_addrs(master_addresses)
+      .default_rpc_timeout(rpc_timeout)
+      .default_admin_operation_timeout(rpc_timeout)
+      .connection_negotiation_timeout(negotiation_timeout)
+      .Build(client);
+}
+
 Status CreateKuduClient(const RunnerContext& context,
                         const char* master_addresses_arg,
                         client::sp::shared_ptr<KuduClient>* client) {
   vector<string> master_addresses;
-  RETURN_NOT_OK(ParseMasterAddresses(context, master_addresses_arg, &master_addresses));
-  return KuduClientBuilder()
-             .master_server_addrs(master_addresses)
-             .Build(client);
+  RETURN_NOT_OK(ParseMasterAddresses(context, master_addresses_arg,
+                                     &master_addresses));
+  return CreateKuduClient(master_addresses, client);
 }
 
 Status CreateKuduClient(const RunnerContext& context,
@@ -809,21 +891,28 @@ Status DataTable::PrintTo(ostream& out) const {
   return Status::OK();
 }
 
-LeaderMasterProxy::LeaderMasterProxy(client::sp::shared_ptr<KuduClient> client) :
-  client_(std::move(client)) {
+LeaderMasterProxy::LeaderMasterProxy(client::sp::shared_ptr<KuduClient> client)
+    : client_(std::move(client)) {
 }
 
-Status LeaderMasterProxy::Init(const vector<string>& master_addrs, const MonoDelta& timeout) {
-  return KuduClientBuilder().master_server_addrs(master_addrs)
-                            .default_rpc_timeout(timeout)
-                            .default_admin_operation_timeout(timeout)
-                            .Build(&client_);
+Status LeaderMasterProxy::Init(const vector<string>& master_addrs,
+                               const MonoDelta& timeout,
+                               const MonoDelta& connection_negotiation_timeout) {
+  return KuduClientBuilder()
+      .master_server_addrs(master_addrs)
+      .default_rpc_timeout(timeout)
+      .default_admin_operation_timeout(timeout)
+      .connection_negotiation_timeout(connection_negotiation_timeout)
+      .Build(&client_);
 }
 
 Status LeaderMasterProxy::Init(const RunnerContext& context) {
   vector<string> master_addresses;
   RETURN_NOT_OK(ParseMasterAddresses(context, &master_addresses));
-  return Init(master_addresses, MonoDelta::FromMilliseconds(FLAGS_timeout_ms));
+  return Init(
+     master_addresses,
+     MonoDelta::FromMilliseconds(FLAGS_timeout_ms),
+     MonoDelta::FromMilliseconds(FLAGS_negotiation_timeout_ms));
 }
 
 template<typename Req, typename Resp>
diff --git a/src/kudu/tools/tool_action_common.h b/src/kudu/tools/tool_action_common.h
index 083a6c6..9bf4d88 100644
--- a/src/kudu/tools/tool_action_common.h
+++ b/src/kudu/tools/tool_action_common.h
@@ -27,6 +27,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/rpc/response_callback.h"
+#include "kudu/tools/tool_action.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/status.h"
 
@@ -43,6 +44,7 @@ class MasterServiceProxy;
 } // namespace master
 
 namespace rpc {
+class Messenger;
 class RpcController;
 } // namespace rpc
 
@@ -56,8 +58,6 @@ class ServerStatusPB;
 
 namespace tools {
 
-struct RunnerContext;
-
 // Constants for parameters and descriptions.
 extern const char* const kMasterAddressesArg;
 extern const char* const kMasterAddressesArgDesc;
@@ -69,8 +69,45 @@ extern const char* const kTabletIdArgDesc;
 extern const char* const kTabletIdsCsvArg;
 extern const char* const kTabletIdsCsvArgDesc;
 
+extern const char* const kMasterAddressArg;
+extern const char* const kMasterAddressDesc;
+
+extern const char* const kTServerAddressArg;
+extern const char* const kTServerAddressDesc;
+
+// Builder for actions involving RPC communications, either with a whole Kudu
+// cluster or a particular Kudu RPC server.
+class RpcActionBuilder : public ActionBuilder {
+ public:
+  RpcActionBuilder(std::string name, ActionRunner runner);
+  std::unique_ptr<Action> Build() override;
+};
+
+// Builder for actions involving RPC communications with a Kudu cluster.
+class ClusterActionBuilder : public RpcActionBuilder {
+ public:
+  ClusterActionBuilder(std::string name, ActionRunner runner);
+};
+
+// Builder for actions involving RPC communications with a Kudu master.
+class MasterActionBuilder : public RpcActionBuilder {
+ public:
+  MasterActionBuilder(std::string name, ActionRunner runner);
+};
+
+// Builder for actions involving RPC communications with a Kudu tablet server.
+class TServerActionBuilder : public RpcActionBuilder {
+ public:
+  TServerActionBuilder(std::string name, ActionRunner runner);
+};
+
 // Utility methods used by multiple actions across different modes.
 
+// Build an RPC messenger with the specified name. The messenger has the
+// properties as defined by the command line flags/options.
+Status BuildMessenger(std::string name,
+                      std::shared_ptr<rpc::Messenger>* messenger);
+
 // Builds a proxy to a Kudu server running at 'address', returning it in
 // 'proxy'.
 //
@@ -126,6 +163,13 @@ Status DumpMemTrackers(const std::string& address, uint16_t default_port);
 // 'patterns' is empty.
 bool MatchesAnyPattern(const std::vector<std::string>& patterns, const std::string& str);
 
+// Create a Kudu client connected to the cluster with the set of the specified
+// master addresses. The 'can_see_all_replicas' controls whether the client sees
+// other than VOTER (e.g., NON_VOTER) tablet replicas.
+Status CreateKuduClient(const std::vector<std::string>& master_addresses,
+                        client::sp::shared_ptr<client::KuduClient>* client,
+                        bool can_see_all_replicas = false);
+
 // Creates a Kudu client connected to the cluster whose master addresses are specified by
 // 'master_addresses_arg'
 Status CreateKuduClient(const RunnerContext& context,
@@ -222,13 +266,18 @@ class LeaderMasterProxy {
   LeaderMasterProxy() = default;
   explicit LeaderMasterProxy(client::sp::shared_ptr<client::KuduClient> client);
 
-  // Initializes the leader master proxy with the given master addresses and timeout.
-  Status Init(const std::vector<std::string>& master_addrs, const MonoDelta& timeout);
+  // Initializes the leader master proxy with the given master addresses,
+  // RPC timeout, and connection negotiation timeout.
+  Status Init(const std::vector<std::string>& master_addrs,
+              const MonoDelta& timeout,
+              const MonoDelta& connection_negotiation_timeout);
 
   // Initialize the leader master proxy given the provided tool context.
   //
-  // Uses the required 'master_addresses' option for the master addresses, and
-  // the optional 'timeout_ms' flag to control admin and operation timeouts.
+  // Uses the required 'master_addresses' option for the master addresses,
+  // the optional 'timeout_ms' flag to control admin and operation timeouts, and
+  // the optional 'connection_negotiation_timeout_ms' flag to control
+  // the client-side RPC connection negotiation timeout.
   Status Init(const RunnerContext& context);
 
   // Calls a master RPC service method on the current leader master.
diff --git a/src/kudu/tools/tool_action_hms.cc b/src/kudu/tools/tool_action_hms.cc
index 8f53c0c..77bf078 100644
--- a/src/kudu/tools/tool_action_hms.cc
+++ b/src/kudu/tools/tool_action_hms.cc
@@ -47,15 +47,13 @@
 #include "kudu/hms/hms_client.h"
 #include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
-#include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 
-DECLARE_string(columns);
 DECLARE_bool(force);
 DECLARE_bool(hive_metastore_sasl_enabled);
-DECLARE_int64(timeout_ms);
+DECLARE_string(columns);
 DECLARE_string(hive_metastore_uris);
 
 DEFINE_bool(dryrun, false,
@@ -75,18 +73,13 @@ DEFINE_bool(ignore_other_clusters, true,
     "Whether to ignore entirely separate Kudu clusters, as indicated by a "
     "different set of master addresses.");
 
-namespace kudu {
-
-namespace tools {
-
-using client::KuduClient;
-using client::KuduClientBuilder;
-using client::KuduSchema;
-using client::KuduTable;
-using client::KuduTableAlterer;
-using client::sp::shared_ptr;
-using hms::HmsCatalog;
-using hms::HmsClient;
+using kudu::client::KuduClient;
+using kudu::client::KuduSchema;
+using kudu::client::KuduTable;
+using kudu::client::KuduTableAlterer;
+using kudu::client::sp::shared_ptr;
+using kudu::hms::HmsCatalog;
+using kudu::hms::HmsClient;
 using std::cout;
 using std::endl;
 using std::make_pair;
@@ -100,6 +93,9 @@ using std::vector;
 using strings::Split;
 using strings::Substitute;
 
+namespace kudu {
+namespace tools {
+
 // Only alter the table in Kudu but not in the Hive Metastore.
 Status RenameTableInKuduCatalog(KuduClient* kudu_client,
                                 const string& name,
@@ -128,10 +124,8 @@ Status Init(const RunnerContext& context,
   RETURN_NOT_OK(ParseMasterAddressesStr(context, &master_addrs_flag));
 
   // Create a Kudu Client.
-  RETURN_NOT_OK(KuduClientBuilder()
-      .default_rpc_timeout(MonoDelta::FromMilliseconds(FLAGS_timeout_ms))
-      .master_server_addrs(Split(master_addrs_flag, ","))
-      .Build(kudu_client));
+  const vector<string> master_addresses(Split(master_addrs_flag, ","));
+  RETURN_NOT_OK(CreateKuduClient(master_addresses, kudu_client));
 
   // Get the configured master addresses from the leader master. It's critical
   // that the check and fix tools use the exact same master address
@@ -844,11 +838,8 @@ Status Precheck(const RunnerContext& context) {
   string master_addrs;
   RETURN_NOT_OK(ParseMasterAddressesStr(context, &master_addrs));
   shared_ptr<KuduClient> client;
-  RETURN_NOT_OK(KuduClientBuilder()
-      .default_rpc_timeout(MonoDelta::FromMilliseconds(FLAGS_timeout_ms))
-      .master_server_addrs(Split(master_addrs, ","))
-      .Build(&client));
-
+  const vector<string> master_addresses(Split(master_addrs, ","));
+  RETURN_NOT_OK(CreateKuduClient(master_addresses, &client));
   vector<string> tables;
   RETURN_NOT_OK(client->ListTables(&tables));
 
@@ -911,61 +902,56 @@ unique_ptr<Mode> BuildHmsMode() {
     "Metastore configuration.";
 
   unique_ptr<Action> hms_check =
-      ActionBuilder("check", &CheckHmsMetadata)
-          .Description("Check metadata consistency between Kudu and the Hive Metastore catalogs")
-          .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-          .AddOptionalParameter("hive_metastore_sasl_enabled", boost::none, kHmsSaslEnabledDesc)
-          .AddOptionalParameter("hive_metastore_uris", boost::none, kHmsUrisDesc)
-          .AddOptionalParameter("ignore_other_clusters")
-          .Build();
+      ClusterActionBuilder("check", &CheckHmsMetadata)
+      .Description("Check metadata consistency between Kudu and the Hive Metastore catalogs")
+      .AddOptionalParameter("hive_metastore_sasl_enabled", boost::none, kHmsSaslEnabledDesc)
+      .AddOptionalParameter("hive_metastore_uris", boost::none, kHmsUrisDesc)
+      .AddOptionalParameter("ignore_other_clusters")
+      .Build();
 
   unique_ptr<Action> hms_downgrade =
-      ActionBuilder("downgrade", &HmsDowngrade)
-          .Description("Downgrade the metadata to legacy format for Kudu and the Hive Metastores")
-          .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-          .AddOptionalParameter("hive_metastore_sasl_enabled", boost::none, kHmsSaslEnabledDesc)
-          .AddOptionalParameter("hive_metastore_uris", boost::none, kHmsUrisDesc)
-          .Build();
+      ClusterActionBuilder("downgrade", &HmsDowngrade)
+      .Description("Downgrade the metadata to legacy format for Kudu and the Hive Metastores")
+      .AddOptionalParameter("hive_metastore_sasl_enabled", boost::none, kHmsSaslEnabledDesc)
+      .AddOptionalParameter("hive_metastore_uris", boost::none, kHmsUrisDesc)
+      .Build();
 
   unique_ptr<Action> hms_fix =
-    ActionBuilder("fix", &FixHmsMetadata)
-        .Description("Fix automatically-repairable metadata inconsistencies in the "
-                     "Kudu and Hive Metastore catalogs")
-        .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-        .AddOptionalParameter("dryrun")
-        .AddOptionalParameter("drop_orphan_hms_tables")
-        .AddOptionalParameter("create_missing_hms_tables")
-        .AddOptionalParameter("fix_inconsistent_tables")
-        .AddOptionalParameter("upgrade_hms_tables")
-        .AddOptionalParameter("hive_metastore_sasl_enabled", boost::none, kHmsSaslEnabledDesc)
-        .AddOptionalParameter("hive_metastore_uris", boost::none, kHmsUrisDesc)
-        .AddOptionalParameter("ignore_other_clusters")
-        .Build();
+      ClusterActionBuilder("fix", &FixHmsMetadata)
+      .Description("Fix automatically-repairable metadata inconsistencies in the "
+                   "Kudu and Hive Metastore catalogs")
+      .AddOptionalParameter("dryrun")
+      .AddOptionalParameter("drop_orphan_hms_tables")
+      .AddOptionalParameter("create_missing_hms_tables")
+      .AddOptionalParameter("fix_inconsistent_tables")
+      .AddOptionalParameter("upgrade_hms_tables")
+      .AddOptionalParameter("hive_metastore_sasl_enabled", boost::none, kHmsSaslEnabledDesc)
+      .AddOptionalParameter("hive_metastore_uris", boost::none, kHmsUrisDesc)
+      .AddOptionalParameter("ignore_other_clusters")
+      .Build();
 
   unique_ptr<Action> hms_list =
-      ActionBuilder("list", &List)
-          .Description("List the Kudu table HMS entries")
-          .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-          .AddOptionalParameter("columns",
-                                Substitute("database,table,type,$0",
-                                                  HmsClient::kKuduTableNameKey),
-                                Substitute("Comma-separated list of HMS entry fields to "
-                                           "include in output.\nPossible values: database, "
-                                           "table, type, owner, $0, $1, $2, $3, $4",
-                                           HmsClient::kKuduTableNameKey,
-                                           HmsClient::kKuduTableIdKey,
-                                           HmsClient::kKuduClusterIdKey,
-                                           HmsClient::kKuduMasterAddrsKey,
-                                           HmsClient::kStorageHandlerKey))
-          .AddOptionalParameter("format")
-          .Build();
+      ClusterActionBuilder("list", &List)
+      .Description("List the Kudu table HMS entries")
+      .AddOptionalParameter("columns",
+                            Substitute("database,table,type,$0",
+                                              HmsClient::kKuduTableNameKey),
+                            Substitute("Comma-separated list of HMS entry fields to "
+                                       "include in output.\nPossible values: database, "
+                                       "table, type, owner, $0, $1, $2, $3, $4",
+                                       HmsClient::kKuduTableNameKey,
+                                       HmsClient::kKuduTableIdKey,
+                                       HmsClient::kKuduClusterIdKey,
+                                       HmsClient::kKuduMasterAddrsKey,
+                                       HmsClient::kStorageHandlerKey))
+      .AddOptionalParameter("format")
+      .Build();
 
   unique_ptr<Action> hms_precheck =
-    ActionBuilder("precheck", &Precheck)
-        .Description("Check that the Kudu cluster is prepared to enable the Hive "
-                     "Metastore integration")
-        .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-        .Build();
+      ClusterActionBuilder("precheck", &Precheck)
+      .Description("Check that the Kudu cluster is prepared to enable "
+                   "the Hive Metastore integration")
+      .Build();
 
   return ModeBuilder("hms").Description("Operate on remote Hive Metastores")
                            .AddAction(std::move(hms_check))
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index e3691fa..2c99e04 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -67,7 +67,6 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/master/sys_catalog.h"
-#include "kudu/rpc/messenger.h"
 #include "kudu/tablet/diskrowset.h"
 #include "kudu/tablet/metadata.pb.h"
 #include "kudu/tablet/rowset.h"
@@ -87,6 +86,12 @@
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 
+namespace kudu {
+namespace rpc {
+class Messenger;
+}  // namespace rpc
+}  // namespace kudu
+
 DEFINE_bool(dump_all_columns, true,
             "If true, dumped rows include all of the columns in the rowset. If "
             "false, dumped rows include just the key columns (in a comparable format).");
@@ -111,23 +116,26 @@ DEFINE_bool(ignore_nonexistent, false,
 
 DECLARE_int32(tablet_copy_download_threads_nums_per_session);
 
-namespace kudu {
-namespace tools {
-
-using consensus::ConsensusMetadata;
-using consensus::ConsensusMetadataManager;
-using consensus::OpId;
-using consensus::RaftConfigPB;
-using consensus::RaftPeerPB;
-using fs::IOContext;
-using fs::ReadableBlock;
-using log::LogEntryPB;
-using log::LogEntryReader;
-using log::LogReader;
-using log::ReadableLogSegment;
-using log::SegmentSequence;
-using rpc::Messenger;
-using rpc::MessengerBuilder;
+using kudu::consensus::ConsensusMetadata;
+using kudu::consensus::ConsensusMetadataManager;
+using kudu::consensus::OpId;
+using kudu::consensus::RaftConfigPB;
+using kudu::consensus::RaftPeerPB;
+using kudu::fs::IOContext;
+using kudu::fs::ReadableBlock;
+using kudu::log::LogEntryPB;
+using kudu::log::LogEntryReader;
+using kudu::log::LogReader;
+using kudu::log::ReadableLogSegment;
+using kudu::log::SegmentSequence;
+using kudu::rpc::Messenger;
+using kudu::tablet::DiskRowSet;
+using kudu::tablet::RowIteratorOptions;
+using kudu::tablet::RowSetMetadata;
+using kudu::tablet::TabletDataState;
+using kudu::tablet::TabletMetadata;
+using kudu::tserver::TSTabletManager;
+using kudu::tserver::TabletCopyClient;
 using std::cout;
 using std::endl;
 using std::map;
@@ -137,13 +145,9 @@ using std::string;
 using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
-using tablet::DiskRowSet;
-using tablet::RowIteratorOptions;
-using tablet::RowSetMetadata;
-using tablet::TabletMetadata;
-using tablet::TabletDataState;
-using tserver::TabletCopyClient;
-using tserver::TSTabletManager;
+
+namespace kudu {
+namespace tools {
 
 namespace {
 
@@ -372,9 +376,8 @@ Status CopyFromRemote(const RunnerContext& context) {
   FsManager fs_manager(Env::Default(), FsManagerOpts());
   RETURN_NOT_OK(fs_manager.Open());
   scoped_refptr<ConsensusMetadataManager> cmeta_manager(new ConsensusMetadataManager(&fs_manager));
-  MessengerBuilder builder("tablet_copy_client");
   shared_ptr<Messenger> messenger;
-  builder.Build(&messenger);
+  RETURN_NOT_OK(BuildMessenger("tablet_copy_client", &messenger));
   TabletCopyClient client(tablet_id, &fs_manager, cmeta_manager,
                           messenger, nullptr /* no metrics */);
   RETURN_NOT_OK(client.Start(hp, nullptr));
@@ -880,7 +883,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
   unique_ptr<Action> print_replica_uuids =
       ActionBuilder("print_replica_uuids", &PrintReplicaUuids)
       .Description("Print all tablet replica peer UUIDs found in a "
-        "tablet's Raft configuration")
+                   "tablet's Raft configuration")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddOptionalParameter("fs_data_dirs")
       .AddOptionalParameter("fs_metadata_dir")
@@ -901,8 +904,9 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
       ActionBuilder("set_term", &SetRaftTerm)
       .Description("Bump the current term stored in consensus metadata")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
-      .AddRequiredParameter({ kTermArg, "the new raft term (must be greater "
-        "than the current term)" })
+      .AddRequiredParameter({ kTermArg,
+                              "the new raft term (must be greater "
+                              "than the current term)" })
       .AddOptionalParameter("fs_data_dirs")
       .AddOptionalParameter("fs_metadata_dir")
       .AddOptionalParameter("fs_wal_dir")
@@ -911,7 +915,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
   unique_ptr<Mode> cmeta =
       ModeBuilder("cmeta")
       .Description("Operate on a local tablet replica's consensus "
-        "metadata file")
+                   "metadata file")
       .AddAction(std::move(print_replica_uuids))
       .AddAction(std::move(rewrite_raft_config))
       .AddAction(std::move(set_term))
@@ -922,7 +926,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
       .Description("Copy a tablet replica from a remote server")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ "source", "Source RPC address of "
-        "form hostname:port" })
+                              "form hostname:port" })
       .AddOptionalParameter("fs_data_dirs")
       .AddOptionalParameter("fs_metadata_dir")
       .AddOptionalParameter("fs_wal_dir")
diff --git a/src/kudu/tools/tool_action_master.cc b/src/kudu/tools/tool_action_master.cc
index d50e0a9..dff2ac1 100644
--- a/src/kudu/tools/tool_action_master.cc
+++ b/src/kudu/tools/tool_action_master.cc
@@ -79,10 +79,6 @@ namespace kudu {
 namespace tools {
 namespace {
 
-const char* const kMasterAddressArg = "master_address";
-const char* const kMasterAddressDesc = "Address of a Kudu Master of form "
-    "'hostname:port'. Port may be omitted if the Master is bound to the "
-    "default port.";
 const char* const kFlagArg = "flag";
 const char* const kValueArg = "value";
 
@@ -327,16 +323,14 @@ unique_ptr<Mode> BuildMasterMode() {
 
   {
     unique_ptr<Action> action_refresh =
-        ActionBuilder("refresh", &RefreshAuthzCache)
-            .Description("Refresh the authorization policies")
-            .AddRequiredParameter(
-                {kMasterAddressesArg, kMasterAddressesArgDesc})
-            .AddOptionalParameter(
-                "force", boost::none,
-                string(
-                    "Ignore mismatches of the specified and the actual lists "
-                    "of master addresses in the cluster"))
-            .Build();
+        ClusterActionBuilder("refresh", &RefreshAuthzCache)
+        .Description("Refresh the authorization policies")
+        .AddOptionalParameter(
+            "force", boost::none,
+            string(
+                "Ignore mismatches of the specified and the actual lists "
+                "of master addresses in the cluster"))
+        .Build();
 
     unique_ptr<Mode> mode_authz_cache = ModeBuilder("authz_cache")
         .Description("Operate on the authz caches of the Kudu Masters")
@@ -346,20 +340,17 @@ unique_ptr<Mode> BuildMasterMode() {
   }
   {
     unique_ptr<Action> dump_memtrackers =
-        ActionBuilder("dump_memtrackers", &MasterDumpMemTrackers)
+        MasterActionBuilder("dump_memtrackers", &MasterDumpMemTrackers)
         .Description("Dump the memtrackers from a Kudu Master")
-        .AddRequiredParameter({ kMasterAddressArg, kMasterAddressDesc })
         .AddOptionalParameter("format")
         .AddOptionalParameter("memtracker_output")
-        .AddOptionalParameter("timeout_ms")
         .Build();
     builder.AddAction(std::move(dump_memtrackers));
   }
   {
     unique_ptr<Action> get_flags =
-        ActionBuilder("get_flags", &MasterGetFlags)
+        MasterActionBuilder("get_flags", &MasterGetFlags)
         .Description("Get the gflags for a Kudu Master")
-        .AddRequiredParameter({ kMasterAddressArg, kMasterAddressDesc })
         .AddOptionalParameter("all_flags")
         .AddOptionalParameter("flags")
         .AddOptionalParameter("flag_tags")
@@ -392,9 +383,8 @@ unique_ptr<Mode> BuildMasterMode() {
   }
   {
     unique_ptr<Action> set_flag =
-        ActionBuilder("set_flag", &MasterSetFlag)
+        MasterActionBuilder("set_flag", &MasterSetFlag)
         .Description("Change a gflag value on a Kudu Master")
-        .AddRequiredParameter({ kMasterAddressArg, kMasterAddressDesc })
         .AddRequiredParameter({ kFlagArg, "Name of the gflag" })
         .AddRequiredParameter({ kValueArg, "New value for the gflag" })
         .AddOptionalParameter("force")
@@ -403,25 +393,22 @@ unique_ptr<Mode> BuildMasterMode() {
   }
   {
     unique_ptr<Action> status =
-        ActionBuilder("status", &MasterStatus)
+        MasterActionBuilder("status", &MasterStatus)
         .Description("Get the status of a Kudu Master")
-        .AddRequiredParameter({ kMasterAddressArg, kMasterAddressDesc })
         .Build();
     builder.AddAction(std::move(status));
   }
   {
     unique_ptr<Action> timestamp =
-        ActionBuilder("timestamp", &MasterTimestamp)
+        MasterActionBuilder("timestamp", &MasterTimestamp)
         .Description("Get the current timestamp of a Kudu Master")
-        .AddRequiredParameter({ kMasterAddressArg, kMasterAddressDesc })
         .Build();
     builder.AddAction(std::move(timestamp));
   }
   {
     unique_ptr<Action> list_masters =
-        ActionBuilder("list", &ListMasters)
+        ClusterActionBuilder("list", &ListMasters)
         .Description("List masters in a Kudu cluster")
-        .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
         .AddOptionalParameter(
             "columns",
             string("uuid,rpc-addresses,role"),
@@ -430,7 +417,6 @@ unique_ptr<Mode> BuildMasterMode() {
                    "rpc-addresses, http-addresses, version, seqno, "
                    "start_time and role"))
         .AddOptionalParameter("format")
-        .AddOptionalParameter("timeout_ms")
         .Build();
     builder.AddAction(std::move(list_masters));
   }
diff --git a/src/kudu/tools/tool_action_perf.cc b/src/kudu/tools/tool_action_perf.cc
index d5b3d53..1129986 100644
--- a/src/kudu/tools/tool_action_perf.cc
+++ b/src/kudu/tools/tool_action_perf.cc
@@ -922,61 +922,59 @@ Status TabletScan(const RunnerContext& context) {
 
 unique_ptr<Mode> BuildPerfMode() {
   unique_ptr<Action> loadgen =
-      ActionBuilder("loadgen", &TestLoadGenerator)
-          .Description("Run load generation with optional scan afterwards")
-          .ExtraDescription(
-              "Run load generation tool which inserts auto-generated data into "
-              "an existing or auto-created table as fast as possible. "
-              "If requested, also scan the inserted rows to check whether the "
-              "actual count of inserted rows matches the expected one.")
-          .AddRequiredParameter({kMasterAddressesArg, kMasterAddressesArgDesc})
-          .AddOptionalParameter("auto_database")
-          .AddOptionalParameter("buffer_flush_watermark_pct")
-          .AddOptionalParameter("buffer_size_bytes")
-          .AddOptionalParameter("buffers_num")
-          .AddOptionalParameter("error_buffer_size_bytes")
-          .AddOptionalParameter("flush_per_n_rows")
-          .AddOptionalParameter("keep_auto_table")
-          .AddOptionalParameter("num_rows_per_thread")
-          .AddOptionalParameter("num_threads")
-          .AddOptionalParameter("run_cleanup")
-          .AddOptionalParameter("run_scan")
-          .AddOptionalParameter("seq_start")
-          .AddOptionalParameter("show_first_n_errors")
-          .AddOptionalParameter("string_fixed")
-          .AddOptionalParameter("string_len")
-          .AddOptionalParameter(
-              "table_name",
-              boost::none,
-              string("Name of an existing table to use for the test. The test will "
-                     "determine the structure of the table schema and "
-                     "populate it with data accordingly. If left empty, "
-                     "the test automatically creates a table of pre-defined columnar "
-                     "structure with unique name and uses it to insert "
-                     "auto-generated data. The auto-created table is dropped "
-                     "upon successful completion of the test if not overridden "
-                     "by the '--keep_auto_table' flag. If running the test against "
-                     "an already existing table, it's recommended to use a dedicated "
-                     "table created just for testing purposes: the tool doesn't delete "
-                     "the rows it inserted into the table. Neither the existing table "
-                     "nor its data is ever dropped/deleted."))
-          .AddOptionalParameter("table_num_hash_partitions")
-          .AddOptionalParameter("table_num_range_partitions")
-          .AddOptionalParameter("table_num_replicas")
-          .AddOptionalParameter("use_client_per_thread")
-          .AddOptionalParameter("use_random")
-          .AddOptionalParameter("use_random_pk")
-          .AddOptionalParameter("use_random_non_pk")
-          .AddOptionalParameter("use_upsert")
-          .Build();
+      ClusterActionBuilder("loadgen", &TestLoadGenerator)
+      .Description("Run load generation with optional scan afterwards")
+      .ExtraDescription(
+          "Run load generation tool which inserts auto-generated data into "
+          "an existing or auto-created table as fast as possible. "
+          "If requested, also scan the inserted rows to check whether the "
+          "actual count of inserted rows matches the expected one.")
+      .AddOptionalParameter("auto_database")
+      .AddOptionalParameter("buffer_flush_watermark_pct")
+      .AddOptionalParameter("buffer_size_bytes")
+      .AddOptionalParameter("buffers_num")
+      .AddOptionalParameter("error_buffer_size_bytes")
+      .AddOptionalParameter("flush_per_n_rows")
+      .AddOptionalParameter("keep_auto_table")
+      .AddOptionalParameter("num_rows_per_thread")
+      .AddOptionalParameter("num_threads")
+      .AddOptionalParameter("run_cleanup")
+      .AddOptionalParameter("run_scan")
+      .AddOptionalParameter("seq_start")
+      .AddOptionalParameter("show_first_n_errors")
+      .AddOptionalParameter("string_fixed")
+      .AddOptionalParameter("string_len")
+      .AddOptionalParameter(
+          "table_name",
+          boost::none,
+          string("Name of an existing table to use for the test. The test will "
+                 "determine the structure of the table schema and "
+                 "populate it with data accordingly. If left empty, "
+                 "the test automatically creates a table of pre-defined columnar "
+                 "structure with unique name and uses it to insert "
+                 "auto-generated data. The auto-created table is dropped "
+                 "upon successful completion of the test if not overridden "
+                 "by the '--keep_auto_table' flag. If running the test against "
+                 "an already existing table, it's recommended to use a dedicated "
+                 "table created just for testing purposes: the tool doesn't delete "
+                 "the rows it inserted into the table. Neither the existing table "
+                 "nor its data is ever dropped/deleted."))
+      .AddOptionalParameter("table_num_hash_partitions")
+      .AddOptionalParameter("table_num_range_partitions")
+      .AddOptionalParameter("table_num_replicas")
+      .AddOptionalParameter("use_client_per_thread")
+      .AddOptionalParameter("use_random")
+      .AddOptionalParameter("use_random_pk")
+      .AddOptionalParameter("use_random_non_pk")
+      .AddOptionalParameter("use_upsert")
+      .Build();
 
   unique_ptr<Action> table_scan =
-      ActionBuilder("table_scan", &TableScan)
+      ClusterActionBuilder("table_scan", &TableScan)
       .Description("Show row count and scanning time cost of tablets in a table")
       .ExtraDescription("Show row count and scanning time of tablets in a table. "
           "This can be useful to check for row count skew across different tablets, "
           "or whether there is a long latency tail when scanning different tables.")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to scan"})
       .AddOptionalParameter("columns")
       .AddOptionalParameter("fill_cache")
@@ -985,6 +983,7 @@ unique_ptr<Mode> BuildPerfMode() {
       .AddOptionalParameter("tablets")
       .Build();
 
+  // TODO(aserbin): move this to tool_local_replica.cc
   unique_ptr<Action> tablet_scan =
       ActionBuilder("tablet_scan", &TabletScan)
       .Description("Show row count of a local tablet")
diff --git a/src/kudu/tools/tool_action_remote_replica.cc b/src/kudu/tools/tool_action_remote_replica.cc
index 89a31b3..3b026d8 100644
--- a/src/kudu/tools/tool_action_remote_replica.cc
+++ b/src/kudu/tools/tool_action_remote_replica.cc
@@ -169,10 +169,6 @@ class ReplicaDumper {
 namespace {
 
 constexpr const char* const kReasonArg = "reason";
-constexpr const char* const kTServerAddressArg = "tserver_address";
-constexpr const char* const kTServerAddressDesc =
-    "Address of a Kudu Tablet Server of form 'hostname:port'. Port may be "
-    "omitted if the Tablet Server is bound to the default port.";
 constexpr const char* const kSrcAddressArg = "src_address";
 constexpr const char* const kDstAddressArg = "dst_address";
 constexpr const char* const kPeerUUIDsArg = "peer uuids";
@@ -424,16 +420,15 @@ Status UnsafeChangeConfig(const RunnerContext& context) {
 
 unique_ptr<Mode> BuildRemoteReplicaMode() {
   unique_ptr<Action> check_replicas =
-      ActionBuilder("check", &CheckReplicas)
+      TServerActionBuilder("check", &CheckReplicas)
       .Description("Check if all tablet replicas on a Kudu tablet server are "
                    "running. Tombstoned replica do not count as not running, "
                    "because they are just records of the previous existence of "
                    "a replica.")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
 
   unique_ptr<Action> copy_replica =
-      ActionBuilder("copy", &CopyReplica)
+      RpcActionBuilder("copy", &CopyReplica)
       .Description("Copy a tablet replica from one Kudu Tablet Server to another")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kSrcAddressArg, kTServerAddressDesc })
@@ -442,24 +437,21 @@ unique_ptr<Mode> BuildRemoteReplicaMode() {
       .Build();
 
   unique_ptr<Action> delete_replica =
-      ActionBuilder("delete", &DeleteReplica)
+      TServerActionBuilder("delete", &DeleteReplica)
       .Description("Delete a tablet replica from a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kReasonArg, "Reason for deleting the replica" })
       .Build();
 
   unique_ptr<Action> dump_replica =
-      ActionBuilder("dump", &DumpReplica)
+      TServerActionBuilder("dump", &DumpReplica)
       .Description("Dump the data of a tablet replica on a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .Build();
 
   unique_ptr<Action> list =
-      ActionBuilder("list", &ListReplicas)
+      TServerActionBuilder("list", &ListReplicas)
       .Description("List all tablet replicas on a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddOptionalParameter("include_schema")
       .AddOptionalParameter("table_name")
       .AddOptionalParameter("tablets",
@@ -469,7 +461,7 @@ unique_ptr<Mode> BuildRemoteReplicaMode() {
       .Build();
 
   unique_ptr<Action> unsafe_change_config =
-      ActionBuilder("unsafe_change_config", &UnsafeChangeConfig)
+      TServerActionBuilder("unsafe_change_config", &UnsafeChangeConfig)
       .Description("Force the specified replica to adopt a new Raft config")
       .ExtraDescription("This tool is useful when a config change is "
                         "necessary because a tablet cannot make progress with "
@@ -478,7 +470,6 @@ unique_ptr<Mode> BuildRemoteReplicaMode() {
                         "The members of the new Raft config must be a subset "
                         "of (or the same as) the members of the existing "
                         "committed Raft config.")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredVariadicParameter({ kPeerUUIDsArg, kPeerUUIDsArgDesc })
       .Build();
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 2071ae7..59b7a94 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -65,7 +65,6 @@ using google::protobuf::util::JsonStringToMessage;
 using google::protobuf::util::JsonParseOptions;
 using google::protobuf::RepeatedPtrField;
 using kudu::client::KuduClient;
-using kudu::client::KuduClientBuilder;
 using kudu::client::KuduColumnSchema;
 using kudu::client::KuduColumnSpec;
 using kudu::client::KuduColumnStorageAttributes;
@@ -116,6 +115,7 @@ DEFINE_string(lower_bound_type, "INCLUSIVE_BOUND",
 DEFINE_string(upper_bound_type, "EXCLUSIVE_BOUND",
               "The type of the upper bound, either inclusive or exclusive. "
               "Defaults to exclusive. This flag is case-insensitive.");
+
 DECLARE_bool(show_values);
 DECLARE_string(tables);
 
@@ -127,12 +127,10 @@ namespace tools {
 class TableLister {
  public:
   static Status ListTablets(const vector<string>& master_addresses) {
-    KuduClientBuilder builder;
-    ReplicaController::SetVisibility(&builder, ReplicaController::Visibility::ALL);
     client::sp::shared_ptr<KuduClient> client;
-    RETURN_NOT_OK(builder
-                  .master_server_addrs(master_addresses)
-                  .Build(&client));
+    RETURN_NOT_OK(CreateKuduClient(master_addresses,
+                                   &client,
+                                   true /* can_see_all_replicas */));
     vector<string> table_names;
     RETURN_NOT_OK(client->ListTables(&table_names));
 
@@ -1177,31 +1175,28 @@ Status CreateTable(const RunnerContext& context) {
 
 unique_ptr<Mode> BuildTableMode() {
   unique_ptr<Action> delete_table =
-      ActionBuilder("delete", &DeleteTable)
+      ClusterActionBuilder("delete", &DeleteTable)
       .Description("Delete a table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to delete" })
       .AddOptionalParameter("modify_external_catalogs")
       .Build();
 
   unique_ptr<Action> describe_table =
-      ActionBuilder("describe", &DescribeTable)
+      ClusterActionBuilder("describe", &DescribeTable)
       .Description("Describe a table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to describe" })
       .AddOptionalParameter("show_attributes")
       .Build();
 
   unique_ptr<Action> list_tables =
-      ActionBuilder("list", &ListTables)
+      ClusterActionBuilder("list", &ListTables)
       .Description("List tables")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddOptionalParameter("tables")
       .AddOptionalParameter("list_tablets")
       .Build();
 
   unique_ptr<Action> locate_row =
-      ActionBuilder("locate_row", &LocateRow)
+      ClusterActionBuilder("locate_row", &LocateRow)
       .Description("Locate which tablet a row belongs to")
       .ExtraDescription("Provide the primary key as a JSON array of primary "
                         "key values, e.g. '[1, \"foo\", 2, \"bar\"]'. The "
@@ -1209,7 +1204,6 @@ unique_ptr<Mode> BuildTableMode() {
                         "key. If there is no such tablet, an error message "
                         "will be printed and the command will return a "
                         "non-zero status")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to look up against" })
       .AddRequiredParameter({ kKeyArg,
                               "String representation of the row's primary key "
@@ -1218,29 +1212,26 @@ unique_ptr<Mode> BuildTableMode() {
       .Build();
 
   unique_ptr<Action> rename_column =
-      ActionBuilder("rename_column", &RenameColumn)
+      ClusterActionBuilder("rename_column", &RenameColumn)
       .Description("Rename a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to rename" })
       .AddRequiredParameter({ kNewColumnNameArg, "New column name" })
       .Build();
 
   unique_ptr<Action> rename_table =
-      ActionBuilder("rename_table", &RenameTable)
+      ClusterActionBuilder("rename_table", &RenameTable)
       .Description("Rename a table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to rename" })
       .AddRequiredParameter({ kNewTableNameArg, "New table name" })
       .AddOptionalParameter("modify_external_catalogs")
       .Build();
 
   unique_ptr<Action> scan_table =
-      ActionBuilder("scan", &ScanTable)
+      ClusterActionBuilder("scan", &ScanTable)
       .Description("Scan rows from a table")
       .ExtraDescription("Scan rows from an existing table. See the help "
                         "for the --predicates flag on how predicates can be specified.")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to scan"})
       .AddOptionalParameter("columns")
       .AddOptionalParameter("fill_cache")
@@ -1250,14 +1241,13 @@ unique_ptr<Mode> BuildTableMode() {
       .Build();
 
   unique_ptr<Action> copy_table =
-      ActionBuilder("copy", &CopyTable)
+      ClusterActionBuilder("copy", &CopyTable)
       .Description("Copy table data to another table")
       .ExtraDescription("Copy table data to another table; the two tables could be in the same "
                         "cluster or not. The two tables must have the same table schema, but "
                         "could have different partition schemas. Alternatively, the tool can "
                         "create the new table using the same table and partition schema as the "
                         "source table.")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the source table" })
       .AddRequiredParameter({ kDestMasterAddressesArg, kDestMasterAddressesArgDesc })
       .AddOptionalParameter("create_table")
@@ -1269,27 +1259,24 @@ unique_ptr<Mode> BuildTableMode() {
       .Build();
 
   unique_ptr<Action> set_extra_config =
-      ActionBuilder("set_extra_config", &SetExtraConfig)
+      ClusterActionBuilder("set_extra_config", &SetExtraConfig)
       .Description("Change a extra configuration value on a table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kConfigNameArg, "Name of the configuration" })
       .AddRequiredParameter({ kConfigValueArg, "New value for the configuration" })
       .Build();
 
   unique_ptr<Action> get_extra_configs =
-      ActionBuilder("get_extra_configs", &GetExtraConfigs)
+      ClusterActionBuilder("get_extra_configs", &GetExtraConfigs)
       .Description("Get the extra configuration properties for a table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg,
                               "Name of the table for which to get extra configurations" })
       .AddOptionalParameter("config_names")
       .Build();
 
   unique_ptr<Action> drop_range_partition =
-      ActionBuilder("drop_range_partition", &DropRangePartition)
+      ClusterActionBuilder("drop_range_partition", &DropRangePartition)
       .Description("Drop a range partition of table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table" })
       .AddRequiredParameter({ kTableRangeLowerBoundArg,
                               "String representation of lower bound of "
@@ -1302,9 +1289,8 @@ unique_ptr<Mode> BuildTableMode() {
       .Build();
 
   unique_ptr<Action> add_range_partition =
-      ActionBuilder("add_range_partition", &AddRangePartition)
+      ClusterActionBuilder("add_range_partition", &AddRangePartition)
       .Description("Add a range partition for table")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table" })
       .AddRequiredParameter({ kTableRangeLowerBoundArg,
                               "String representation of lower bound of "
@@ -1321,9 +1307,8 @@ unique_ptr<Mode> BuildTableMode() {
       .Build();
 
   unique_ptr<Action> column_set_default =
-      ActionBuilder("column_set_default", &ColumnSetDefault)
+      ClusterActionBuilder("column_set_default", &ColumnSetDefault)
       .Description("Set write_default value for a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to alter" })
       .AddRequiredParameter({ kDefaultValueArg,
@@ -1332,67 +1317,60 @@ unique_ptr<Mode> BuildTableMode() {
       .Build();
 
   unique_ptr<Action> column_remove_default =
-      ActionBuilder("column_remove_default", &ColumnRemoveDefault)
+      ClusterActionBuilder("column_remove_default", &ColumnRemoveDefault)
       .Description("Remove write_default value for a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to alter" })
       .Build();
 
   unique_ptr<Action> column_set_compression =
-      ActionBuilder("column_set_compression", &ColumnSetCompression)
+      ClusterActionBuilder("column_set_compression", &ColumnSetCompression)
       .Description("Set compression type for a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to alter" })
       .AddRequiredParameter({ kCompressionTypeArg, "Compression type of the column" })
       .Build();
 
   unique_ptr<Action> column_set_encoding =
-      ActionBuilder("column_set_encoding", &ColumnSetEncoding)
+      ClusterActionBuilder("column_set_encoding", &ColumnSetEncoding)
       .Description("Set encoding type for a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to alter" })
       .AddRequiredParameter({ kEncodingTypeArg, "Encoding type of the column" })
       .Build();
 
   unique_ptr<Action> column_set_block_size =
-      ActionBuilder("column_set_block_size", &ColumnSetBlockSize)
+      ClusterActionBuilder("column_set_block_size", &ColumnSetBlockSize)
       .Description("Set block size for a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to alter" })
       .AddRequiredParameter({ kBlockSizeArg, "Block size of the column" })
       .Build();
 
   unique_ptr<Action> column_set_comment =
-      ActionBuilder("column_set_comment", &ColumnSetComment)
+      ClusterActionBuilder("column_set_comment", &ColumnSetComment)
       .Description("Set comment for a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to alter" })
       .AddRequiredParameter({ kColumnCommentArg, "Comment of the column" })
       .Build();
 
   unique_ptr<Action> delete_column =
-      ActionBuilder("delete_column", &DeleteColumn)
+      ClusterActionBuilder("delete_column", &DeleteColumn)
       .Description("Delete a column")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kColumnNameArg, "Name of the table column to delete" })
       .Build();
 
 
   unique_ptr<Action> statistics =
-      ActionBuilder("statistics", &GetTableStatistics)
-          .Description("Get table statistics")
-          .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
-          .AddRequiredParameter({ kTableNameArg, "Name of the table to get statistics" })
-          .Build();
+      ClusterActionBuilder("statistics", &GetTableStatistics)
+      .Description("Get table statistics")
+      .AddRequiredParameter({ kTableNameArg, "Name of the table to get statistics" })
+      .Build();
 
   unique_ptr<Action> create_table =
-      ActionBuilder("create", &CreateTable)
+      ClusterActionBuilder("create", &CreateTable)
       .Description("Create a new table")
       .ExtraDescription("Provide the  table-build statements as a JSON object, e.g."
                         "'{\"table_name\":\"test\",\"schema\":{\"columns\":[{\"column_name"
@@ -1408,7 +1386,6 @@ unique_ptr<Mode> BuildTableMode() {
                         "\": [\"2\"]},\"upper_bound\":{\"bound_type\":\"inclusive\",\""
                         "bound_values\":[\"3\"]}}]}},\"extra_configs\":{\"configs\":{\""
                         "kudu.table.history_max_age_sec\":\"3600\"}},\"num_replicas\":3}'.")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kCreateTableJSONArg, "JSON object for creating table" })
       .Build();
 
diff --git a/src/kudu/tools/tool_action_tablet.cc b/src/kudu/tools/tool_action_tablet.cc
index 92fb7f5..b057b17 100644
--- a/src/kudu/tools/tool_action_tablet.cc
+++ b/src/kudu/tools/tool_action_tablet.cc
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include <boost/optional/optional.hpp>
+#include <boost/type_traits/decay.hpp>
 #include <gflags/gflags.h>
 
 #include "kudu/client/client.h"
@@ -37,6 +38,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
+#include "kudu/rpc/response_callback.h"
 #include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/tools/tool_replica_util.h"
@@ -64,7 +66,6 @@ DEFINE_int64(move_leader_timeout_sec, 30,
              "Number of seconds to wait for a leader when relocating a leader tablet");
 
 using kudu::client::KuduClient;
-using kudu::client::KuduClientBuilder;
 using kudu::consensus::ADD_PEER;
 using kudu::consensus::ChangeConfigType;
 using kudu::consensus::LeaderStepDownMode;
@@ -153,9 +154,7 @@ Status LeaderStepDown(const RunnerContext& context) {
   }
 
   client::sp::shared_ptr<KuduClient> client;
-  RETURN_NOT_OK(KuduClientBuilder()
-                .master_server_addrs(master_addresses)
-                .Build(&client));
+  RETURN_NOT_OK(CreateKuduClient(master_addresses, &client));
 
   string leader_uuid;
   HostPort leader_hp;
@@ -227,9 +226,7 @@ Status MoveReplica(const RunnerContext& context) {
                                          MonoDelta::FromSeconds(5)),
                         "ksck pre-move health check failed");
   client::sp::shared_ptr<KuduClient> client;
-  RETURN_NOT_OK(KuduClientBuilder()
-                .master_server_addrs(master_addresses)
-                .Build(&client));
+  RETURN_NOT_OK(CreateKuduClient(master_addresses, &client));
   RETURN_NOT_OK(ScheduleReplicaMove(master_addresses, client,
                                     tablet_id, from_ts_uuid, to_ts_uuid));
   const auto copy_timeout = MonoDelta::FromSeconds(FLAGS_move_copy_timeout_sec);
@@ -264,9 +261,8 @@ Status ReplaceTablet(const RunnerContext& context) {
 
 unique_ptr<Mode> BuildTabletMode() {
   unique_ptr<Action> add_replica =
-      ActionBuilder("add_replica", &AddReplica)
+      ClusterActionBuilder("add_replica", &AddReplica)
       .Description("Add a new replica to a tablet's Raft configuration")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kTsUuidArg,
                               "UUID of the tablet server that should host the new replica" })
@@ -276,10 +272,9 @@ unique_ptr<Mode> BuildTabletMode() {
       .Build();
 
   unique_ptr<Action> change_replica_type =
-      ActionBuilder("change_replica_type", &ChangeReplicaType)
+      ClusterActionBuilder("change_replica_type", &ChangeReplicaType)
       .Description(
           "Change the type of an existing replica in a tablet's Raft configuration")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kTsUuidArg,
                               "UUID of the tablet server hosting the existing replica" })
@@ -289,9 +284,8 @@ unique_ptr<Mode> BuildTabletMode() {
       .Build();
 
   unique_ptr<Action> remove_replica =
-      ActionBuilder("remove_replica", &RemoveReplica)
+      ClusterActionBuilder("remove_replica", &RemoveReplica)
       .Description("Remove an existing replica from a tablet's Raft configuration")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kTsUuidArg,
                               "UUID of the tablet server hosting the existing replica" })
@@ -306,19 +300,17 @@ unique_ptr<Mode> BuildTabletMode() {
       "new replica will be deleted automatically after some time, and then the "
       "move can be retried.";
   unique_ptr<Action> move_replica =
-      ActionBuilder("move_replica", &MoveReplica)
+      ClusterActionBuilder("move_replica", &MoveReplica)
       .Description("Move a tablet replica from one tablet server to another")
       .ExtraDescription(move_extra_desc)
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kFromTsUuidArg, "UUID of the tablet server to move from" })
       .AddRequiredParameter({ kToTsUuidArg, "UUID of the tablet server to move to" })
       .Build();
 
   unique_ptr<Action> leader_step_down =
-      ActionBuilder("leader_step_down", &LeaderStepDown)
+      ClusterActionBuilder("leader_step_down", &LeaderStepDown)
       .Description("Change the tablet's leader")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddOptionalParameter("abrupt")
       .AddOptionalParameter("new_leader_uuid")
@@ -334,7 +326,7 @@ unique_ptr<Mode> BuildTabletMode() {
       .Build();
 
   unique_ptr<Action> replace_tablet =
-      ActionBuilder("unsafe_replace_tablet", &ReplaceTablet)
+      ClusterActionBuilder("unsafe_replace_tablet", &ReplaceTablet)
       .Description("Replace a tablet with an empty one, deleting the previous tablet.")
       .ExtraDescription("Use this tool to repair a table when one of its tablets has permanently "
                         "lost all of its replicas. It replaces the unrecoverable tablet with a new "
@@ -344,7 +336,6 @@ unique_ptr<Mode> BuildTabletMode() {
                         "NOTE: The original tablet will be deleted. Its data will be permanently "
                         "lost. Additionally, clients should be restarted before attempting to "
                         "use the repaired table (see KUDU-2376).")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .Build();
 
diff --git a/src/kudu/tools/tool_action_tserver.cc b/src/kudu/tools/tool_action_tserver.cc
index 0ea95a6..840b1d3 100644
--- a/src/kudu/tools/tool_action_tserver.cc
+++ b/src/kudu/tools/tool_action_tserver.cc
@@ -82,10 +82,6 @@ using tserver::TabletServerAdminServiceProxy;
 namespace tools {
 namespace {
 
-const char* const kTServerAddressArg = "tserver_address";
-const char* const kTServerAddressDesc = "Address of a Kudu Tablet Server of "
-    "form 'hostname:port'. Port may be omitted if the Tablet Server is bound "
-    "to the default port.";
 const char* const kTServerIdArg = "tserver_uuid";
 const char* const kTServerIdDesc = "UUID of a Kudu Tablet Server";
 const char* const kFlagArg = "flag";
@@ -304,18 +300,15 @@ Status QuiescingStatus(const RunnerContext& context) {
 
 unique_ptr<Mode> BuildTServerMode() {
   unique_ptr<Action> dump_memtrackers =
-      ActionBuilder("dump_memtrackers", &TserverDumpMemTrackers)
+      TServerActionBuilder("dump_memtrackers", &TserverDumpMemTrackers)
       .Description("Dump the memtrackers from a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddOptionalParameter("format")
       .AddOptionalParameter("memtracker_output")
-      .AddOptionalParameter("timeout_ms")
       .Build();
 
   unique_ptr<Action> get_flags =
-      ActionBuilder("get_flags", &TServerGetFlags)
+      TServerActionBuilder("get_flags", &TServerGetFlags)
       .Description("Get the gflags for a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddOptionalParameter("all_flags")
       .AddOptionalParameter("flags")
       .AddOptionalParameter("flag_tags")
@@ -345,58 +338,50 @@ unique_ptr<Mode> BuildTServerMode() {
       .Build();
 
   unique_ptr<Action> set_flag =
-      ActionBuilder("set_flag", &TServerSetFlag)
+      TServerActionBuilder("set_flag", &TServerSetFlag)
       .Description("Change a gflag value on a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kFlagArg, "Name of the gflag" })
       .AddRequiredParameter({ kValueArg, "New value for the gflag" })
       .AddOptionalParameter("force")
       .Build();
 
   unique_ptr<Action> status =
-      ActionBuilder("status", &TServerStatus)
+      TServerActionBuilder("status", &TServerStatus)
       .Description("Get the status of a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
 
   unique_ptr<Action> timestamp =
-      ActionBuilder("timestamp", &TServerTimestamp)
+      TServerActionBuilder("timestamp", &TServerTimestamp)
       .Description("Get the current timestamp of a Kudu Tablet Server")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
 
   unique_ptr<Action> list_tservers =
-      ActionBuilder("list", &ListTServers)
+      ClusterActionBuilder("list", &ListTServers)
       .Description("List tablet servers in a Kudu cluster")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddOptionalParameter("columns", string("uuid,rpc-addresses"),
                             string("Comma-separated list of tserver info fields to "
                                    "include in output.\nPossible values: uuid, "
                                    "rpc-addresses, http-addresses, version, seqno, "
                                    "heartbeat, start_time, state"))
       .AddOptionalParameter("format")
-      .AddOptionalParameter("timeout_ms")
       .Build();
 
   unique_ptr<Action> quiescing_status =
-      ActionBuilder("status", &QuiescingStatus)
+      TServerActionBuilder("status", &QuiescingStatus)
       .Description("Output information about the quiescing state of a Tablet "
                    "Server.")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
   unique_ptr<Action> start_quiescing =
-      ActionBuilder("start", &StartQuiescingTServer)
+      TServerActionBuilder("start", &StartQuiescingTServer)
       .Description("Start quiescing the given Tablet Server. While a Tablet "
                    "Server is quiescing, Tablet replicas on it will no longer "
                    "attempt to become leader, and new scan requests will be "
                    "retried at other servers.")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddOptionalParameter("error_if_not_fully_quiesced")
       .Build();
   unique_ptr<Action> stop_quiescing =
-      ActionBuilder("stop", &StopQuiescingTServer)
+      TServerActionBuilder("stop", &StopQuiescingTServer)
       .Description("Stop quiescing a Tablet Server.")
-      .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
   unique_ptr<Mode> quiesce = ModeBuilder("quiesce")
       .Description("Operate on the quiescing state of a Kudu Tablet Server.")
@@ -406,11 +391,10 @@ unique_ptr<Mode> BuildTServerMode() {
       .Build();
 
   unique_ptr<Action> enter_maintenance =
-      ActionBuilder("enter_maintenance", &EnterMaintenance)
+      ClusterActionBuilder("enter_maintenance", &EnterMaintenance)
       .Description("Begin maintenance on the Tablet Server. While under "
                    "maintenance, downtime of the Tablet Server will not lead "
                    "to the immediate re-replication of its tablet replicas.")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTServerIdArg, kTServerIdDesc })
       .AddOptionalParameter("allow_missing_tserver")
       .Build();
@@ -418,9 +402,8 @@ unique_ptr<Mode> BuildTServerMode() {
   // because if the tserver is missing, the non-existent tserver's state is
   // already NONE and so exit_maintenance is a no-op.
   unique_ptr<Action> exit_maintenance =
-      ActionBuilder("exit_maintenance", &ExitMaintenance)
+      ClusterActionBuilder("exit_maintenance", &ExitMaintenance)
       .Description("End maintenance of the Tablet Server.")
-      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTServerIdArg, kTServerIdDesc })
       .Build();
   unique_ptr<Mode> state = ModeBuilder("state")
diff --git a/src/kudu/tools/tool_replica_util.cc b/src/kudu/tools/tool_replica_util.cc
index 0a23cc8..34e5174 100644
--- a/src/kudu/tools/tool_replica_util.cc
+++ b/src/kudu/tools/tool_replica_util.cc
@@ -49,7 +49,6 @@
 
 using kudu::MonoDelta;
 using kudu::client::KuduClient;
-using kudu::client::KuduClientBuilder;
 using kudu::client::KuduTablet;
 using kudu::client::KuduTabletServer;
 using kudu::consensus::ADD_PEER;
@@ -674,9 +673,7 @@ Status DoChangeConfig(const vector<string>& master_addresses,
   }
 
   client::sp::shared_ptr<KuduClient> client;
-  RETURN_NOT_OK(KuduClientBuilder()
-                .master_server_addrs(master_addresses)
-                .Build(&client));
+  RETURN_NOT_OK(CreateKuduClient(master_addresses, &client));
 
   // When adding a new server, we need to provide the server's RPC address.
   if (cc_type == consensus::ADD_PEER) {
@@ -724,9 +721,7 @@ Status Is343SchemeCluster(const vector<string>& master_addresses,
                           const boost::optional<string>& tablet_id_in,
                           bool* is_343_scheme) {
   client::sp::shared_ptr<client::KuduClient> client;
-  RETURN_NOT_OK(client::KuduClientBuilder()
-                .master_server_addrs(master_addresses)
-                .Build(&client));
+  RETURN_NOT_OK(CreateKuduClient(master_addresses, &client));
   string tablet_id;
   if (tablet_id_in) {
     tablet_id = *tablet_id_in;