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 2021/10/26 23:13:02 UTC

[kudu] branch master updated: [client] fix recently introduced ABI compatiblity issue

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 9e70182  [client] fix recently introduced ABI compatiblity issue
9e70182 is described below

commit 9e70182d28bebddc7adcbac60a0482225382c27a
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Oct 25 19:14:56 2021 -0700

    [client] fix recently introduced ABI compatiblity issue
    
    This patch fixes the recently introduced ABI compatibility issue in Kudu
    C++ client once the KuduClient::ListTables() method was overloaded
    (see [1] for details).  As I could see, the overloaded method was only
    used in the Kudu CLI tooling with no apparent justification why it's
    needed in the public client API.  With that, I moved the method to
    be a part of the KuduClient::Data internal class.
    
    In addition, I renamed --list_statistics into --show_table_info for the
    `kudu table list` CLI tool and switched to the lower_camel_case for
    the tags used to output a table's stats.
    
    This is a follow-up to 3ccc434ea6dcbef15a12bd1734c8e66b69a611c5.
    
    [1] https://community.kde.org/Policies/Binary_Compatibility_Issues_With_C%2B%2B
    
    Change-Id: I214d424eb0e91b90db813fde0e0b7150253bca91
    Reviewed-on: http://gerrit.cloudera.org:8080/17972
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/client/client-internal.cc  | 33 ++++++++++++++++++++
 src/kudu/client/client-internal.h   | 10 ++++++
 src/kudu/client/client.cc           | 62 +++----------------------------------
 src/kudu/client/client.h            | 18 ++---------
 src/kudu/tools/kudu-tool-test.cc    | 43 +++++++++++++------------
 src/kudu/tools/tool_action_table.cc | 26 +++++++++-------
 6 files changed, 88 insertions(+), 104 deletions(-)

diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index ef65685..99934e2 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -93,6 +93,8 @@ using kudu::master::IsAlterTableDoneRequestPB;
 using kudu::master::IsAlterTableDoneResponsePB;
 using kudu::master::IsCreateTableDoneRequestPB;
 using kudu::master::IsCreateTableDoneResponsePB;
+using kudu::master::ListTablesRequestPB;
+using kudu::master::ListTablesResponsePB;
 using kudu::master::ListTabletServersRequestPB;
 using kudu::master::ListTabletServersResponsePB;
 using kudu::master::MasterFeatures;
@@ -432,6 +434,37 @@ Status KuduClient::Data::WaitForAlterTableToFinish(
       });
 }
 
