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 2022/06/30 18:47:20 UTC

[kudu] branch master updated: [c++ client] KUDU-2671 Custom hash schema alter table support

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 1889d4c44 [c++ client] KUDU-2671  Custom hash schema alter table support
1889d4c44 is described below

commit 1889d4c44385fec5efeeb2d287d9ab7a3544dcfe
Author: Abhishek Chennaka <ac...@cloudera.com>
AuthorDate: Wed Jun 22 11:50:09 2022 -0400

    [c++ client] KUDU-2671  Custom hash schema alter table support
    
    This patch adds public methods to C++ client to alter a table and add
    a new range partition with custom hash sub-partitioning. We make use
    of the KuduTableCreator::KuduRangePartition() for this purpose. The
    necessary changes are done in table_alterer-internal classes and
    methods to use the above mentioned KuduRangePartition() to store
    the table bounds information as well as custom hash schema
    information.
    
    Necessary tests are included in this patch which include adding and
    dropping the ranges with custom hash schema by altering the table.
    We also read and write the data into these partitions.
    
    The pending work in this patch is to rebase on top of
    https://gerrit.cloudera.org/#/c/17879/ and include test cases with
    scans with predicates on these partitions.
    
    Change-Id: Id4b1e306cca096d9479f06669cc22cc40d77fb42
    Reviewed-on: http://gerrit.cloudera.org:8080/18663
    Tested-by: Alexey Serbin <al...@apache.org>
    Reviewed-by: Alexey Serbin <al...@apache.org>
