You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2017/04/20 20:40:54 UTC

kudu git commit: Add 'kudu tserver list' tool

Repository: kudu
Updated Branches:
  refs/heads/master b2843ee5e -> 319a1bc15


Add 'kudu tserver list' tool

This adds a new tool action to list tablet servers, and associated
information. There are a few output formatting options: the default is a
human-readable table in the psql style; the others are meant for machine
parsing.

The formatting and leader RPC proxy code is abstracted so that
future list tools (e.g. table, tablet, and master) can reuse them.

Change-Id: I047a7675c186a02dd5d8b405431ae885159fcfee
Reviewed-on: http://gerrit.cloudera.org:8080/6654
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 319a1bc15923b7859a93d6ee8258f03df60a2385
Parents: b2843ee
Author: Dan Burkert <da...@apache.org>
Authored: Sun Apr 16 16:17:56 2017 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Thu Apr 20 20:40:38 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client.h              |   5 +
 src/kudu/tools/kudu-tool-test.cc      |  59 +++++++++-
 src/kudu/tools/tool_action.cc         |  61 +++++++---
 src/kudu/tools/tool_action.h          |  29 ++++-
 src/kudu/tools/tool_action_common.cc  | 181 ++++++++++++++++++++++++++++-
 src/kudu/tools/tool_action_common.h   |  48 ++++++++
 src/kudu/tools/tool_action_tserver.cc |  92 ++++++++++++++-
 7 files changed, 447 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 388ff9b..b43c4e1 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -53,6 +53,10 @@ class ClientStressTest_TestUniqueClientIds_Test;
 class LinkedListTester;
 class PartitionSchema;
 
+namespace tools {
+class LeaderMasterProxy;
+} // namespace tools
+
 namespace client {
 
 class KuduLoggingCallback;
@@ -520,6 +524,7 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   friend class KuduTable;
   friend class KuduTableAlterer;
   friend class KuduTableCreator;
+  friend class tools::LeaderMasterProxy;
 
   FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
   FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);

http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index e082cf9..cab9d8d 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -453,7 +453,8 @@ TEST_F(ToolTest, TestModeHelp) {
     const vector<string> kTServerModeRegexes = {
         "set_flag.*Change a gflag value",
         "status.*Get the status",
-        "timestamp.*Get the current timestamp"
+        "timestamp.*Get the current timestamp",
+        "list.*List tablet servers"
     };
     NO_FATALS(RunTestHelp("tserver", kTServerModeRegexes));
   }
@@ -1357,5 +1358,61 @@ TEST_F(ToolTest, TestLocalReplicaTombstoneDelete) {
   }
 }
 
+TEST_F(ToolTest, TestTserverList) {
+  NO_FATALS(StartExternalMiniCluster({}, {}, 1));
+
+  string master_addr = cluster_->master()->bound_rpc_addr().ToString();
+  const auto& tserver = cluster_->tablet_server(0);
+
+  { // TSV
+    string out;
+    NO_FATALS(RunActionStdoutString(Substitute("tserver list $0 --columns=uuid --format=tsv",
+                                              master_addr),
+                                    &out));
+
+    ASSERT_EQ(tserver->uuid(), out);
+  }
+
+  { // JSON
+    string out;
+    NO_FATALS(RunActionStdoutString(
+          Substitute("tserver list $0 --columns=uuid,rpc-addresses --format=json", master_addr),
+          &out));
+
+    ASSERT_EQ(Substitute("[{\"uuid\":\"$0\",\"rpc-addresses\":\"$1\"}]",
+                         tserver->uuid(), tserver->bound_rpc_hostport().ToString()),
+              out);
+  }
+
+  { // Pretty
+    string out;
+    NO_FATALS(RunActionStdoutString(
+          Substitute("tserver list $0 --columns=uuid,rpc-addresses", master_addr),
+          &out));
+
+    ASSERT_STR_CONTAINS(out, tserver->uuid());
+    ASSERT_STR_CONTAINS(out, tserver->bound_rpc_hostport().ToString());
+  }
+
+  { // Add a tserver
+    ASSERT_OK(cluster_->AddTabletServer());
+
+    vector<string> lines;
+    NO_FATALS(RunActionStdoutLines(
+          Substitute("tserver list $0 --columns=uuid --format=space", master_addr),
+          &lines));
+
+    vector<string> expected = {
+      tserver->uuid(),
+      cluster_->tablet_server(1)->uuid(),
+    };
+
+    std::sort(lines.begin(), lines.end());
+    std::sort(expected.begin(), expected.end());
+
+    ASSERT_EQ(expected, lines);
+  }
+}
+
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/tools/tool_action.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action.cc b/src/kudu/tools/tool_action.cc
index a88849e..4bef6e8 100644
--- a/src/kudu/tools/tool_action.cc
+++ b/src/kudu/tools/tool_action.cc
@@ -220,13 +220,17 @@ ActionBuilder& ActionBuilder::AddRequiredVariadicParameter(
   return *this;
 }
 
