You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2018/09/05 16:00:39 UTC

[3/3] kudu git commit: KUDU-2529: Add a "-tables=" flag to the "kudu table list".

KUDU-2529: Add a "-tables=<tables>" flag to the "kudu table list".

"-tables=<tables>" flag can help to filter the tables that you want
  while running "kudu table list <master> -list_tables".

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


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

Branch: refs/heads/master
Commit: 4fceeae3f314ef794bd58c47439033941a1aa568
Parents: 6b70c27
Author: helifu <hz...@corp.netease.com>
Authored: Fri Aug 24 17:14:19 2018 +0800
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Wed Sep 5 16:00:18 2018 +0000

----------------------------------------------------------------------
 src/kudu/tools/ksck.cc                |  14 +---
 src/kudu/tools/kudu-tool-test.cc      | 127 ++++++++++++++++++++++++++++-
 src/kudu/tools/tool_action_cluster.cc |   6 +-
 src/kudu/tools/tool_action_common.cc  |  13 +++
 src/kudu/tools/tool_action_common.h   |   4 +
 src/kudu/tools/tool_action_table.cc   |   7 +-
 6 files changed, 152 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4fceeae3/src/kudu/tools/ksck.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/ksck.cc b/src/kudu/tools/ksck.cc
index eab1ac1..c426d38 100644
--- a/src/kudu/tools/ksck.cc
+++ b/src/kudu/tools/ksck.cc
@@ -39,9 +39,9 @@
 #include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/gutil/strings/util.h"
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/tools/color.h"
+#include "kudu/tools/tool_action_common.h"
 #include "kudu/util/atomic.h"
 #include "kudu/util/blocking_queue.h"
 #include "kudu/util/countdown_latch.h"