---
 src/kudu/client/client.cc                        |  47 ++++--
 src/kudu/client/client.h                         |  22 +++
 src/kudu/client/flex_partitioning_client-test.cc | 202 +++++++++++++++++++++++
 src/kudu/client/table_alterer-internal.cc        |  41 +++--
 src/kudu/client/table_alterer-internal.h         |   8 +-
 5 files changed, 290 insertions(+), 30 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 281d1d9aa..1f3b86c56 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1485,7 +1485,7 @@ KuduTableAlterer* KuduTableAlterer::SetComment(const string& new_comment) {
 
 KuduColumnSpec* KuduTableAlterer::AddColumn(const string& name) {
   Data::Step s = { AlterTableRequestPB::ADD_COLUMN,
-                   new KuduColumnSpec(name), nullptr, nullptr };
+                   new KuduColumnSpec(name), nullptr };
   auto* spec = s.spec;
   data_->steps_.emplace_back(std::move(s));
   return spec;
@@ -1493,7 +1493,7 @@ KuduColumnSpec* KuduTableAlterer::AddColumn(const string& name) {
 
 KuduColumnSpec* KuduTableAlterer::AlterColumn(const string& name) {
   Data::Step s = { AlterTableRequestPB::ALTER_COLUMN,
-                   new KuduColumnSpec(name), nullptr, nullptr };
+                   new KuduColumnSpec(name), nullptr };
   auto* spec = s.spec;
   data_->steps_.emplace_back(std::move(s));
   return spec;
@@ -1501,7 +1501,7 @@ KuduColumnSpec* KuduTableAlterer::AlterColumn(const string& name) {
 
 KuduTableAlterer* KuduTableAlterer::DropColumn(const string& name) {
   Data::Step s = { AlterTableRequestPB::DROP_COLUMN,
-                   new KuduColumnSpec(name), nullptr, nullptr };
+                   new KuduColumnSpec(name), nullptr };
   data_->steps_.emplace_back(std::move(s));
   return this;
 }
@@ -1539,16 +1539,42 @@ KuduTableAlterer* KuduTableAlterer::AddRangePartitionWithDimension(
 
   Data::Step s { AlterTableRequestPB::ADD_RANGE_PARTITION,
                  nullptr,
-                 unique_ptr<KuduPartialRow>(lower_bound),
-                 unique_ptr<KuduPartialRow>(upper_bound),
-                 lower_bound_type,
-                 upper_bound_type,
+                 std::unique_ptr<KuduTableCreator::KuduRangePartition>(
+                     new KuduTableCreator::KuduRangePartition(
+                         lower_bound, upper_bound, lower_bound_type, upper_bound_type)),
                  dimension_label.empty() ? nullopt : make_optional(dimension_label) };
   data_->steps_.emplace_back(std::move(s));
   data_->has_alter_partitioning_steps = true;
   return this;
 }
 
+KuduTableAlterer* KuduTableAlterer::AddRangePartition(
+    KuduTableCreator::KuduRangePartition* partition) {
+  CHECK(partition);
+  if (partition->data_->lower_bound_ == nullptr || partition->data_->upper_bound_  == nullptr) {
+    data_->status_ = Status::InvalidArgument("range partition bounds may not be null");
+    return this;
+  }
+  if (partition->data_->lower_bound_->schema() != partition->data_->upper_bound_->schema()) {
+    data_->status_ = Status::InvalidArgument("range partition bounds must have matching schemas");
+    return this;
+  }
+  if (data_->schema_ == nullptr) {
+    data_->schema_ = partition->data_->lower_bound_->schema();
+  } else if (partition->data_->lower_bound_->schema() != data_->schema_) {
+    data_->status_ = Status::InvalidArgument("range partition bounds must have matching schemas");
+    return this;
+  }
+
+  Data::Step s { AlterTableRequestPB::ADD_RANGE_PARTITION,
+                 nullptr,
+                 std::unique_ptr<KuduTableCreator::KuduRangePartition>(partition),
+                 nullopt };
+  data_->steps_.emplace_back(std::move(s));
+  data_->has_alter_partitioning_steps = true;
+  return this;
+}
+
 KuduTableAlterer* KuduTableAlterer::DropRangePartition(
     KuduPartialRow* lower_bound,
     KuduPartialRow* upper_bound,
@@ -1571,10 +1597,9 @@ KuduTableAlterer* KuduTableAlterer::DropRangePartition(
 
   Data::Step s { AlterTableRequestPB::DROP_RANGE_PARTITION,
                  nullptr,
-                 unique_ptr<KuduPartialRow>(lower_bound),
-                 unique_ptr<KuduPartialRow>(upper_bound),
-                 lower_bound_type,
-                 upper_bound_type };
+                 std::unique_ptr<KuduTableCreator::KuduRangePartition>(
+                     new KuduTableCreator::KuduRangePartition(
+                         lower_bound, upper_bound, lower_bound_type, upper_bound_type)) };
   data_->steps_.emplace_back(std::move(s));
   data_->has_alter_partitioning_steps = true;
   return this;
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 65980a075..296a0daf7 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -1274,6 +1274,7 @@ class KUDU_EXPORT KuduTableCreator {
     class KUDU_NO_EXPORT Data;
 
     friend class KuduTableCreator;
+    friend class KuduTableAlterer;
 
     // Owned.
     Data* data_;
@@ -1890,6 +1891,27 @@ class KUDU_EXPORT KuduTableAlterer {
       KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
       KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
 
+  /// Add the specified range partition with custom hash schema to the table.
+  ///
+  /// @note The table alterer takes ownership of the partition object.
+  ///
+  /// @note Multiple range partitions may be added as part of a single alter
+  ///   table transaction by calling this method multiple times on the table
+  ///   alterer.
+  ///
+  /// @note This client may immediately write and scan the new tablets when
+  ///   Alter() returns success, however other existing clients may have to wait
+  ///   for a timeout period to elapse before the tablets become visible. This
+  ///   period is configured by the master's 'table_locations_ttl_ms' flag, and
+  ///   defaults to 5 minutes.
+  ///
+  /// @param [in] partition
+  ///   The Kudu Range partition to be created. This Kudu Range partition can
+  ///   have a custom hash schema defined.
+  /// @return Raw pointer to this alterer object.
+  KuduTableAlterer* AddRangePartition(
+      KuduTableCreator::KuduRangePartition* partition);
+
   /// Add a range partition to the table with dimension label.
   ///
   /// @note The table alterer takes ownership of the rows.
diff --git a/src/kudu/client/flex_partitioning_client-test.cc b/src/kudu/client/flex_partitioning_client-test.cc
index 0c72faa2e..3ce4f0273 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -30,7 +30,9 @@
 
 #include "kudu/client/client.h"
 #include "kudu/client/scan_batch.h"
+#include "kudu/client/scan_predicate.h"
 #include "kudu/client/schema.h"
+#include "kudu/client/value.h"
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
 #include "kudu/gutil/port.h"
@@ -53,6 +55,7 @@ DECLARE_bool(enable_per_range_hash_schemas);
 DECLARE_int32(heartbeat_interval_ms);
 
 using kudu::client::sp::shared_ptr;
+using kudu::client::KuduValue;
 using kudu::cluster::InternalMiniCluster;
 using kudu::cluster::InternalMiniClusterOptions;
 using kudu::master::CatalogManager;
@@ -261,6 +264,27 @@ class FlexPartitioningTest : public KuduTest {
     ASSERT_EQ(expected_count, count);
   }
 
+  void CheckTableRowsNum(const char* table_name, const char* col_name,
+                         int lower, int upper, int expected_row_count) {
+    shared_ptr<KuduTable> table;
+    ASSERT_OK(client_->OpenTable(table_name, &table));
+    KuduScanner scanner(table.get());
+    ASSERT_OK(scanner.SetTimeoutMillis(60000));
+    ASSERT_OK(scanner.AddConjunctPredicate(table->NewComparisonPredicate(
+        col_name, KuduPredicate::GREATER_EQUAL, KuduValue::FromInt(lower))));
+    ASSERT_OK(scanner.AddConjunctPredicate(table->NewComparisonPredicate(
+        col_name, KuduPredicate::LESS_EQUAL, KuduValue::FromInt(upper))));
+
+    ASSERT_OK(scanner.Open());
+    KuduScanBatch batch;
+    int live_row_count = 0;
+    while (scanner.HasMoreRows()) {
+      ASSERT_OK(scanner.NextBatch(&batch));
+      live_row_count += batch.NumRows();
+    }
+    ASSERT_EQ(expected_row_count, live_row_count);
+  }
+
   KuduSchema schema_;
   unique_ptr<InternalMiniCluster> cluster_;
   shared_ptr<KuduClient> client_;
@@ -1039,5 +1063,183 @@ TEST_F(FlexPartitioningCreateTableTest, Negatives) {
   }
 }
 
+// Test for scenarios covering range partitioning with custom hash schemas
+// specified when adding a new custom hash schema partition to a table.
+class FlexPartitioningAlterTableTest : public FlexPartitioningTest {};
+
+TEST_F(FlexPartitioningAlterTableTest, AddAndDropCustomRangePartition) {
+  // Create a table with the following partitions:
+  //
+  //            hash bucket
+  //   key    0           1
+  //         -------------------------
+  // <111    x:{key}     x:{key}
+  //
+  // Later add the following partition:
+  //            hash bucket
+  //   key    0           1           2
+  //         --------------------------------
+  // 111-222 x:{key}     x:{key}     x:{key}
+  constexpr const char *const kTableName = "AddAndDropCustomRangePartition";
+
+  // Create a table with a single Range partition.
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  unique_ptr<KuduPartialRow> lower(schema_.NewRow());
+  ASSERT_OK(lower->SetInt32(kKeyColumn, INT32_MIN));
+  unique_ptr<KuduPartialRow> upper(schema_.NewRow());
+  ASSERT_OK(upper->SetInt32(kKeyColumn, 111));
+  table_creator->table_name(kTableName)
+      .schema(&schema_)
+      .num_replicas(1)
+      .add_hash_partitions({ kKeyColumn }, 2)
+      .set_range_partition_columns({ kKeyColumn })
+      .add_range_partition(lower.release(), upper.release());
+  ASSERT_OK(table_creator->Create());
+
+  NO_FATALS(CheckTabletCount(kTableName, 2));
+
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 3 buckets with hash based on the "key" column with hash seed 1.
+  {
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    auto p = CreateRangePartition(111, 222);
+    ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 1));
+    table_alterer->AddRangePartition(p.release());
+    ASSERT_OK(table_alterer->Alter());
+  }
+
+  NO_FATALS(CheckTabletCount(kTableName, 5));
+
+  // Try adding a range partition with custom hash sub-partitioning rules:
+  // 3 buckets with hash based on the "key" column with hash seed 1.
+  // 2 buckets with hash based on the "key" column with hash seed 1.
+  {
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    auto p = CreateRangePartition(222, 333);
+    ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 1));
+    ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 2, 1));
+    table_alterer->AddRangePartition(p.release());
+    const auto s = table_alterer->Alter();
+    ASSERT_TRUE(s.IsNotSupported()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "varying number of hash dimensions per range is not yet supported");
+  }
+
+  // Drop the successfully created partition.
+  {
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    unique_ptr<KuduPartialRow> lower_drop(schema_.NewRow());
+    ASSERT_OK(lower_drop->SetInt32(kKeyColumn, 111));
+    unique_ptr<KuduPartialRow> upper_drop(schema_.NewRow());
+    ASSERT_OK(upper_drop->SetInt32(kKeyColumn, 222));
+    table_alterer->DropRangePartition(lower_drop.release(), upper_drop.release());
+    ASSERT_OK(table_alterer->Alter());
+  }
+
+  NO_FATALS(CheckTabletCount(kTableName, 2));
+}
+
+TEST_F(FlexPartitioningAlterTableTest, ReadAndWriteToCustomRangePartition) {
+  // Create a table with the following partitions:
+  //
+  //            hash bucket
+  //   key    0           1           2               3
+  //         -----------------------------------------------------------
+  //  <111    x:{key}     x:{key}     -               -
+
+  // Later add the following partitions:
+  //            hash bucket
+  //   key    0           1           2               3
+  //         -----------------------------------------------------------
+  // 111-222  x:{key}     x:{key}     x:{key}         -
+  // 222-333  x:{key}     x:{key}     x:{key}     x:{key}
+  // 333-444  x:{key}     x:{key}     -               -
+
+  constexpr const char* const kTableName = "ReadAndWriteToCustomRangePartition";
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  unique_ptr<KuduPartialRow> lower(schema_.NewRow());
+  ASSERT_OK(lower->SetInt32(kKeyColumn, INT32_MIN));
+  unique_ptr<KuduPartialRow> upper(schema_.NewRow());
+  ASSERT_OK(upper->SetInt32(kKeyColumn, 111));
+  table_creator->table_name(kTableName)
+      .schema(&schema_)
+      .num_replicas(1)
+      .add_hash_partitions({ kKeyColumn }, 2)
+      .set_range_partition_columns({ kKeyColumn })
+      .add_range_partition(lower.release(), upper.release());
+  ASSERT_OK(table_creator->Create());
+
+  unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 3 buckets with hash based on the "key" column with hash seed 1.
+  {
+    auto p = CreateRangePartition(111, 222);
+    ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 1));
+    table_alterer->AddRangePartition(p.release());
+  }
+
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 4 buckets with hash based on the "key" column with hash seed 2.
+  {
+    auto p = CreateRangePartition(222, 333);
+    ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 4, 2));
+    table_alterer->AddRangePartition(p.release());
+  }
+
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 2 buckets hashing on the "key" column with hash seed 3.
+  {
+    auto p = CreateRangePartition(333, 444);
+    ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 2, 3));
+    table_alterer->AddRangePartition(p.release());
+  }
+
+  ASSERT_OK(table_alterer->Alter());
+  NO_FATALS(CheckTabletCount(kTableName, 11));
+
+  // Make sure it's possible to insert rows into the table for all the existing
+  // the partitions: first check the range of table-wide schema, then check
+  // the ranges with custom hash schemas.
+  ASSERT_OK(InsertTestRows(kTableName, 0, 111));
+  NO_FATALS(CheckLiveRowCount(kTableName, 111));
+  ASSERT_OK(InsertTestRows(kTableName, 111, 444));
+  NO_FATALS(CheckLiveRowCount(kTableName, 444));
+
+  // WIP: Scan the data present after rebasing on pruning patches
+  /*
+   NO_FATALS(CheckTableRowsNum(kTableName, kKeyColumn, 111, 222, 111));
+   NO_FATALS(CheckTableRowsNum(kTableName, kKeyColumn, 0, 444, 444));
+  // Drop a partition and re-scan
+  {
+    unique_ptr<KuduTableAlterer> table_alterer_drop(client_->NewTableAlterer(kTableName));
+    unique_ptr<KuduPartialRow> lower_drop(schema_.NewRow());
+    ASSERT_OK(lower_drop->SetInt32(kKeyColumn, 111));
+    unique_ptr<KuduPartialRow> upper_drop(schema_.NewRow());
+    ASSERT_OK(upper_drop->SetInt32(kKeyColumn, 222));
+    table_alterer_drop->DropRangePartition(lower_drop.release(), upper_drop.release());
+    ASSERT_OK(table_alterer_drop->Alter());
+  }
+  NO_FATALS(CheckTableRowsNum(kTableName, kKeyColumn, 0, 444, 333));
+  */
+
+  // Meanwhile, inserting into non-covered ranges should result in a proper
+  // error status return to the client attempting such an operation.
+  {
+    constexpr int kNumRows = 10;
+    vector<KuduError*> errors;
+    ElementDeleter drop(&errors);
+    auto s = InsertTestRows(
+        kTableName, 445, 445 + kNumRows, KuduSession::MANUAL_FLUSH, &errors);
+    ASSERT_TRUE(s.IsIOError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "failed to flush data");
+    ASSERT_EQ(kNumRows, errors.size());
+    for (const auto& e : errors) {
+      const auto& err = e->status();
+      EXPECT_TRUE(err.IsNotFound()) << err.ToString();
+      ASSERT_STR_CONTAINS(err.ToString(),
+                          "No tablet covering the requested range partition");
+    }
+  }
+}
 } // namespace client
 } // namespace kudu