-ActionBuilder& ActionBuilder::AddOptionalParameter(const string& param) {
+ActionBuilder& ActionBuilder::AddOptionalParameter(string param,
+                                                   boost::optional<std::string> default_value,
+                                                   boost::optional<std::string> description) {
 #ifndef NDEBUG
   // Make sure this gflag exists.
   string option;
   DCHECK(google::GetCommandLineOption(param.c_str(), &option));
 #endif
-  args_.optional.push_back(param);
+  args_.optional.emplace_back(ActionArgsDescriptor::Flag({ std::move(param),
+                                                           std::move(default_value),
+                                                           std::move(description) }));
   return *this;
 }
 
@@ -244,10 +248,12 @@ unique_ptr<Action> ActionBuilder::Build() {
 Status Action::Run(const vector<Mode*>& chain,
                    const unordered_map<string, string>& required_args,
                    const vector<string>& variadic_args) const {
+  SetOptionalParameterDefaultValues();
   return runner_({ chain, this, required_args, variadic_args });
 }
 
 string Action::BuildHelp(const vector<Mode*>& chain) const {
+  SetOptionalParameterDefaultValues();
   string usage_msg = Substitute("Usage: $0 $1", BuildUsageString(chain), name());
   string desc_msg;
   for (const auto& param : args_.required) {
@@ -263,24 +269,28 @@ string Action::BuildHelp(const vector<Mode*>& chain) const {
   }
   for (const auto& param : args_.optional) {
     google::CommandLineFlagInfo gflag_info =
-        google::GetCommandLineFlagInfoOrDie(param.c_str());
+        google::GetCommandLineFlagInfoOrDie(param.name.c_str());
+
+    if (param.description) {
+      gflag_info.description = *param.description;
+    }
 
     if (gflag_info.type == "bool") {
       if (gflag_info.default_value == "false") {
-        usage_msg += Substitute(" [-$0]", param);
+        usage_msg += Substitute(" [-$0]", param.name);
       } else {
-        usage_msg += Substitute(" [-no$0]", param);
+        usage_msg += Substitute(" [-no$0]", param.name);
       }
     } else {
       string noun;
-      string::size_type last_underscore_idx = param.rfind('_');
+      string::size_type last_underscore_idx = param.name.rfind('_');
       if (last_underscore_idx != string::npos &&
-          last_underscore_idx != param.size() - 1) {
-        noun = param.substr(last_underscore_idx + 1);
+          last_underscore_idx != param.name.size() - 1) {
+        noun = param.name.substr(last_underscore_idx + 1);
       } else {
-        noun = param;
+        noun = param.name;
       }
-      usage_msg += Substitute(" [-$0=<$1>]", param, noun);
+      usage_msg += Substitute(" [-$0=<$1>]", param.name, noun);
     }
     desc_msg += google::DescribeOneFlag(gflag_info);
     desc_msg += "\n";
@@ -299,6 +309,7 @@ string Action::BuildHelp(const vector<Mode*>& chain) const {
 }
 
 string Action::BuildHelpXML(const vector<Mode*>& chain) const {
+  SetOptionalParameterDefaultValues();
   string usage = Substitute("$0 $1", BuildUsageString(chain), name());
   string xml;
   xml += "<action>";
@@ -333,24 +344,28 @@ string Action::BuildHelpXML(const vector<Mode*>& chain) const {
 
   for (const auto& o : args().optional) {
     google::CommandLineFlagInfo gflag_info =
-        google::GetCommandLineFlagInfoOrDie(o.c_str());
+        google::GetCommandLineFlagInfoOrDie(o.name.c_str());
+
+    if (o.description) {
+      gflag_info.description = *o.description;
+    }
 
     if (gflag_info.type == "bool") {
       if (gflag_info.default_value == "false") {
-        usage += Substitute(" [-$0]", o);
+        usage += Substitute(" [-$0]", o.name);
       } else {
-        usage += Substitute(" [-no$0]", o);
+        usage += Substitute(" [-no$0]", o.name);
       }
     } else {
       string noun;
-      string::size_type last_underscore_idx = o.rfind('_');
+      string::size_type last_underscore_idx = o.name.rfind('_');
       if (last_underscore_idx != string::npos &&
-          last_underscore_idx != o.size() - 1) {
-        noun = o.substr(last_underscore_idx + 1);
+          last_underscore_idx != o.name.size() - 1) {
+        noun = o.name.substr(last_underscore_idx + 1);
       } else {
-        noun = o;
+        noun = o.name;
       }
-      usage += Substitute(" [-$0=&lt;$1&gt;]", o, noun);
+      usage += Substitute(" [-$0=&lt;$1&gt;]", o.name, noun);
     }
 
     xml += "<argument>";
@@ -367,5 +382,15 @@ string Action::BuildHelpXML(const vector<Mode*>& chain) const {
   return xml;
 }
 
+void Action::SetOptionalParameterDefaultValues() const {
+  for (const auto& param : args_.optional) {
+    if (param.default_value) {
+      google::SetCommandLineOptionWithMode(param.name.c_str(),
+                                           param.default_value->c_str(),
+                                           google::FlagSettingMode::SET_FLAGS_DEFAULT);
+    }
+  }
+}
+
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/tools/tool_action.h
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action.h b/src/kudu/tools/tool_action.h
index 522b895..bf9e48a 100644
--- a/src/kudu/tools/tool_action.h
+++ b/src/kudu/tools/tool_action.h
@@ -161,17 +161,24 @@ struct ActionArgsDescriptor {
     std::string description;
   };
 
+  // Holds an optional command line argument flag.
+  struct Flag {
+    // The gflag name.
+    std::string name;
+    // A default value to override the default gflag value.
+    boost::optional<std::string> default_value;
+    // A description to override the gflag description.
+    boost::optional<std::string> description;
+  };
+
   // Positional (required) command line arguments.
   std::vector<Arg> required;
 
-  // Key-value command line arguments. These must actually implemented as
-  // gflags, which means all that must be specified here are the gflag names.
-  // The gflag definitions themselves will be accessed to get the argument
-  // descriptions.
+  // Key-value command line arguments. These must correspond to defined gflags.
   //
   // Optional by definition, though some are required internally
   // (e.g. fs_wal_dir).
-  std::vector<std::string> optional;
+  std::vector<Flag> optional;
 
   // Variable length command line argument. There may be at most one per
   // Action, and it's always found at the end of the command line.
@@ -221,7 +228,13 @@ class ActionBuilder {
   // provided by the user at any point in the command line. It must match a
   // previously-defined gflag; if a gflag with the same name cannot be found,
   // the tool will crash.
-  ActionBuilder& AddOptionalParameter(const std::string& param);
+  //
+  // The default value and description of the flag can be optionally overriden,
+  // for cases where the values are action-dependent. Otherwise, the default
+  // value and description from the gflag declaration will be used.
+  ActionBuilder& AddOptionalParameter(std::string param,
+                                      boost::optional<std::string> default_value = boost::none,
+                                      boost::optional<std::string> description = boost::none);
 
   // Creates an action using builder state.
   std::unique_ptr<Action> Build();
@@ -269,6 +282,10 @@ class Action {
 
   Action() = default;
 
+  // Sets optional flag parameter default value in cases where it has been
+  // overridden from the default gflag value.
+  void SetOptionalParameterDefaultValues() const;
+
   std::string name_;
 
   std::string description_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/tools/tool_action_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_common.cc b/src/kudu/tools/tool_action_common.cc
index 5f12072..7aa6702 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -17,17 +17,24 @@
 
 #include "kudu/tools/tool_action_common.h"
 
+#include <algorithm>
+#include <iomanip>
 #include <iostream>
 #include <memory>
+#include <numeric>
 #include <string>
 #include <vector>
 
+#include <boost/algorithm/string/predicate.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 
+#include "kudu/client/client-internal.h"
+#include "kudu/client/client.h"
+#include "kudu/client/shared_ptr.h"
 #include "kudu/common/common.pb.h"
-#include "kudu/common/schema.h"
 #include "kudu/common/row_operations.h"
+#include "kudu/common/schema.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/consensus.proxy.h"
@@ -35,14 +42,18 @@
 #include "kudu/consensus/log_util.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/master/master.proxy.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
+#include "kudu/tools/tool_action.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_admin.proxy.h"
 #include "kudu/tserver/tserver_service.proxy.h"
+#include "kudu/util/jsonwriter.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
@@ -66,14 +77,23 @@ DEFINE_int32(truncate_data, 100,
              "Truncate the data fields to the given number of bytes "
              "before printing. Set to 0 to disable");
 
+DEFINE_string(columns, "", "Comma-separated list of column fields to include in output tables");
+DEFINE_string(format, "pretty",
+              "Format to use for printing list output tables.\n"
+              "Possible values: pretty, space, tsv, csv, and json");
+
 namespace kudu {
 namespace tools {
 
+using client::KuduClientBuilder;
 using consensus::ConsensusServiceProxy;
 using consensus::ReplicateMsg;
 using log::LogEntryPB;
 using log::LogEntryReader;
 using log::ReadableLogSegment;
+using master::ListTabletServersRequestPB;
+using master::ListTabletServersResponsePB;
+using master::MasterServiceProxy;
 using rpc::Messenger;
 using rpc::MessengerBuilder;
 using rpc::RequestIdPB;
@@ -88,6 +108,8 @@ using server::SetFlagRequestPB;
 using server::SetFlagResponsePB;
 using std::cout;
 using std::endl;
+using std::setfill;
+using std::setw;
 using std::shared_ptr;
 using std::string;
 using std::unique_ptr;
@@ -236,6 +258,10 @@ template
 Status BuildProxy(const string& address,
                   uint16_t default_port,
                   unique_ptr<TabletServerAdminServiceProxy>* proxy);
+template
+Status BuildProxy(const string& address,
+                  uint16_t default_port,
+                  unique_ptr<MasterServiceProxy>* proxy);
 
 Status GetServerStatus(const string& address, uint16_t default_port,
                        ServerStatusPB* status) {
@@ -342,5 +368,158 @@ Status PrintServerTimestamp(const string& address, uint16_t default_port) {
   return Status::OK();
 }
 
+namespace {
+
+// Pretty print a table using the psql format. For example:
+//
+//                uuid               |         rpc-addresses          |      seqno
+// ----------------------------------+--------------------------------+------------------
+//  335d132897de4bdb9b87443f2c487a42 | 126.rack1.dc1.example.com:7050 | 1492596790237811
+//  7425c65d80f54f2da0a85494a5eb3e68 | 122.rack1.dc1.example.com:7050 | 1492596755322350
+//  dd23284d3a334f1a8306c19d89c1161f | 130.rack1.dc1.example.com:7050 | 1492596704536543
+//  d8009e07d82b4e66a7ab50f85e60bc30 | 136.rack1.dc1.example.com:7050 | 1492596696557549
+//  c108a85a68504c2bb9f49e4ee683d981 | 128.rack1.dc1.example.com:7050 | 1492596646623301
+void PrettyPrintTable(const vector<string>& headers, const vector<vector<string>>& columns) {
+  CHECK_EQ(headers.size(), columns.size());
+  if (headers.empty()) return;
+  size_t num_columns = headers.size();
+
+  vector<size_t> widths;
+  for (int col = 0; col < num_columns; col++) {
+    size_t width = std::accumulate(columns[col].begin(), columns[col].end(), headers[col].size(),
+                                   [](size_t acc, const string& cell) {
+                                     return std::max(acc, cell.size());
+                                   });
+    widths.push_back(width);
+  }
+
+  // Print the header row.
+  for (int col = 0; col < num_columns; col++) {
+    int padding = widths[col] - headers[col].size();
+    cout << setw(padding / 2) << "" << " " << headers[col];
+    if (col != num_columns - 1) cout << setw((padding + 1) / 2) << "" << " |";
+  }
+  cout << endl;
+
+  // Print the separator row.
+  for (int col = 0; col < num_columns; col++) {
+    cout << setfill('-') << setw(widths[col] + 2) << "";
+    if (col != num_columns - 1) cout << "+";
+  }
+  cout << endl;
+
+  // Print the data rows.
+  int num_rows = columns.empty() ? 0 : columns[0].size();
+  for (int row = 0; row < num_rows; row++) {
+    for (int col = 0; col < num_columns; col++) {
+      const auto& value = columns[col][row];
+      cout << " " << value;
+      if (col != num_columns - 1) {
+        size_t padding = value.size() - widths[col];
+        cout << setw(padding) << " |";
+      }
+    }
+    cout << endl;
+  }
+}
+
+// Print a table using JSON formatting.
+//
+// The table is formatted as an array of objects. Each object corresponds
+// to a row whose fields are the column values.
+void JsonPrintTable(const vector<string>& headers, const vector<vector<string>>& columns) {
+  std::ostringstream stream;
+  JsonWriter writer(&stream, JsonWriter::COMPACT);
+
+  int num_columns = columns.size();
+  int num_rows = columns.empty() ? 0 : columns[0].size();
+
+  writer.StartArray();
+  for (int row = 0; row < num_rows; row++) {
+    writer.StartObject();
+    for (int col = 0; col < num_columns; col++) {
+      writer.String(headers[col]);
+      writer.String(columns[col][row]);
+    }
+    writer.EndObject();
+  }
+  writer.EndArray();
+
+  cout << stream.str() << endl;
+}
+
+// Print the table using the provided separator. For example, with a comma
+// separator:
+//
+// 335d132897de4bdb9b87443f2c487a42,126.rack1.dc1.example.com:7050,1492596790237811
+// 7425c65d80f54f2da0a85494a5eb3e68,122.rack1.dc1.example.com:7050,1492596755322350
+// dd23284d3a334f1a8306c19d89c1161f,130.rack1.dc1.example.com:7050,1492596704536543
+// d8009e07d82b4e66a7ab50f85e60bc30,136.rack1.dc1.example.com:7050,1492596696557549
+// c108a85a68504c2bb9f49e4ee683d981,128.rack1.dc1.example.com:7050,1492596646623301
+void PrintTable(const vector<vector<string>>& columns, const string& separator) {
+  // TODO(dan): proper escaping of string values.
+  int num_columns = columns.size();
+  int num_rows = columns.empty() ? 0 : columns[0].size();
+  for (int row = 0; row < num_rows; row++) {
+      for (int col = 0; col < num_columns; col++) {
+        cout << columns[col][row];
+        if (col != num_columns - 1) cout << separator;
+      }
+      cout << endl;
+  }
+}
+
+} // anonymous namespace
+
+Status PrintTable(const vector<string>& headers, const vector<vector<string>>& columns) {
+  if (boost::iequals(FLAGS_format, "pretty")) {
+    PrettyPrintTable(headers, columns);
+  } else if (boost::iequals(FLAGS_format, "space")) {
+    PrintTable(columns, " ");
+  } else if (boost::iequals(FLAGS_format, "tsv")) {
+    PrintTable(columns, "	");
+  } else if (boost::iequals(FLAGS_format, "csv")) {
+    PrintTable(columns, ",");
+  } else if (boost::iequals(FLAGS_format, "json")) {
+    JsonPrintTable(headers, columns);
+  } else {
+    return Status::InvalidArgument("unknown format (--format)", FLAGS_format);
+  }
+  return Status::OK();
+}
+
+Status LeaderMasterProxy::Init(const RunnerContext& context) {
+  const string& master_addrs_str = FindOrDie(context.required_args, kMasterAddressesArg);
+  auto master_addrs = strings::Split(master_addrs_str, ",");
+  auto timeout = MonoDelta::FromMilliseconds(FLAGS_timeout_ms);
+
+  return KuduClientBuilder().master_server_addrs(master_addrs)
+                            .default_rpc_timeout(timeout)
+                            .default_admin_operation_timeout(timeout)
+                            .Build(&client_);
+}
+
+template<typename Req, typename Resp>
+Status LeaderMasterProxy::SyncRpc(const Req& req,
+                                  Resp* resp,
+                                  const char* func_name,
+                                  const boost::function<Status(master::MasterServiceProxy*,
+                                                               const Req&, Resp*,
+                                                               rpc::RpcController*)>& func) {
+  MonoTime deadline = MonoTime::Now() + MonoDelta::FromMilliseconds(FLAGS_timeout_ms);
+  return client_->data_->SyncLeaderMasterRpc(deadline, client_.get(), req, resp,
+                                             func_name, func, {});
+}
+
+// Explicit specialization for callers outside this compilation unit.
+template
+Status LeaderMasterProxy::SyncRpc(const ListTabletServersRequestPB& req,
+                                  ListTabletServersResponsePB* resp,
+                                  const char* func_name,
+                                  const boost::function<Status(MasterServiceProxy*,
+                                                               const ListTabletServersRequestPB&,
+                                                               ListTabletServersResponsePB*,
+                                                               RpcController*)>& func);
+
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/tools/tool_action_common.h
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_common.h b/src/kudu/tools/tool_action_common.h
index 27284f1..a0e39ef 100644
--- a/src/kudu/tools/tool_action_common.h
+++ b/src/kudu/tools/tool_action_common.h
@@ -19,12 +19,31 @@
 
 #include <memory>
 #include <string>
+#include <vector>
 
+#include <boost/function.hpp>
+
+#include "kudu/client/shared_ptr.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
 
+class MonoTime;
+
+namespace client {
+class KuduClient;
+} // namespace client
+
+
+namespace master {
+class MasterServiceProxy;
+} // namespace master
+
+namespace rpc {
+class RpcController;
+} // namespace rpc
+
 namespace log {
 class ReadableLogSegment;
 } // namespace log
@@ -35,6 +54,8 @@ class ServerStatusPB;
 
 namespace tools {
 
+struct RunnerContext;
+
 // Constants for parameters and descriptions.
 extern const char* const kMasterAddressesArg;
 extern const char* const kMasterAddressesArgDesc;
@@ -84,5 +105,32 @@ Status PrintServerTimestamp(const std::string& address, uint16_t default_port);
 Status SetServerFlag(const std::string& address, uint16_t default_port,
                      const std::string& flag, const std::string& value);
 
+// Prints a table.
+Status PrintTable(const std::vector<std::string>& headers,
+                  const std::vector<std::vector<std::string>>& columns);
+
+// Wrapper around a Kudu client which allows calling proxy methods on the leader
+// master.
+class LeaderMasterProxy {
+ public:
+  // 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.
+  Status Init(const RunnerContext& context);
+
+  // Calls a master RPC service method on the current leader master.
+  template<typename Req, typename Resp>
+  Status SyncRpc(const Req& req,
+                 Resp* resp,
+                 const char* func_name,
+                 const boost::function<Status(master::MasterServiceProxy*,
+                                              const Req&, Resp*,
+                                              rpc::RpcController*)>& func);
+
+ private:
+  client::sp::shared_ptr<client::KuduClient> client_;
+};
+
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/319a1bc1/src/kudu/tools/tool_action_tserver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_tserver.cc b/src/kudu/tools/tool_action_tserver.cc
index 54397f8..700092e 100644
--- a/src/kudu/tools/tool_action_tserver.cc
+++ b/src/kudu/tools/tool_action_tserver.cc
@@ -17,23 +17,37 @@
 
 #include "kudu/tools/tool_action.h"
 
+#include <functional>
 #include <memory>
 #include <string>
 #include <utility>
 
+#include <boost/algorithm/string/predicate.hpp>
 #include <gflags/gflags.h>
 
 #include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/master/master.h"
+#include "kudu/master/master.proxy.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/tserver/tablet_server.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 
-namespace kudu {
-namespace tools {
+DECLARE_string(columns);
 
 using std::string;
 using std::unique_ptr;
 
+namespace kudu {
+
+using master::ListTabletServersRequestPB;
+using master::ListTabletServersResponsePB;
+using master::MasterServiceProxy;
+
+namespace tools {
 namespace {
 
 const char* const kTServerAddressArg = "tserver_address";
@@ -61,6 +75,67 @@ Status TServerTimestamp(const RunnerContext& context) {
   return PrintServerTimestamp(address, tserver::TabletServer::kDefaultPort);
 }
 
+Status ListTServers(const RunnerContext& context) {
+  LeaderMasterProxy proxy;
+  RETURN_NOT_OK(proxy.Init(context));
+
+  ListTabletServersRequestPB req;
+  ListTabletServersResponsePB resp;
+
+  proxy.SyncRpc<ListTabletServersRequestPB, ListTabletServersResponsePB>(
+      req, &resp, "ListTabletServers", &MasterServiceProxy::ListTabletServers);
+
+  if (resp.has_error()) {
+    return StatusFromPB(resp.error().status());
+  }
+
+  vector<string> headers;
+  vector<vector<string>> columns;
+
+  const auto& servers = resp.servers();
+
+  auto hostport_to_string = [](const HostPortPB& hostport) {
+    return strings::Substitute("$0:$1", hostport.host(), hostport.port());
+  };
+
+  for (const auto& column : strings::Split(FLAGS_columns, ",", strings::SkipEmpty())) {
+    headers.emplace_back(column.ToString());
+    vector<string> values;
+    if (boost::iequals(column, "uuid")) {
+      for (const auto& server : servers) {
+        values.push_back(server.instance_id().permanent_uuid());
+      }
+    } else if (boost::iequals(column, "seqno")) {
+      for (const auto& server : servers) {
+        values.push_back(std::to_string(server.instance_id().instance_seqno()));
+      }
+    } else if (boost::iequals(column, "rpc-addresses") ||
+               boost::iequals(column, "rpc_addresses")) {
+      for (const auto& server : servers) {
+        values.push_back(JoinMapped(server.registration().rpc_addresses(),
+                                    hostport_to_string, ","));
+      }
+    } else if (boost::iequals(column, "http-addresses") ||
+               boost::iequals(column, "http_addresses")) {
+      for (const auto& server : servers) {
+        values.push_back(JoinMapped(server.registration().http_addresses(),
+                                    hostport_to_string, ","));
+      }
+    } else if (boost::iequals(column, "version")) {
+      for (const auto& server : servers) {
+        values.push_back(server.registration().software_version());
+      }
+    } else {
+      return Status::InvalidArgument("unknown column (--columns)", column);
+    }
+
+    columns.emplace_back(std::move(values));
+  }
+
+  RETURN_NOT_OK(PrintTable(headers, columns));
+  return Status::OK();
+}
+
 } // anonymous namespace
 
 unique_ptr<Mode> BuildTServerMode() {
@@ -85,11 +160,24 @@ unique_ptr<Mode> BuildTServerMode() {
       .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
 
+  unique_ptr<Action> list_tservers =
+      ActionBuilder("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, and seqno"))
+      .AddOptionalParameter("format")
+      .AddOptionalParameter("timeout_ms")
+      .Build();
+
   return ModeBuilder("tserver")
       .Description("Operate on a Kudu Tablet Server")
       .AddAction(std::move(set_flag))
       .AddAction(std::move(status))
       .AddAction(std::move(timestamp))
+      .AddAction(std::move(list_tservers))
       .Build();
 }