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 66fe336  [tools] add beginning of the key range into TabletSummary
66fe336 is described below

commit 66fe33642ac9662b8d2a3e52a90168b2184e4d88
Author: Alexey Serbin <ale...@apache.org>
AuthorDate: Thu Mar 10 16:57:23 2022 -0800

    [tools] add beginning of the key range into TabletSummary
    
    This patch adds the beginning of the key range into the TabletSummary
    class used in ksck reporting.  Corresponding TabletSummaryPB has been
    updated as well.
    
    This patch also contains a test to verify the presence of the newly
    added field in TabletSummary entries produced by ksck.
    
    The newly added field will be used in a follow-up changelist for
    range-aware rebalancing.
    
    Change-Id: Icf083e1ae33919200f6e0b0f0a667ee6eea1bab3
    Reviewed-on: http://gerrit.cloudera.org:8080/18356
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/client/CMakeLists.txt                   |  1 +
 src/kudu/client/client.h                         |  2 +
 src/kudu/client/tablet_info_provider-internal.cc | 50 ++++++++++++++
 src/kudu/client/tablet_info_provider-internal.h  | 49 +++++++++++++
 src/kudu/rebalance/cluster_status.h              |  1 +
 src/kudu/tools/ksck-test.cc                      | 16 +++--
 src/kudu/tools/ksck.cc                           | 13 ++++
 src/kudu/tools/ksck.h                            | 15 ++--
 src/kudu/tools/ksck_remote-test.cc               | 87 +++++++++++++++++++++++-
 src/kudu/tools/ksck_remote.cc                    | 13 +++-
 src/kudu/tools/ksck_results.cc                   |  1 +
 src/kudu/tools/tool.proto                        |  1 +
 12 files changed, 236 insertions(+), 13 deletions(-)