diff --git a/src/kudu/client/table_alterer-internal.cc b/src/kudu/client/table_alterer-internal.cc
index 5f80c93d9..8e43ebfe2 100644
--- a/src/kudu/client/table_alterer-internal.cc
+++ b/src/kudu/client/table_alterer-internal.cc
@@ -17,6 +17,7 @@
 
 #include "kudu/client/table_alterer-internal.h"
 
+#include <memory>
 #include <ostream>
 #include <string>
 #include <type_traits>
@@ -27,6 +28,8 @@
 
 #include "kudu/client/schema-internal.h"
 #include "kudu/client/schema.h"
+#include "kudu/client/table_creator-internal.h"
+#include "kudu/common/common.pb.h"
 #include "kudu/common/row_operations.h"
 #include "kudu/common/row_operations.pb.h"
 #include "kudu/common/schema.h"
@@ -164,18 +167,29 @@ Status KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
       {
         RowOperationsPBEncoder encoder(pb_step->mutable_add_range_partition()
                                               ->mutable_range_bounds());
+        auto* partition_data = s.range_partition->data_;
         RowOperationsPB_Type lower_bound_type =
-          s.lower_bound_type == KuduTableCreator::INCLUSIVE_BOUND ?
-          RowOperationsPB::RANGE_LOWER_BOUND :
-          RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
+            partition_data->lower_bound_type_ == KuduTableCreator::INCLUSIVE_BOUND ?
+            RowOperationsPB::RANGE_LOWER_BOUND :
+            RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
 
         RowOperationsPB_Type upper_bound_type =
-          s.upper_bound_type == KuduTableCreator::EXCLUSIVE_BOUND ?
-          RowOperationsPB::RANGE_UPPER_BOUND :
-          RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
-
-        encoder.Add(lower_bound_type, *s.lower_bound);
-        encoder.Add(upper_bound_type, *s.upper_bound);
+            partition_data->upper_bound_type_ == KuduTableCreator::EXCLUSIVE_BOUND ?
+            RowOperationsPB::RANGE_UPPER_BOUND :
+            RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
+
+        encoder.Add(lower_bound_type, *partition_data->lower_bound_);
+        encoder.Add(upper_bound_type, *partition_data->upper_bound_);
+
+        for (const auto& hash_dimension : partition_data->hash_schema_) {
+          auto* custom_hash_schema_pb = pb_step->mutable_add_range_partition()->
+              add_custom_hash_schema();
+          for (const auto& column_name : hash_dimension.column_names) {
+            custom_hash_schema_pb->add_columns()->set_name(column_name);
+          }
+          custom_hash_schema_pb->set_num_buckets(hash_dimension.num_buckets);
+          custom_hash_schema_pb->set_seed(hash_dimension.seed);
+        }
 
         if (s.dimension_label) {
           pb_step->mutable_add_range_partition()->set_dimension_label(
@@ -187,18 +201,19 @@ Status KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
       {
         RowOperationsPBEncoder encoder(pb_step->mutable_drop_range_partition()
                                               ->mutable_range_bounds());
+        auto* partition_data = s.range_partition->data_;
         RowOperationsPB_Type lower_bound_type =
-          s.lower_bound_type == KuduTableCreator::INCLUSIVE_BOUND ?
+            partition_data->lower_bound_type_ == KuduTableCreator::INCLUSIVE_BOUND ?
           RowOperationsPB::RANGE_LOWER_BOUND :
           RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
 
         RowOperationsPB_Type upper_bound_type =
-          s.upper_bound_type == KuduTableCreator::EXCLUSIVE_BOUND ?
+            partition_data->upper_bound_type_ == KuduTableCreator::EXCLUSIVE_BOUND ?
           RowOperationsPB::RANGE_UPPER_BOUND :
           RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
 
-        encoder.Add(lower_bound_type, *s.lower_bound);
-        encoder.Add(upper_bound_type, *s.upper_bound);
+        encoder.Add(lower_bound_type, *partition_data->lower_bound_);
+        encoder.Add(upper_bound_type, *partition_data->upper_bound_);
         break;
       }
       default:
diff --git a/src/kudu/client/table_alterer-internal.h b/src/kudu/client/table_alterer-internal.h
index f9eed1dd7..19b43493f 100644
--- a/src/kudu/client/table_alterer-internal.h
+++ b/src/kudu/client/table_alterer-internal.h
@@ -24,7 +24,6 @@
 #include <vector>
 
 #include "kudu/client/client.h"
-#include "kudu/common/partial_row.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/util/monotime.h"
@@ -56,12 +55,9 @@ class KuduTableAlterer::Data {
     // [ADD|DROP|RENAME|ALTER]_COLUMN.
     KuduColumnSpec *spec;
 
-    // Lower and upper bound partition keys. Only set when the StepType is
+    // The Kudu range partition to add or drop. Only set when the StepType is
     // [ADD|DROP]_RANGE_PARTITION.
-    std::unique_ptr<KuduPartialRow> lower_bound;
-    std::unique_ptr<KuduPartialRow> upper_bound;
-    KuduTableCreator::RangePartitionBound lower_bound_type;
-    KuduTableCreator::RangePartitionBound upper_bound_type;
+    std::unique_ptr<KuduTableCreator::KuduRangePartition> range_partition;
 
     // The dimension label for tablet. Only set when the StepType is ADD_RANGE_PARTITION.
     std::optional<std::string> dimension_label;