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 2018/10/19 21:12:00 UTC

kudu git commit: Use preferred private client API method for a couple of tools functions

Repository: kudu
Updated Branches:
  refs/heads/master b3486d99a -> ef714838a


Use preferred private client API method for a couple of tools functions

Change-Id: Ic5bc5e59c9e8a77faa99ca0be11db19eb04dbcb9
Reviewed-on: http://gerrit.cloudera.org:8080/11736
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Will Berkeley <wd...@gmail.com>


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

Branch: refs/heads/master
Commit: ef714838a8ba4e2a0dbf1d2f7998f8909d591402
Parents: b3486d9
Author: Will Berkeley <wd...@gmail.org>
Authored: Fri Oct 19 10:28:20 2018 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Fri Oct 19 20:19:44 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client.cc            | 38 ++++++++++++++++++++++++++
 src/kudu/client/client.h             | 32 ++++++++++++++++------
 src/kudu/tools/tool_action_common.cc | 45 -------------------------------
 src/kudu/tools/tool_action_common.h  |  9 -------
 src/kudu/tools/tool_action_hms.cc    |  2 +-
 src/kudu/tools/tool_action_table.cc  |  4 +--
 6 files changed, 65 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/ef714838/src/kudu/client/client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 1986257..d301105 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -85,6 +85,7 @@
 #include "kudu/security/token.pb.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_service.proxy.h"
+#include "kudu/util/async_util.h"
 #include "kudu/util/debug-util.h"
 #include "kudu/util/init.h"
 #include "kudu/util/logging.h"
@@ -598,6 +599,10 @@ Status KuduClient::GetTablet(const string& tablet_id, KuduTablet** tablet) {
   return Status::OK();
 }
 
+string KuduClient::GetMasterAddresses() const {
+  return HostPort::ToCommaSeparatedString(data_->master_hostports());
+}
+
 bool KuduClient::IsMultiMaster() const {
   return data_->master_server_addrs_.size() > 1;
 }
@@ -926,6 +931,39 @@ KuduPredicate* KuduTable::NewIsNullPredicate(const Slice& col_name) {
   });
 }
 
+// The strategy for retrieving the partitions from the metacache is adapted
+// from KuduScanTokenBuilder::Data::Build.
+Status KuduTable::ListPartitions(vector<Partition>* partitions) {
+  DCHECK(partitions);
+  partitions->clear();
+  auto& client = data_->client_;
+  const auto deadline = MonoTime::Now() + client->default_admin_operation_timeout();
+  PartitionPruner pruner;
+  pruner.Init(*data_->schema_.schema_, data_->partition_schema_, ScanSpec());
+  while (pruner.HasMorePartitionKeyRanges()) {
+    scoped_refptr<client::internal::RemoteTablet> tablet;
+    Synchronizer sync;
+    const string& partition_key = pruner.NextPartitionKey();
+    client->data_->meta_cache_->LookupTabletByKey(
+        this,
+        partition_key,
+        deadline,
+        client::internal::MetaCache::LookupType::kLowerBound,
+        &tablet,
+        sync.AsStatusCallback());
+    Status s = sync.Wait();
+    if (s.IsNotFound()) {
+      // No more tablets.
+      break;
+    }
+    RETURN_NOT_OK(s);
+
+    partitions->emplace_back(tablet->partition());
+    pruner.RemovePartitionKeyRange(tablet->partition().partition_key_end());
+  }
+  return Status::OK();
+}
+
 ////////////////////////////////////////////////////////////
 // Error
 ////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/kudu/blob/ef714838/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 79153a0..5335e00 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -63,9 +63,6 @@ class KuduTable;
 
 namespace tools {
 class LeaderMasterProxy;
-Status ListPartitions(const client::sp::shared_ptr<client::KuduTable>& table,
-                      std::vector<Partition>* partitions);
-std::string GetMasterAddresses(const client::KuduClient&);
 } // namespace tools
 
 namespace client {
@@ -77,7 +74,6 @@ class KuduPartitioner;
 class KuduScanBatch;
 class KuduSession;
 class KuduStatusCallback;
-class KuduTable;
 class KuduTableAlterer;
 class KuduTableCreator;
 class KuduTablet;
@@ -452,6 +448,15 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   Status GetTablet(const std::string& tablet_id,
                    KuduTablet** tablet) KUDU_NO_EXPORT;
 
+  /// Get the master RPC addresses as configured on the last leader master this
+  /// client connected to, as a CSV. If the client has not connected to a leader
+  /// master, an empty string is returned.
+  ///
+  /// Private API.
+  ///
+  /// @return The master addresses as a CSV.
+  std::string GetMasterAddresses() const KUDU_NO_EXPORT;
+
   /// @endcond
 
   /// Policy with which to choose amongst multiple replicas.
@@ -590,10 +595,6 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   friend class internal::WriteRpc;
   friend class kudu::SecurityUnknownTskTest;
   friend class tools::LeaderMasterProxy;
-  friend Status tools::ListPartitions(
-      const client::sp::shared_ptr<client::KuduTable>& table,
-      std::vector<Partition>* partitions);
-  friend std::string tools::GetMasterAddresses(const client::KuduClient&);
 
   FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
   FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
@@ -1050,6 +1051,21 @@ class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
   /// @return The partition schema for the table.
   const PartitionSchema& partition_schema() const;
 
+  /// @cond false
+
+  /// List the partitions of this table in 'partitions'. This operation may
+  /// involve RPC roundtrips to the leader master, and has a timeout equal
+  /// to the table's client instance's default admin operation timeout.
+  ///
+  /// Private API.
+  ///
+  /// @param [out] partitions
+  ///   The list of partitions of the table.
+  /// @return Status object for the operation.
+  Status ListPartitions(std::vector<Partition>* partitions);
+
+  /// @end cond
+
  private:
   class KUDU_NO_EXPORT Data;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/ef714838/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 099a32c..22e5b66 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -37,15 +37,10 @@
 #include <google/protobuf/util/json_util.h>
 
 #include "kudu/client/client-internal.h"  // IWYU pragma: keep
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
-#include "kudu/client/meta_cache.h"
 #include "kudu/client/shared_ptr.h"
 #include "kudu/common/common.pb.h"
-#include "kudu/common/partition.h"
-#include "kudu/common/partition_pruner.h"
 #include "kudu/common/row_operations.h"
-#include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/consensus.pb.h"
@@ -73,7 +68,6 @@
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_admin.proxy.h"   // IWYU pragma: keep
 #include "kudu/tserver/tserver_service.proxy.h" // IWYU pragma: keep
-#include "kudu/util/async_util.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/jsonwriter.h"
 #include "kudu/util/memory/arena.h"
@@ -451,10 +445,6 @@ Status SetServerFlag(const string& address, uint16_t default_port,
   }
 }
 