+Status KuduClient::Data::ListTablesWithInfo(KuduClient* client,
+                                            vector<TableInfo>* tables_info,
+                                            const string& filter) {
+  ListTablesRequestPB req;
+  if (!filter.empty()) {
+    req.set_name_filter(filter);
+  }
+
+  auto deadline = MonoTime::Now() + client->default_admin_operation_timeout();
+  Synchronizer sync;
+  ListTablesResponsePB resp;
+  AsyncLeaderMasterRpc<ListTablesRequestPB, ListTablesResponsePB> rpc(
+      deadline, client, BackoffType::EXPONENTIAL, req, &resp,
+      &MasterServiceProxy::ListTablesAsync, "ListTables",
+      sync.AsStatusCallback(), {});
+  rpc.SendRpc();
+  RETURN_NOT_OK(sync.Wait());
+  if (resp.has_error()) {
+    return StatusFromPB(resp.error().status());
+  }
+  for (const auto& table : resp.tables()) {
+    TableInfo info;
+    info.table_name = table.name();
+    info.live_row_count = table.has_live_row_count() ? table.live_row_count() : 0;
+    info.num_tablets = table.has_num_tablets() ? table.num_tablets() : 0;
+    info.num_replicas = table.has_num_replicas() ? table.num_replicas() : 0;
+    tables_info->emplace_back(std::move(info));
+  }
+  return Status::OK();
+}
+
 Status KuduClient::Data::InitLocalHostNames() {
   // Currently, we just use our configured hostname, and resolve it to come up with
   // a list of potentially local hosts. It would be better to iterate over all of
diff --git a/src/kudu/client/client-internal.h b/src/kudu/client/client-internal.h
index 2971f51..deab47d 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -136,6 +136,16 @@ class KuduClient::Data {
                                           master::TableIdentifierPB table,
                                           const MonoTime& deadline);
 
+  struct TableInfo {
+    std::string table_name;
+    uint64_t live_row_count;
+    int num_tablets;
+    int num_replicas;
+  };
+  static Status ListTablesWithInfo(KuduClient* client,
+                                   std::vector<TableInfo>* tables_info,
+                                   const std::string& filter);
+
   // Open the table identified by 'table_identifier'.
   Status OpenTable(KuduClient* client,
                    const master::TableIdentifierPB& table_identifier,
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index ba1bbf9..757a598 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -111,16 +111,10 @@ using kudu::master::AlterTableRequestPB;
 using kudu::master::AlterTableResponsePB;
 using kudu::master::CreateTableRequestPB;
 using kudu::master::CreateTableResponsePB;
-using kudu::master::DeleteTableRequestPB;
-using kudu::master::DeleteTableResponsePB;
-using kudu::master::GetTableSchemaRequestPB;
-using kudu::master::GetTableSchemaResponsePB;
 using kudu::master::GetTableStatisticsRequestPB;
 using kudu::master::GetTableStatisticsResponsePB;
 using kudu::master::GetTabletLocationsRequestPB;
 using kudu::master::GetTabletLocationsResponsePB;
-using kudu::master::ListTablesRequestPB;
-using kudu::master::ListTablesResponsePB;
 using kudu::master::ListTabletServersRequestPB;
 using kudu::master::ListTabletServersResponsePB;
 using kudu::master::ListTabletServersResponsePB_Entry;
@@ -567,57 +561,11 @@ Status KuduClient::ListTabletServers(vector<KuduTabletServer*>* tablet_servers)
   return Status::OK();
 }
 
-Status KuduClient::ListTables(vector<string>* tables,
-                              const string& filter) {
-  ListTablesRequestPB req;
-  ListTablesResponsePB resp;
-
-  if (!filter.empty()) {
-    req.set_name_filter(filter);
-  }
-  MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout();
-  Synchronizer sync;
-  AsyncLeaderMasterRpc<ListTablesRequestPB, ListTablesResponsePB> rpc(
-      deadline, this, BackoffType::EXPONENTIAL, req, &resp,
-      &MasterServiceProxy::ListTablesAsync, "ListTables",
-      sync.AsStatusCallback(), {});
-  rpc.SendRpc();
-  RETURN_NOT_OK(sync.Wait());
-  if (resp.has_error()) {
-    return StatusFromPB(resp.error().status());
-  }
-  for (int i = 0; i < resp.tables_size(); i++) {
-    tables->push_back(resp.tables(i).name());
-  }
-  return Status::OK();
-}
-
-Status KuduClient::ListTables(std::vector<ListTableInfo>* list_table_infos,
-                              const std::string& filter) {
-  ListTablesRequestPB req;
-  ListTablesResponsePB resp;
-
-  if (!filter.empty()) {
-    req.set_name_filter(filter);
-  }
-  MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout();
-  Synchronizer sync;
-  AsyncLeaderMasterRpc<ListTablesRequestPB, ListTablesResponsePB> rpc(
-    deadline, this, BackoffType::EXPONENTIAL, req, &resp,
-    &MasterServiceProxy::ListTablesAsync, "ListTables",
-    sync.AsStatusCallback(), {});
-  rpc.SendRpc();
-    RETURN_NOT_OK(sync.Wait());
-  if (resp.has_error()) {
-    return StatusFromPB(resp.error().status());
-  }
-  for (const auto& table : resp.tables()) {
-    ListTableInfo list_table_info;
-    list_table_info.table_name = table.name();
-    list_table_info.live_row_count = table.has_live_row_count() ? table.live_row_count() : 0;
-    list_table_info.num_tablets = table.has_num_tablets() ? table.num_tablets() : 0;
-    list_table_info.num_replicas = table.has_num_replicas() ? table.num_replicas() : 0;
-    list_table_infos->emplace_back(std::move(list_table_info));
+Status KuduClient::ListTables(vector<string>* tables, const string& filter) {
+  vector<Data::TableInfo> tables_info;
+  RETURN_NOT_OK(data_->ListTablesWithInfo(this, &tables_info, filter));
+  for (auto& info : tables_info) {
+    tables->emplace_back(std::move(info.table_name));
   }
   return Status::OK();
 }
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 243865d..ff3910b 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -83,6 +83,7 @@ namespace tools {
 class LeaderMasterProxy;
 class RemoteKsckCluster;
 class TableAlter;
+class TableLister;
 } // namespace tools
 
 namespace client {
@@ -747,22 +748,6 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   Status ListTables(std::vector<std::string>* tables,
                     const std::string& filter = "");
 
-  struct KUDU_EXPORT ListTableInfo {
-    std::string table_name;
-    uint64_t live_row_count;
-    int num_tablets;
-    int num_replicas;
-  };
-  /// List only those tables whose names pass a substring match on @c filter.
-  ///
-  /// @param [out] list_table_infos
-  ///   The placeholder for the result. Appended only on success.
-  /// @param [in] filter
-  ///   Substring filter to use; empty sub-string filter matches all tables.
-  /// @return Status object for the operation.
-  Status ListTables(std::vector<ListTableInfo>* list_table_infos,
-                    const std::string& filter = "");
-
   /// Check if the table given by 'table_name' exists.
   ///
   /// @param [in] table_name
@@ -1009,6 +994,7 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   friend class transactions::TxnSystemClient;
   friend class tools::LeaderMasterProxy;
   friend class tools::RemoteKsckCluster;
+  friend class tools::TableLister;
 
   FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
   FRIEND_TEST(ClientTest, ClearCacheAndConcurrentWorkload);
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index a0ba47f..0c60b9f 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -4088,9 +4088,9 @@ TEST_F(ToolTest, TestListTables) {
   string expect_replica = Substitute("    L $0 $1", ts_uuid, ts_addr);
 
   // Create some tables.
-  const int kNumTables = 10;
+  constexpr int kNumTables = 10;
+  constexpr int kReplicaNum = 1;
   vector<string> table_names;
-  const int kReplicaNum = 1;
   for (int i = 0; i < kNumTables; ++i) {
     string table_name = Substitute("kudu.table_$0", i);
     table_names.push_back(table_name);
@@ -4102,16 +4102,17 @@ TEST_F(ToolTest, TestListTables) {
   }
   std::sort(table_names.begin(), table_names.end());
 
-  const auto& ProcessTables = [&] (const int num) {
+  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);
+    expected.insert(expected.end(),
+                    table_names.begin(), table_names.begin() + num);
 
     string filter = "";
     if (kNumTables != num) {
-      filter = Substitute("-tables=$0", JoinStrings(expected, ","));
+      filter = Substitute("--tables=$0", JoinStrings(expected, ","));
     }
     vector<string> lines;
     NO_FATALS(RunActionStdoutLines(
@@ -4121,22 +4122,24 @@ TEST_F(ToolTest, TestListTables) {
     ASSERT_EQ(expected, lines);
   };
 
-  const auto& ProcessTablets = [&] (const int num) {
+  const auto ProcessTablets = [&] (const int num) {
     ASSERT_GE(num, 1);
     ASSERT_LE(num, kNumTables);
 
     string filter = "";
     if (kNumTables != num) {
-      filter = Substitute("-tables=$0",
+      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));
+        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;
+      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;
@@ -4157,26 +4160,28 @@ TEST_F(ToolTest, TestListTables) {
     }
   };
 
-  const auto& ProcessTablesStatistics = [&] (const int num) {
+  const auto ProcessTablesStatistics = [&] (const int num) {
     ASSERT_GE(num, 1);
     ASSERT_LE(num, kNumTables);
 
     vector<string> expected;
     expected.reserve(num);
     for (int i = 0; i < num; i++) {
-      expected.push_back(Substitute("$0 Num_Tablets:1 Num_Replicas:$1 Live_Row_Count:0",
-                                    table_names[i], kReplicaNum));
+      expected.emplace_back(
+          Substitute("$0 num_tablets:1 num_replicas:$1 live_row_count:0",
+                     table_names[i], kReplicaNum));
     }
     vector<string> expected_table;
-    expected_table.insert(expected_table.end(), table_names.begin(), table_names.begin() + num);
-
+    expected_table.reserve(num);
+    expected_table.insert(expected_table.end(),
+                          table_names.begin(), table_names.begin() + num);
     string filter = "";
     if (kNumTables != num) {
-      filter = Substitute("-tables=$0", JoinStrings(expected_table, ","));
+      filter = Substitute("--tables=$0", JoinStrings(expected_table, ","));
     }
     vector<string> lines;
     NO_FATALS(RunActionStdoutLines(
-      Substitute("table list $0 $1 -list_statistics", master_addr, filter), &lines));
+        Substitute("table list $0 $1 --show_table_info", master_addr, filter), &lines));
 
     std::sort(lines.begin(), lines.end());
     ASSERT_EQ(expected, lines);
@@ -4184,9 +4189,9 @@ TEST_F(ToolTest, TestListTables) {
 
   // List the tables and tablets.
   for (int i = 1; i <= kNumTables; ++i) {
-    ProcessTables(i);
-    ProcessTablets(i);
-    ProcessTablesStatistics(i);
+    NO_FATALS(ProcessTables(i));
+    NO_FATALS(ProcessTablets(i));
+    NO_FATALS(ProcessTablesStatistics(i));
   }
 }
 
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 77e2f75..3adab58 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -36,6 +36,7 @@
 #include <google/protobuf/util/json_util.h>
 #include <rapidjson/document.h>
 
+#include "kudu/client/client-internal.h"
 #include "kudu/client/client.h"
 #include "kudu/client/replica_controller-internal.h"
 #include "kudu/client/table_alterer-internal.h"
@@ -103,9 +104,9 @@ DEFINE_string(dst_table, "",
               "If the empty string, use the same name as the source table.");
 DEFINE_bool(list_tablets, false,
             "Include tablet and replica UUIDs in the output");
-DEFINE_bool(list_statistics, false,
-            "Include statistics such as number of tablets, replicas"
-            "and live row count in the output");
+DEFINE_bool(show_table_info, false,
+            "Include extra information such as number of tablets, replicas, "
+            "and live row count for a table in the output");
 DEFINE_bool(modify_external_catalogs, true,
             "Whether to modify external catalogs, such as the Hive Metastore, "
             "when renaming or dropping a table.");
@@ -142,17 +143,18 @@ class TableLister {
     RETURN_NOT_OK(CreateKuduClient(master_addresses,
                                    &client,
                                    true /* can_see_all_replicas */));
-    std::vector<kudu::client::KuduClient::ListTableInfo> list_table_infos;
-    RETURN_NOT_OK(client->ListTables(&list_table_infos));
+    vector<kudu::client::KuduClient::Data::TableInfo> tables_info;
+    RETURN_NOT_OK(client->data_->ListTablesWithInfo(
+        client.get(), &tables_info, "" /* filter */));
 
     vector<string> table_filters = Split(FLAGS_tables, ",", strings::SkipEmpty());
-    for (const auto& list_table_info : list_table_infos) {
-      const auto& tname = list_table_info.table_name;
+    for (const auto& tinfo : tables_info) {
+      const auto& tname = tinfo.table_name;
       if (!MatchesAnyPattern(table_filters, tname)) continue;
-      if (FLAGS_list_statistics) {
-        cout << tname << " " << "Num_Tablets:" << list_table_info.num_tablets
-             << " Num_Replicas:" << list_table_info.num_replicas
-             << " Live_Row_Count:" << list_table_info.live_row_count << endl;
+      if (FLAGS_show_table_info) {
+        cout << tname << " " << "num_tablets:" << tinfo.num_tablets
+             << " num_replicas:" << tinfo.num_replicas
+             << " live_row_count:" << tinfo.live_row_count << endl;
       } else {
         cout << tname << endl;
       }
@@ -1327,7 +1329,7 @@ unique_ptr<Mode> BuildTableMode() {
       .Description("List tables")
       .AddOptionalParameter("tables")
       .AddOptionalParameter("list_tablets")
-      .AddOptionalParameter("list_statistics")
+      .AddOptionalParameter("show_table_info")
       .Build();
 
   unique_ptr<Action> locate_row =