diff --git a/src/kudu/client/CMakeLists.txt b/src/kudu/client/CMakeLists.txt
index fcf5c2c..65821b3 100644
--- a/src/kudu/client/CMakeLists.txt
+++ b/src/kudu/client/CMakeLists.txt
@@ -50,6 +50,7 @@ set(CLIENT_SRCS
   scanner-internal.cc
   replica-internal.cc
   replica_controller-internal.cc
+  tablet_info_provider-internal.cc
   resource_metrics.cc
   schema.cc
   session-internal.cc
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 3905ec8..35dab86 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -122,6 +122,7 @@ class RemoteTabletServer;
 class ReplicaController;
 class RetrieveAuthzTokenRpc;
 class ScanBatchDataInterface;
+class TabletInfoProvider;
 class WriteRpc;
 template <class ReqClass, class RespClass>
 class AsyncLeaderMasterRpc; // IWYU pragma: keep
@@ -986,6 +987,7 @@ class KUDU_EXPORT KuduClient : public 
sp::enable_shared_from_this<KuduClient> {
   friend class internal::RemoteTablet;
   friend class internal::RemoteTabletServer;
   friend class internal::RetrieveAuthzTokenRpc;
+  friend class internal::TabletInfoProvider;
   friend class internal::WriteRpc;
   friend class kudu::AuthzTokenTest;
   friend class kudu::DisableWriteWhenExceedingQuotaTest;
diff --git a/src/kudu/client/tablet_info_provider-internal.cc 
b/src/kudu/client/tablet_info_provider-internal.cc
new file mode 100644
index 0000000..ef6383e
--- /dev/null
+++ b/src/kudu/client/tablet_info_provider-internal.cc
@@ -0,0 +1,50 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/client/tablet_info_provider-internal.h"
+
+#include <glog/logging.h>
+
+#include "kudu/client/client-internal.h"
+#include "kudu/client/client.h"
+#include "kudu/client/meta_cache.h"
+#include "kudu/common/partition.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/async_util.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+namespace client {
+namespace internal {
+
+Status TabletInfoProvider::GetPartitionInfo(KuduClient* client,
+                                            const std::string& tablet_id,
+                                            Partition* partition) {
+  DCHECK(partition);
+  scoped_refptr<RemoteTablet> rt;
+  Synchronizer sync;
+  client->data_->meta_cache_->LookupTabletById(
+      client, tablet_id, MonoTime::Max(), &rt,
+      sync.AsStatusCallback());
+  RETURN_NOT_OK(sync.Wait());
+  *partition = rt->partition();
+  return Status::OK();
+}
+
+} // namespace internal
+} // namespace client
+} // namespace kudu
diff --git a/src/kudu/client/tablet_info_provider-internal.h 
b/src/kudu/client/tablet_info_provider-internal.h
new file mode 100644
index 0000000..ac0798b
--- /dev/null
+++ b/src/kudu/client/tablet_info_provider-internal.h
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Partition;
+
+namespace client {
+
+class KuduClient;
+
+namespace internal {
+
+// This is a class whose sole responsibility is to provide access to the
+// client's metacache to fetch various information on tablets.
+//
+// Having this single class reduces cruft in friend class declarations in the
+// client.h file.
+class TabletInfoProvider {
+ public:
+  // Provides partition information for the tablet with the specified
+  // identifier.
+  static Status GetPartitionInfo(KuduClient* client,
+                                 const std::string& tablet_id,
+                                 Partition* partition);
+};
+
+} // namespace internal
+} // namespace client
+} // namespace kudu
diff --git a/src/kudu/rebalance/cluster_status.h 
b/src/kudu/rebalance/cluster_status.h
index cd3c3b0..aa95aff 100644
--- a/src/kudu/rebalance/cluster_status.h
+++ b/src/kudu/rebalance/cluster_status.h
@@ -212,6 +212,7 @@ struct TabletSummary {
   std::string id;
   std::string table_id;
   std::string table_name;
+  std::string range_key_begin;
   HealthCheckResult result;
   std::string status;
   ConsensusState master_cstate;
diff --git a/src/kudu/tools/ksck-test.cc b/src/kudu/tools/ksck-test.cc
index 388821b..36dae8f 100644
--- a/src/kudu/tools/ksck-test.cc
+++ b/src/kudu/tools/ksck-test.cc
@@ -39,6 +39,7 @@
 #include <gtest/gtest.h>
 #include <rapidjson/document.h>
 
+#include "kudu/common/partition.h"
 #include "kudu/common/schema.h"
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/gutil/map-util.h"
@@ -337,13 +338,15 @@ class KsckTest : public KuduTest {
     NO_FATALS(CreateDefaultAssignmentPlan(create_txn_status_table ? 2 : 1));
 
     auto table = CreateAndAddTable("test", 1);
-    auto tablet(make_shared<KsckTablet>(table.get(), "tablet-id-1"));
+    auto tablet(make_shared<KsckTablet>(
+        table.get(), "tablet-id-1", Partition{}));
     NO_FATALS(CreateAndFillTablet(tablet, 1, true, true));
     table->set_tablets({ tablet });
 
     if (create_txn_status_table) {
       auto sys_table = CreateAndAddTxnStatusTable(1);
-      auto sys_tablet(make_shared<KsckTablet>(sys_table.get(), 
"sys-tablet-id-1"));
+      auto sys_tablet(make_shared<KsckTablet>(
+          sys_table.get(), "sys-tablet-id-1", Partition{}));
       NO_FATALS(CreateAndFillTablet(sys_tablet, 1, true, true));
       sys_table->set_tablets({ sys_tablet });
     }
@@ -359,7 +362,9 @@ class KsckTest : public KuduTest {
     vector<shared_ptr<KsckTablet>> tablets;
     for (int i = 0; i < num_tablets; i++) {
       auto tablet(make_shared<KsckTablet>(
-          table.get(), Substitute("$0tablet-id-$1", tablet_id_prefix, i)));
+          table.get(),
+          Substitute("$0tablet-id-$1", tablet_id_prefix, i),
+          Partition{}));
       CreateAndFillTablet(tablet, num_replicas, true, true);
       tablets.push_back(std::move(tablet));
     }
@@ -375,7 +380,7 @@ class KsckTest : public KuduTest {
     vector<shared_ptr<KsckTablet>> tablets;
     for (int i = 0; i < num_tablets; i++) {
       auto tablet(make_shared<KsckTablet>(
-          table.get(), Substitute("tablet-id-$0", i)));
+          table.get(), Substitute("tablet-id-$0", i), Partition{}));
       CreateAndFillTablet(tablet, num_replicas, true, i != 0);
       tablets.push_back(std::move(tablet));
     }