-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;
@@ -858,40 +848,5 @@ Status ControlShellProtocol::SendMessage(const ControlShellRequestPB& message);
 template
 Status ControlShellProtocol::SendMessage(const ControlShellResponsePB& message);
 
-// The strategy for retrieving the partitions from the metacache is adapted
-// from KuduScanTokenBuilder::Data::Build.
-Status ListPartitions(const client::sp::shared_ptr<KuduTable>& table,
-                      vector<Partition>* partitions) {
-  DCHECK(table);
-  DCHECK(partitions);
-  auto* client = table->client();
-  const auto deadline = MonoTime::Now() + client->default_admin_operation_timeout();
-  PartitionPruner pruner;
-  const auto& schema_internal = SchemaFromKuduSchema(table->schema());
-  pruner.Init(schema_internal, table->partition_schema(), ScanSpec());
-  while (pruner.HasMorePartitionKeyRanges()) {
-    scoped_refptr<client::internal::RemoteTablet> tablet;
-    Synchronizer sync;
-    const string& partition_key = pruner.NextPartitionKey();
-    client->data_->meta_cache_->LookupTabletByKey(
-        table.get(),
-        partition_key,
-        deadline,
-        client::internal::MetaCache::LookupType::kLowerBound,
-        &tablet,
-        sync.AsStatusCallback());
-    Status s = sync.Wait();
-    if (s.IsNotFound()) {
-      // No more tablets.
-      break;
-    }
-    RETURN_NOT_OK(s);
-
-    partitions->emplace_back(tablet->partition());
-    pruner.RemovePartitionKeyRange(tablet->partition().partition_key_end());
-  }
-
-  return Status::OK();
-}
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/ef714838/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 b71889c..b90e12b 100644
--- a/src/kudu/tools/tool_action_common.h
+++ b/src/kudu/tools/tool_action_common.h
@@ -37,12 +37,10 @@ class function;
 namespace kudu {
 
 class MonoDelta;
-class Partition;
 class faststring; // NOLINT
 
 namespace client {
 class KuduClient;
-class KuduTable;
 } // namespace client
 
 namespace master {
@@ -135,17 +133,10 @@ Status PrintServerFlags(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);
 
-// 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);
 
-// Populates `partitions` with the partitions of the table `table`.
-Status ListPartitions(const client::sp::shared_ptr<client::KuduTable>& table,
-                      std::vector<Partition>* partitions);
-
 // 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/ef714838/src/kudu/tools/tool_action_hms.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_hms.cc b/src/kudu/tools/tool_action_hms.cc
index ded8871..ac963d6 100644
--- a/src/kudu/tools/tool_action_hms.cc
+++ b/src/kudu/tools/tool_action_hms.cc
@@ -118,7 +118,7 @@ Status Init(const RunnerContext& context,
   // that the check and fix tools use the exact same master address
   // configuration that the masters do, otherwise the HMS table entries will
   // disagree on the master addresses property.
-  *master_addrs = GetMasterAddresses(**kudu_client);
+  *master_addrs = (*kudu_client)->GetMasterAddresses();
 
   if (FLAGS_hive_metastore_uris.empty()) {
     string hive_metastore_uris = (*kudu_client)->GetHiveMetastoreUris();

http://git-wip-us.apache.org/repos/asf/kudu/blob/ef714838/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 310cb97..dbdf31f 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -164,7 +164,7 @@ Status DescribeTable(const RunnerContext& context) {
 
   // The partition schema with current range partitions.
   vector<Partition> partitions;
-  RETURN_NOT_OK_PREPEND(ListPartitions(table, &partitions),
+  RETURN_NOT_OK_PREPEND(table->ListPartitions(&partitions),
                         "failed to retrieve current partitions");
   const auto& schema_internal = client::SchemaFromKuduSchema(schema);
   const auto& partition_schema = table->partition_schema();
@@ -181,7 +181,7 @@ Status DescribeTable(const RunnerContext& context) {
         partition_schema.RangePartitionDebugString(partition.range_key_start(),
                                                    partition.range_key_end(),
                                                    schema_internal);
-    partition_strs.push_back(std::move(range_partition_str));
+    partition_strs.emplace_back(std::move(range_partition_str));
   }
   cout << partition_schema.DisplayString(schema_internal, partition_strs)
        << endl;