@@ -99,18 +99,6 @@ namespace kudu {
 namespace tools {
 
 namespace {
-// Return true if 'str' matches any of the patterns in 'patterns', or if
-// 'patterns' is empty.
-bool MatchesAnyPattern(const vector<string>& patterns, const string& str) {
-  // Consider no filter a wildcard.
-  if (patterns.empty()) return true;
-
-  for (const auto& p : patterns) {
-    if (MatchPattern(str, p)) return true;
-  }
-  return false;
-}
-
 void BuildKsckConsensusStateForConfigMember(const consensus::ConsensusStatePB& cstate,
                                             KsckConsensusState* ksck_cstate) {
   CHECK(ksck_cstate);

http://git-wip-us.apache.org/repos/asf/kudu/blob/4fceeae3/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 28180c4..8669695 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -140,6 +140,8 @@ using kudu::client::KuduClient;
 using kudu::client::KuduClientBuilder;
 using kudu::client::KuduSchema;
 using kudu::client::KuduSchemaBuilder;
+using kudu::client::KuduScanToken;
+using kudu::client::KuduScanTokenBuilder;
 using kudu::client::KuduTable;
 using kudu::client::sp::shared_ptr;
 using kudu::cluster::ExternalMiniCluster;
@@ -176,6 +178,7 @@ using kudu::tserver::MiniTabletServer;
 using kudu::tserver::WriteRequestPB;
 using std::back_inserter;
 using std::copy;
+using std::map;
 using std::make_pair;
 using std::ostringstream;
 using std::pair;
@@ -581,7 +584,9 @@ TEST_F(ToolTest, TestModeHelp) {
   {
     const vector<string> kTableModeRegexes = {
         "delete.*Delete a table",
-        "list.*List all tables",
+        "rename_table.*Rename a table",
+        "rename_column.*Rename a column",
+        "list.*List tables",
     };
     NO_FATALS(RunTestHelp("table", kTableModeRegexes));
   }
@@ -2117,6 +2122,39 @@ TEST_F(ToolTest, TestMasterList) {
   ASSERT_STR_CONTAINS(out, master->bound_rpc_hostport().ToString());
 }
 
+// Operate on Kudu tables:
+// (1)delete a table
+// (2)rename a table
+// (3)rename a column
+// (4)list tables
+TEST_F(ToolTest, TestDeleteTable) {
+  NO_FATALS(StartExternalMiniCluster());
+  shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+  string master_addr = cluster_->master()->bound_rpc_addr().ToString();
+
+  const string& kTableName = "kudu.table";
+
+  // Create a table.
+  TestWorkload workload(cluster_.get());
+  workload.set_table_name(kTableName);
+  workload.set_num_replicas(1);
+  workload.Setup();
+
+  // Check that the table exists.
+  bool exist = false;
+  ASSERT_OK(client->TableExists(kTableName, &exist));
+  ASSERT_EQ(exist, true);
+
+  // Delete the table.
+  NO_FATALS(RunActionStdoutNone(Substitute("table delete $0 $1",
+                                           master_addr, kTableName)));
+
+  // Check that the table does not exist.
+  ASSERT_OK(client->TableExists(kTableName, &exist));
+  ASSERT_EQ(exist, false);
+}
+
 TEST_F(ToolTest, TestRenameTable) {
   NO_FATALS(StartExternalMiniCluster());
   shared_ptr<KuduClient> client;
@@ -2183,6 +2221,93 @@ TEST_F(ToolTest, TestRenameColumn) {
   ASSERT_STR_CONTAINS(table->schema().ToString(), kNewColumnName);
 }
 
+TEST_F(ToolTest, TestListTables) {
+  NO_FATALS(StartExternalMiniCluster());
+  shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+  string master_addr = cluster_->master()->bound_rpc_addr().ToString();
+
+  // Replica's format.
+  string ts_uuid = cluster_->tablet_server(0)->uuid();
+  string ts_addr = cluster_->tablet_server(0)->bound_rpc_addr().ToString();
+  string expect_replica = Substitute("    L $0 $1", ts_uuid, ts_addr);
+
+  // Create some tables.
+  const int kNumTables = 10;
+  vector<string> table_names;
+  for (int i = 0; i < kNumTables; ++i) {
+    string table_name = Substitute("kudu.table_$0", i);
+    table_names.push_back(table_name);
+
+    TestWorkload workload(cluster_.get());
+    workload.set_table_name(table_name);
+    workload.set_num_replicas(1);
+    workload.Setup();
+  }
+  std::sort(table_names.begin(), table_names.end());
+
+  const auto& ProcessTables = [&] (const int num) {
+    ASSERT_GE(num, 1);
+    ASSERT_LE(num, kNumTables);
+
+    vector<string> expected;
+    expected.insert(expected.end(), table_names.begin(), table_names.begin() + num);
+
+    string filter = "";
+    if (kNumTables != num) {
+      filter = Substitute("-tables=$0", JoinStrings(expected, ","));
+    }
+    vector<string> lines;
+    NO_FATALS(RunActionStdoutLines(
+        Substitute("table list $0 $1", master_addr, filter), &lines));
+
+    std::sort(lines.begin(), lines.end());
+    ASSERT_EQ(expected, lines);
+  };
+
+  const auto& ProcessTablets = [&] (const int num) {
+    ASSERT_GE(num, 1);
+    ASSERT_LE(num, kNumTables);
+
+    string filter = "";
+    if (kNumTables != num) {
+      filter = Substitute("-tables=$0",
+        JoinStringsIterator(table_names.begin(), table_names.begin() + num, ","));
+    }
+    vector<string> lines;
+    NO_FATALS(RunActionStdoutLines(
+        Substitute("table list $0 $1 -list_tablets", master_addr, filter), &lines));
+
+    map<string, pair<string, string>> output;
+    for (int i = 0; i < lines.size(); ++i) {
+      if (lines[i].empty()) continue;
+      ASSERT_LE(i + 2, lines.size());
+      output[lines[i]] = pair<string, string>(lines[i + 1], lines[i + 2]);
+      i += 2;
+    }
+
+    for (const auto& e : output) {
+      shared_ptr<KuduTable> table;
+      ASSERT_OK(client->OpenTable(e.first, &table));
+      vector<KuduScanToken*> tokens;
+      ElementDeleter deleter(&tokens);
+      KuduScanTokenBuilder builder(table.get());
+      ASSERT_OK(builder.Build(&tokens));
+      ASSERT_EQ(1, tokens.size()); // Only one partition(tablet) under table.
+      // Tablet's format.
+      string expect_tablet = Substitute("  T $0", tokens[0]->tablet().id());
+      ASSERT_EQ(expect_tablet, e.second.first);
+      ASSERT_EQ(expect_replica, e.second.second);
+    }
+  };
+
+  // List the tables and tablets.
+  for (int i = 1; i <= kNumTables; ++i) {
+    ProcessTables(i);
+    ProcessTablets(i);
+  }
+}
+
 Status CreateLegacyHmsTable(HmsClient* client,
                             const string& hms_database_name,
                             const string& hms_table_name,

http://git-wip-us.apache.org/repos/asf/kudu/blob/4fceeae3/src/kudu/tools/tool_action_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_cluster.cc b/src/kudu/tools/tool_action_cluster.cc
index 8fccd84..b6a6981 100644
--- a/src/kudu/tools/tool_action_cluster.cc
+++ b/src/kudu/tools/tool_action_cluster.cc
@@ -28,6 +28,7 @@
 #include <boost/algorithm/string/predicate.hpp>
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/gutil/basictypes.h"
@@ -62,10 +63,7 @@ using strings::Substitute;
   } \
 } while (0);
 
-DEFINE_string(tables, "",
-              "Tables to check (comma-separated list of names). "
-              "If not specified, checks all tables.");
-
+DECLARE_string(tables);
 DEFINE_string(tablets, "",
               "Tablets to check (comma-separated list of IDs) "
               "If not specified, checks all tablets.");

http://git-wip-us.apache.org/repos/asf/kudu/blob/4fceeae3/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 db86b02..e615915 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -56,6 +56,7 @@
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
 #include "kudu/master/master.proxy.h" // IWYU pragma: keep
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/rpc_controller.h"
@@ -101,6 +102,8 @@ DEFINE_string(flag_tags, "", "Comma-separated list of tags used to restrict whic
                              "flags are returned. An empty value matches all tags");
 DEFINE_bool(all_flags, false, "Whether to return all flags, or only flags that "
                               "were explicitly set.");
+DEFINE_string(tables, "", "Tables to include (comma-separated list of table names)"
+                          "If not specified, includes all tables.");
 
 namespace boost {
 template <typename Signature>
@@ -449,6 +452,16 @@ string GetMasterAddresses(const client::KuduClient& client) {
   return HostPort::ToCommaSeparatedString(client.data_->master_hostports());
 }
 
+bool MatchesAnyPattern(const vector<string>& patterns, const string& str) {
+  // Consider no filter a wildcard.
+  if (patterns.empty()) return true;
+
+  for (const auto& p : patterns) {
+    if (MatchPattern(str, p)) return true;
+  }
+  return false;
+}
+
 Status PrintServerStatus(const string& address, uint16_t default_port) {
   ServerStatusPB status;
   RETURN_NOT_OK(GetServerStatus(address, default_port, &status));

http://git-wip-us.apache.org/repos/asf/kudu/blob/4fceeae3/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 3043bd8..d3a4524 100644
--- a/src/kudu/tools/tool_action_common.h
+++ b/src/kudu/tools/tool_action_common.h
@@ -140,6 +140,10 @@ void SetAlterExternalCatalogs(client::KuduTableAlterer* alterer, bool alter_exte
 // Get the configured master addresses on the most recently connected to leader master.
 std::string GetMasterAddresses(const client::KuduClient& client);
 
+// Return true if 'str' matches any of the patterns in 'patterns', or if
+// 'patterns' is empty.
+bool MatchesAnyPattern(const std::vector<std::string>& patterns, const std::string& str);
+
 // A table of data to present to the user.
 //
 // Supports formatting based on the --format flag.

http://git-wip-us.apache.org/repos/asf/kudu/blob/4fceeae3/src/kudu/tools/tool_action_table.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 75d3a61..69521c2 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -23,6 +23,7 @@
 #include <vector>
 
 #include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
 
 #include "kudu/client/client.h"
 #include "kudu/client/replica_controller-internal.h"
@@ -36,6 +37,7 @@
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/util/status.h"
 
+DECLARE_string(tables);
 DEFINE_bool(alter_external_catalogs, true,
             "Whether to alter external catalogs, such as the Hive Metastore, "
             "when renaming a table.");
@@ -73,7 +75,9 @@ class TableLister {
     vector<string> table_names;
     RETURN_NOT_OK(client->ListTables(&table_names));
 
+    vector<string> table_filters = Split(FLAGS_tables, ",", strings::SkipEmpty());
     for (const auto& tname : table_names) {
+      if (!MatchesAnyPattern(table_filters, tname)) continue;
       cout << tname << endl;
       if (!FLAGS_list_tablets) {
         continue;
@@ -186,8 +190,9 @@ unique_ptr<Mode> BuildTableMode() {
 
   unique_ptr<Action> list_tables =
       ActionBuilder("list", &ListTables)
-      .Description("List all tables")
+      .Description("List tables")
       .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
+      .AddOptionalParameter("tables")
       .AddOptionalParameter("list_tablets")
       .Build();