@@ -388,7 +393,7 @@ class KsckTest : public KuduTest {
 
     auto table = CreateAndAddTable("test", 3);
 
-    auto tablet(make_shared<KsckTablet>(table.get(), "tablet-id-1"));
+    auto tablet(make_shared<KsckTablet>(table.get(), "tablet-id-1", 
Partition{}));
     CreateAndFillTablet(tablet, 2, false, true);
     table->set_tablets({ tablet });
   }
@@ -742,6 +747,7 @@ void CheckJsonVsTabletSummaries(const JsonReader& r,
       const auto* replica = replicas[j];
       CheckJsonVsReplicaSummary(r, replica, ref_replica);
     }
+    EXPECT_JSON_STRING_FIELD(r, tablet, "range_key_begin", 
ref_tablet.range_key_begin);
   }
 }
 
diff --git a/src/kudu/tools/ksck.cc b/src/kudu/tools/ksck.cc
index 050bc06..4846ea0 100644
--- a/src/kudu/tools/ksck.cc
+++ b/src/kudu/tools/ksck.cc
@@ -21,6 +21,7 @@
 #include <atomic>
 #include <cstddef>
 #include <functional>
+#include <iomanip>
 #include <iostream>
 #include <iterator>
 #include <map>
@@ -34,6 +35,7 @@
 #include <glog/logging.h>
 #include <google/protobuf/stubs/port.h>
 
+#include "kudu/common/partition.h"
 #include "kudu/consensus/quorum_util.h"
 #include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/map-util.h"
@@ -1010,6 +1012,17 @@ HealthCheckResult Ksck::VerifyTablet(const 
shared_ptr<KsckTablet>& tablet,
   tablet_summary.status = status;
   tablet_summary.master_cstate = std::move(master_config);
   tablet_summary.replicas.swap(replicas);
+
+  // Add printable representation of the key for the start of the range.
+  const auto& range_key_begin = tablet->partition().begin().range_key();
+  ostringstream ss_range_key_begin;
+  for (size_t i = 0; i < range_key_begin.size(); ++i) {
+    ss_range_key_begin << std::hex << std::setw(2) << std::setfill('0')
+                       << static_cast<uint16_t>(range_key_begin[i]);
+  }
+  tablet_summary.range_key_begin = ss_range_key_begin.str();
+  VLOG(1) << Substitute("range start key for tablet $0: '$1'",
+      tablet_summary.id, tablet_summary.range_key_begin);
   
results_.cluster_status.tablet_summaries.push_back(std::move(tablet_summary));
   return result;
 }
diff --git a/src/kudu/tools/ksck.h b/src/kudu/tools/ksck.h
index 2466e18..8b4c078 100644
--- a/src/kudu/tools/ksck.h
+++ b/src/kudu/tools/ksck.h
@@ -33,6 +33,7 @@
 #include <glog/logging.h>
 #include <gtest/gtest_prod.h>
 
+#include "kudu/common/partition.h"
 #include "kudu/common/schema.h"
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/gutil/macros.h"
@@ -89,9 +90,10 @@ class KsckTabletReplica {
 // Representation of a tablet belonging to a table. The tablet is composed of 
replicas.
 class KsckTablet {
  public:
-  KsckTablet(KsckTable* table, std::string id)
-      : id_(std::move(id)),
-        table_(table) {
+  KsckTablet(KsckTable* table, std::string id, Partition partition)
+      : table_(table),
+        id_(std::move(id)),
+        partition_(std::move(partition)) {
   }
 
   const std::string& id() const {
@@ -102,6 +104,10 @@ class KsckTablet {
     return replicas_;
   }
 
+  const Partition& partition() const {
+    return partition_;
+  }
+
   void set_replicas(std::vector<std::shared_ptr<KsckTabletReplica>> replicas) {
     replicas_.swap(replicas);
   }
@@ -111,9 +117,10 @@ class KsckTablet {
   }
 
  private:
+  KsckTable* table_;
   const std::string id_;
+  const Partition partition_;
   std::vector<std::shared_ptr<KsckTabletReplica>> replicas_;
-  KsckTable* table_;
   DISALLOW_COPY_AND_ASSIGN(KsckTablet);
 };
 
diff --git a/src/kudu/tools/ksck_remote-test.cc 
b/src/kudu/tools/ksck_remote-test.cc
index 344f740..2098019 100644
--- a/src/kudu/tools/ksck_remote-test.cc
+++ b/src/kudu/tools/ksck_remote-test.cc
@@ -34,6 +34,7 @@
 #include "kudu/client/shared_ptr.h" // IWYU pragma: keep
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
+#include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/stl_util.h"
@@ -41,6 +42,7 @@
 #include "kudu/integration-tests/data_gen_util.h"
 #include "kudu/master/mini_master.h"
 #include "kudu/mini-cluster/internal_mini_cluster.h"
+#include "kudu/rebalance/cluster_status.h"
 #include "kudu/tools/ksck.h"
 #include "kudu/tools/ksck_checksum.h"
 #include "kudu/tools/ksck_results.h"
@@ -86,8 +88,8 @@ using kudu::client::KuduTableCreator;
 using kudu::client::sp::shared_ptr;
 using kudu::cluster::InternalMiniCluster;
 using kudu::cluster::InternalMiniClusterOptions;
-using std::thread;
 using std::string;
+using std::thread;
 using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
@@ -824,5 +826,88 @@ TEST_F(RemoteKsckTest, TestTabletFilters) {
       " Replicas       | 3\n");
 }
 
+// Make sure the tablet summaries have the 'range_key_begin' field set
+// correspondingly for tables with and without range partitions.
+TEST_F(RemoteKsckTest, RangeKeysInTabletSummaries) {
+  static constexpr const char* const kColumn0 = "c0";
+  static constexpr const char* const kColumn1 = "c1";
+  static constexpr const char* const kTableWithRanges = "table_with_ranges";
+  static constexpr const char* const kTableWithoutRanges = 
"table_without_ranges";
+
+  SKIP_IF_SLOW_NOT_ALLOWED();
+
+  client::KuduSchema schema;
+  KuduSchemaBuilder b;
+  b.AddColumn(kColumn0)->Type(KuduColumnSchema::INT32)->NotNull();
+  b.AddColumn(kColumn1)->Type(KuduColumnSchema::INT32)->NotNull();
+  b.SetPrimaryKey({ kColumn0, kColumn1 });
+  ASSERT_OK(b.Build(&schema));
+
+  // Create range-partitioned table.
+  {
+    unique_ptr<KuduPartialRow> lower_bound(schema.NewRow());
+    ASSERT_OK(lower_bound->SetInt32(kColumn0, 0));
+    unique_ptr<KuduPartialRow> upper_bound(schema.NewRow());
+    ASSERT_OK(upper_bound->SetInt32(kColumn0, 1000));
+
+    unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+    ASSERT_OK(table_creator->table_name(kTableWithRanges)
+        .schema(&schema)
+        .set_range_partition_columns({ kColumn0 })
+        .add_range_partition(lower_bound.release(), upper_bound.release())
+        .add_hash_partitions({ kColumn1 }, 2)
+        .num_replicas(3)
+        .Create());
+  }
+
+  // Create a table with no range partitions.
+  {
+    unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+    ASSERT_OK(table_creator->table_name(kTableWithoutRanges)
+        .schema(&schema)
+        .add_hash_partitions({ kColumn1 }, 3)
+        .num_replicas(3)
+        .Create());
+  }
+
+  ASSERT_OK(ksck_->FetchTableAndTabletInfo());
+  ASSERT_OK(ksck_->FetchInfoFromTabletServers());
+  // This scenario doesn't care about consistency of the tables in terms of
+  // matching Raft status between the catalog manager and tablet servers, etc.
+  // For this scenario, the important point is just to collect the information
+  // on tablets.
+  ignore_result(ksck_->CheckTablesConsistency());
+
+  vector<string> range_keys;
+  vector<string> no_range_keys;
+  const auto& tablet_summaries = 
ksck_->results().cluster_status.tablet_summaries;
+  for (const auto& summary : tablet_summaries) {
+    if (summary.table_name == kTableWithoutRanges) {
+      no_range_keys.emplace_back(summary.range_key_begin);
+    } else if (summary.table_name == kTableWithRanges) {
+      range_keys.emplace_back(summary.range_key_begin);
+    }
+  }
+
+  // Check the results for the range-paritioned table.
+  // There is 1 range partition [0, 1000) and 2 hash buckets.
+  ASSERT_EQ(2, range_keys.size());
+  for (const auto& key : range_keys) {
+    ASSERT_FALSE(key.empty()) << key;
+  }
+  // The keys for the beginning of the range partition should be the same for
+  // every tablet reported: that's exactly the same range partition,
+  // but different hash buckets.
+  ASSERT_EQ(range_keys.front(), range_keys.back());
+
+  // Check the results for the table without range partitions.
+  // There are no range partitions: just 3 hash buckets, so all the range start
+  // keys in the tablet summaries should be empty.
+  ASSERT_EQ(3, no_range_keys.size());
+  for (const auto& key : no_range_keys) {
+    ASSERT_TRUE(key.empty()) << key;
+  }
+}
+
 } // namespace tools
 } // namespace kudu
diff --git a/src/kudu/tools/ksck_remote.cc b/src/kudu/tools/ksck_remote.cc
index 09dfe21..16f9628 100644
--- a/src/kudu/tools/ksck_remote.cc
+++ b/src/kudu/tools/ksck_remote.cc
@@ -31,8 +31,10 @@
 #include "kudu/client/client-internal.h"
 #include "kudu/client/client.h"
 #include "kudu/client/replica_controller-internal.h"
+#include "kudu/client/tablet_info_provider-internal.h"
 #include "kudu/client/schema.h"
 #include "kudu/common/common.pb.h"
+#include "kudu/common/partition.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
@@ -84,6 +86,7 @@ using kudu::client::KuduScanTokenBuilder;
 using kudu::client::KuduSchema;
 using kudu::client::KuduTable;
 using kudu::client::internal::ReplicaController;
+using kudu::client::internal::TabletInfoProvider;
 using kudu::cluster_summary::ServerHealth;
 using kudu::master::ListTabletServersRequestPB;
 using kudu::master::ListTabletServersResponsePB;
@@ -684,13 +687,17 @@ Status RemoteKsckCluster::RetrieveTabletsList(const 
shared_ptr<KsckTable>& table
 
   vector<shared_ptr<KsckTablet>> tablets;
   for (const auto* t : tokens) {
-    if (!MatchesAnyPattern(tablet_id_filters_, t->tablet().id())) {
+    const auto& tablet_id = t->tablet().id();
+    if (!MatchesAnyPattern(tablet_id_filters_, tablet_id)) {
       filtered_tablets_count_++;
-      VLOG(1) << "Skipping tablet " << t->tablet().id();
+      VLOG(1) << "Skipping tablet " << tablet_id;
       continue;
     }
+    Partition partition;
+    RETURN_NOT_OK(TabletInfoProvider::GetPartitionInfo(
+        client_.get(), tablet_id, &partition));
     shared_ptr<KsckTablet> tablet(
-        new KsckTablet(table.get(), t->tablet().id()));
+        new KsckTablet(table.get(), t->tablet().id(), std::move(partition)));
     vector<shared_ptr<KsckTabletReplica>> replicas;
     for (const auto* r : t->tablet().replicas()) {
       replicas.push_back(make_shared<KsckTabletReplica>(
diff --git a/src/kudu/tools/ksck_results.cc b/src/kudu/tools/ksck_results.cc
index 7c503c5..8798d39 100644
--- a/src/kudu/tools/ksck_results.cc
+++ b/src/kudu/tools/ksck_results.cc
@@ -920,6 +920,7 @@ void TabletSummaryToPb(const TabletSummary& tablet, 
TabletSummaryPB* pb) {
   for (const auto& replica : tablet.replicas) {
     ReplicaSummaryToPb(replica, pb->add_replicas());
   }
+  pb->set_range_key_begin(tablet.range_key_begin);
 }
 
 void TableSummaryToPb(const TableSummary& table, TableSummaryPB* pb) {
diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index 3aea8d4..b731513 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -297,6 +297,7 @@ message TabletSummaryPB {
   optional string status = 5;
   optional ConsensusStatePB master_cstate = 6;
   repeated ReplicaSummaryPB replicas = 7;
+  optional string range_key_begin = 8;
 }
 
 message ReplicaSummaryPB {

Reply via email to