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 dc4031f69 KUDU-2671 check for duplicate columns in hash schema when 
adding range
dc4031f69 is described below

commit dc4031f693382df08c0fab1d0c5ac6bc3c203c35
Author: Alexey Serbin <ale...@apache.org>
AuthorDate: Wed Jul 13 19:33:02 2022 -0700

    KUDU-2671 check for duplicate columns in hash schema when adding range
    
    This patch adds validation for range-specific hash schemas when
    adding new range partitions.  Without this patch, invalid requests
    to add range partitions with duplicate columns across dimensions
    of the custom hash schema would be accepted, but tablets could not be
    created, resulting in timed-out IsAlterTableDone() RPC.  The patch also
    contains new test scenarios for both C++ and Java Kudu clients, making
    sure the corresponding error is reported back to the client.  I verified
    that the new test scenarios failed as expected if commenting out
    the newly added hash schema validation code.
    
    This patch also fixes a few typos in test scenarios from master-test.cc
    since the new verification code exposed those mistakes.
    
    Change-Id: Iefe6a97028ae12585ac5496ac8608448ffacd95e
    Reviewed-on: http://gerrit.cloudera.org:8080/18728
    Reviewed-by: Mahesh Reddy <mre...@cloudera.com>
    Tested-by: Alexey Serbin <ale...@apache.org>
    Reviewed-by: Abhishek Chennaka <achenn...@cloudera.com>
    Reviewed-by: Alexey Serbin <ale...@apache.org>
---
 .../org/apache/kudu/client/TestAlterTable.java     |  62 ++++++
 src/kudu/client/flex_partitioning_client-test.cc   | 106 ++++++++++-
 src/kudu/common/partition.h                        |   8 +-
 src/kudu/master/catalog_manager.cc                 |   1 +
 src/kudu/master/master-test.cc                     | 209 +++++++++++----------
 5 files changed, 276 insertions(+), 110 deletions(-)

diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index 2c4908020..7e722e0de 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -819,6 +819,68 @@ public class TestAlterTable {
     }
   }
 
+  @Test(timeout = 100000)
+  public void testAlterTryAddRangeWithCustomHashSchemaDuplicateColumns()
+      throws Exception {
+    ArrayList<ColumnSchema> columns = new ArrayList<>(2);
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c0", Type.INT32)
+        .nullable(false)
+        .key(true)
+        .build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c1", Type.INT32)
+        .nullable(false)
+        .key(true)
+        .build());
+    final Schema schema = new Schema(columns);
+
+    CreateTableOptions createOptions =
+        new CreateTableOptions()
+            .setRangePartitionColumns(ImmutableList.of("c0"))
+            .addHashPartitions(ImmutableList.of("c0"), 2, 0)
+            .addHashPartitions(ImmutableList.of("c1"), 3, 0)
+            .setNumReplicas(1);
+
+    // Add range partition with table-wide hash schema.
+    {
+      PartialRow lower = schema.newPartialRow();
+      lower.addInt("c0", -100);
+      PartialRow upper = schema.newPartialRow();
+      upper.addInt("c0", 0);
+      createOptions.addRangePartition(lower, upper);
+    }
+
+    client.createTable(tableName, schema, createOptions);
+
+    // Try adding a range partition with custom hash schema having multiple
+    // hash dimensions and conflicting on columns used for hash function:
+    // different dimensions should not intersect on the set of columns
+    // used for hashing.
+    {
+      PartialRow lower = schema.newPartialRow();
+      lower.addInt("c0", 0);
+      PartialRow upper = schema.newPartialRow();
+      upper.addInt("c0", 100);
+      RangePartitionWithCustomHashSchema range =
+          new RangePartitionWithCustomHashSchema(
+              lower,
+              upper,
+              RangePartitionBound.INCLUSIVE_BOUND,
+              RangePartitionBound.EXCLUSIVE_BOUND);
+      range.addHashPartitions(ImmutableList.of("c0"), 3, 0);
+      range.addHashPartitions(ImmutableList.of("c0"), 3, 0);
+      try {
+        client.alterTable(tableName, new 
AlterTableOptions().addRangePartition(range));
+        fail("shouldn't be able to add a range with custom hash schema " +
+            "having duplicate hash columns across different dimensions");
+      } catch (KuduException ex) {
+        final String errmsg = ex.getMessage();
+        assertTrue(errmsg, ex.getStatus().isInvalidArgument());
+        assertTrue(errmsg, errmsg.matches(
+            "hash bucket schema components must not contain columns in 
common"));
+      }
+    }
+  }
+
   @Test
   public void testAlterExtraConfigs() throws Exception {
     KuduTable table = createTable(ImmutableList.of());
diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index c8af1b26d..8623789bd 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -195,17 +195,24 @@ class FlexPartitioningTest : public KuduTest {
     return table_creator->Create();
   }
 
-  RangePartition CreateRangePartition(int32_t lower_bound = 0,
-                                      int32_t upper_bound = 100) {
-    unique_ptr<KuduPartialRow> lower(schema_.NewRow());
-    CHECK_OK(lower->SetInt32(kKeyColumn, lower_bound));
-    unique_ptr<KuduPartialRow> upper(schema_.NewRow());
-    CHECK_OK(upper->SetInt32(kKeyColumn, upper_bound));
+  static RangePartition CreateRangePartition(const KuduSchema& schema,
+                                             const string& key_column,
+                                             int32_t lower_bound,
+                                             int32_t upper_bound) {
+    unique_ptr<KuduPartialRow> lower(schema.NewRow());
+    CHECK_OK(lower->SetInt32(key_column, lower_bound));
+    unique_ptr<KuduPartialRow> upper(schema.NewRow());
+    CHECK_OK(upper->SetInt32(key_column, upper_bound));
     return unique_ptr<KuduTableCreator::KuduRangePartition>(
         new KuduTableCreator::KuduRangePartition(lower.release(),
                                                  upper.release()));
   }
 
+  RangePartition CreateRangePartition(int32_t lower_bound = 0,
+                                      int32_t upper_bound = 100) {
+    return CreateRangePartition(schema_, kKeyColumn, lower_bound, upper_bound);
+  }
+
   RangePartition CreateRangePartitionNoUpperBound(int32_t lower_bound) {
     unique_ptr<KuduPartialRow> lower(schema_.NewRow());
     CHECK_OK(lower->SetInt32(kKeyColumn, lower_bound));
@@ -1853,5 +1860,92 @@ TEST_F(FlexPartitioningAlterTableTest, 
AddDropTableWideHashSchemaPartitions) {
   ASSERT_OK(client_->DeleteTable(kTableName));
 }
 
+// Try adding range partition with custom hash schema where hash columns are
+// duplicated across different dimensions. That should not be possible and the
+// client should receive a proper error in response.
+TEST_F(FlexPartitioningAlterTableTest, AddRangeWithDuplicateHashColumns) {
+  constexpr const char* const kCol0 = "c0";
+  constexpr const char* const kCol1 = "c1";
+  constexpr const char* const kCol2 = "c2";
+  constexpr const char* const kErrMsg =
+      "hash bucket schema components must not contain columns in common";
+  constexpr const char* const kTableName = "AddRangeWithDuplicateHashColumns";
+
+
+  KuduSchema schema;
+  {
+    KuduSchemaBuilder b;
+    b.AddColumn(kCol0)->Type(KuduColumnSchema::INT32)->NotNull();
+    b.AddColumn(kCol1)->Type(KuduColumnSchema::INT32)->NotNull();
+    b.AddColumn(kCol2)->Type(KuduColumnSchema::STRING)->Nullable();
+    b.SetPrimaryKey({kCol0, kCol1});
+    ASSERT_OK(b.Build(&schema));
+  }
+
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  table_creator->table_name(kTableName)
+      .schema(&schema)
+      .num_replicas(1)
+      .add_hash_partitions({ kCol0 }, 2)
+      .add_hash_partitions({ kCol1 }, 3)
+      .set_range_partition_columns({ kCol0 });
+
+  // Add a range partition with the table-wide hash schema.
+  {
+    unique_ptr<KuduPartialRow> lower(schema.NewRow());
+    ASSERT_OK(lower->SetInt32(kCol0, 0));
+    unique_ptr<KuduPartialRow> upper(schema.NewRow());
+    ASSERT_OK(upper->SetInt32(kCol0, 111));
+    table_creator->add_range_partition(lower.release(), upper.release());
+  }
+  ASSERT_OK(table_creator->Create());
+
+  // Try to add hash partitions with duplicate hash columns across dimensions.
+
+  {
+    unique_ptr<KuduTableAlterer> alterer(client_->NewTableAlterer(kTableName));
+    auto p = CreateRangePartition(schema, kCol0, -111, 0);
+    ASSERT_OK(p->add_hash_partitions({ kCol0 }, 3, 1));
+    ASSERT_OK(p->add_hash_partitions({ kCol0 }, 5, 2));
+    alterer->AddRangePartition(p.release());
+    const auto s = alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), kErrMsg);
+  }
+
+  {
+    unique_ptr<KuduTableAlterer> alterer(client_->NewTableAlterer(kTableName));
+    auto p = CreateRangePartition(schema, kCol0, -111, 0);
+    ASSERT_OK(p->add_hash_partitions({ kCol0, kCol1 }, 3, 3));
+    ASSERT_OK(p->add_hash_partitions({ kCol0 }, 2, 4));
+    alterer->AddRangePartition(p.release());
+    const auto s = alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), kErrMsg);
+  }
+
+  {
+    unique_ptr<KuduTableAlterer> alterer(client_->NewTableAlterer(kTableName));
+    auto p = CreateRangePartition(schema, kCol0, -111, 0);
+    ASSERT_OK(p->add_hash_partitions({ kCol1 }, 5, 6));
+    ASSERT_OK(p->add_hash_partitions({ kCol0, kCol1 }, 7, 5));
+    alterer->AddRangePartition(p.release());
+    const auto s = alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), kErrMsg);
+  }
+
+  {
+    unique_ptr<KuduTableAlterer> alterer(client_->NewTableAlterer(kTableName));
+    auto p = CreateRangePartition(schema, kCol0, -111, 0);
+    ASSERT_OK(p->add_hash_partitions({ kCol0, kCol1 }, 3, 7));
+    ASSERT_OK(p->add_hash_partitions({ kCol1, kCol0 }, 7, 8));
+    alterer->AddRangePartition(p.release());
+    const auto s = alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), kErrMsg);
+  }
+}
+
 } // namespace client
 } // namespace kudu
diff --git a/src/kudu/common/partition.h b/src/kudu/common/partition.h
index bcfb1f2d1..7b0878b15 100644
--- a/src/kudu/common/partition.h
+++ b/src/kudu/common/partition.h
@@ -314,6 +314,10 @@ class PartitionSchema {
       PartitionSchema* partition_schema,
       RangesWithHashSchemas* ranges_with_hash_schemas = nullptr) 
WARN_UNUSED_RESULT;
 
+  // Helper function that validates the hash schemas.
+  static Status ValidateHashSchema(const Schema& schema,
+                                   const HashSchema& hash_schema);
+
   // Serializes a partition schema into a protobuf message.
   // Requires a schema to encode the range bounds.
   Status ToPB(const Schema& schema, PartitionSchemaPB* pb) const;
@@ -548,10 +552,6 @@ class PartitionSchema {
   static uint32_t HashValueForRow(const Row& row,
                                   const HashDimension& hash_dimension);
 
-  // Helper function that validates the hash schemas.
-  static Status ValidateHashSchema(const Schema& schema,
-                                   const HashSchema& hash_schema);
-
   // Generates hash partitions for each combination of hash buckets in 
hash_schemas.
   static std::vector<Partition> GenerateHashPartitions(
       const HashSchema& hash_schema,
diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index e52e4225b..3048abcad 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2693,6 +2693,7 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           return Status::NotSupported(
               "varying number of hash dimensions per range is not yet 
supported");
         }
+        RETURN_NOT_OK(PartitionSchema::ValidateHashSchema(schema, 
hash_schema));
         RETURN_NOT_OK(partition_schema.CreatePartitionsForRange(
             range_bound, hash_schema, schema, &partitions));
 
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index b82bbb1e0..7f2a61d83 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -988,15 +988,18 @@ TEST_P(AlterTableWithRangeSpecificHashSchema, 
TestAlterTableWithDifferentHashDim
     hash_dimension_pb->set_seed(hash_dimension.seed);
   }
 
-  ColumnSchemaPB* col1 = req.mutable_schema()->add_columns();
-  col1->set_name("key");
-  col1->set_type(INT32);
-  col1->set_is_key(true);
-
-  ColumnSchemaPB* col2 = req.mutable_schema()->add_columns();
-  col2->set_name("val");
-  col2->set_type(INT32);
-  col2->set_is_key(true);
+  {
+    auto* col = req.mutable_schema()->add_columns();
+    col->set_name("key");
+    col->set_type(INT32);
+    col->set_is_key(true);
+  }
+  {
+    auto* col = req.mutable_schema()->add_columns();
+    col->set_name("val");
+    col->set_type(INT32);
+    col->set_is_key(true);
+  }
 
   // Check the number of tablets in the table
   std::vector<scoped_refptr<TableInfo>> tables;
@@ -1046,6 +1049,21 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
         {}, {{{kCol0}, 2, 0}}, &create_table_resp));
   }
 
+  // Check the number of tablets in the table before ALTER TABLE.
+  {
+    CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
+    std::vector<scoped_refptr<TableInfo>> tables;
+    master_->catalog_manager()->GetAllTables(&tables);
+    ASSERT_EQ(1, tables.size());
+    // 2 tablets (because of 2 hash buckets) for already existing range.
+    ASSERT_EQ(2, tables.front()->num_tablets());
+
+    // Check the partition schema stored in the system catalog.
+    PartitionSchemaPB ps_pb;
+    ASSERT_OK(GetTablePartitionSchema(kTableName, &ps_pb));
+    ASSERT_EQ(0, ps_pb.custom_hash_schema_ranges_size());
+  }
+
   const auto& table_id = create_table_resp.table_id();
   const HashSchema custom_hash_schema{{{kCol0,kCol1}, 5, 1}};
 
@@ -1059,15 +1077,16 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
     // Add the required information on the table's schema:
     // key and non-null columns must be present in the request.
     {
-      ColumnSchemaPB* col0 = req.mutable_schema()->add_columns();
-      col0->set_name(kCol0);
-      col0->set_type(INT32);
-      col0->set_is_key(true);
-
-      ColumnSchemaPB* col1 = req.mutable_schema()->add_columns();
-      col1->set_name(kCol1);
-      col1->set_type(INT64);
-      col0->set_is_key(true);
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol0);
+      col->set_type(INT32);
+      col->set_is_key(true);
+    }
+    {
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol1);
+      col->set_type(INT64);
+      col->set_is_key(true);
     }
 
     AlterTableRequestPB::Step* step = req.add_alter_schema_steps();
@@ -1090,65 +1109,50 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
       hash_dimension_pb->set_seed(hash_dimension.seed);
     }
 
-    // Check the number of tablets in the table before ALTER TABLE.
-    {
-      CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
-      std::vector<scoped_refptr<TableInfo>> tables;
-      master_->catalog_manager()->GetAllTables(&tables);
-      ASSERT_EQ(1, tables.size());
-      // 2 tablets (because of 2 hash buckets) for already existing range.
-      ASSERT_EQ(2, tables.front()->num_tablets());
-
-      // Check the partition schema stored in the system catalog.
-      PartitionSchemaPB ps_pb;
-      ASSERT_OK(GetTablePartitionSchema(kTableName, &ps_pb));
-      ASSERT_EQ(0, ps_pb.custom_hash_schema_ranges_size());
-    }
-
     RpcController ctl;
     ASSERT_OK(proxy_->AlterTable(req, &resp, &ctl));
     ASSERT_FALSE(resp.has_error())
         << StatusFromPB(resp.error().status()).ToString();
+  }
 
-    // Check the number of tablets in the table after ALTER TABLE.
-    {
-      CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
-      std::vector<scoped_refptr<TableInfo>> tables;
-      master_->catalog_manager()->GetAllTables(&tables);
-      ASSERT_EQ(1, tables.size());
-      // Extra 5 tablets (because of 5 hash buckets) for newly added range.
-      ASSERT_EQ(7, tables.front()->num_tablets());
+  // Check the number of tablets in the table after ALTER TABLE.
+  {
+    CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
+    std::vector<scoped_refptr<TableInfo>> tables;
+    master_->catalog_manager()->GetAllTables(&tables);
+    ASSERT_EQ(1, tables.size());
+    // Extra 5 tablets (because of 5 hash buckets) for newly added range.
+    ASSERT_EQ(7, tables.front()->num_tablets());
 
-      // Check the partition schema stored in the system catalog.
-      PartitionSchemaPB ps_pb;
-      ASSERT_OK(GetTablePartitionSchema(kTableName, &ps_pb));
-      ASSERT_EQ(1, ps_pb.custom_hash_schema_ranges_size());
+    // Check the partition schema stored in the system catalog.
+    PartitionSchemaPB ps_pb;
+    ASSERT_OK(GetTablePartitionSchema(kTableName, &ps_pb));
+    ASSERT_EQ(1, ps_pb.custom_hash_schema_ranges_size());
 
-      // Check the hash schema parameters (i.e. columns and number of hash
-      // buckets) are stored and read back by the client as expected.
-      const auto& range = ps_pb.custom_hash_schema_ranges(0);
-      ASSERT_EQ(1, range.hash_schema_size());
-      const auto& hash_schema = range.hash_schema(0);
+    // Check the hash schema parameters (i.e. columns and number of hash
+    // buckets) are stored and read back by the client as expected.
+    const auto& range = ps_pb.custom_hash_schema_ranges(0);
+    ASSERT_EQ(1, range.hash_schema_size());
+    const auto& hash_schema = range.hash_schema(0);
 
-      ASSERT_EQ(5, hash_schema.num_buckets());
-      ASSERT_EQ(1, hash_schema.seed());
+    ASSERT_EQ(5, hash_schema.num_buckets());
+    ASSERT_EQ(1, hash_schema.seed());
 
-      ASSERT_EQ(2, hash_schema.columns_size());
-      const auto schema = kTableSchema.CopyWithColumnIds();
+    ASSERT_EQ(2, hash_schema.columns_size());
+    const auto schema = kTableSchema.CopyWithColumnIds();
 
-      const auto ref_col_0_id = int32_t(schema.column_id(0));
-      const auto& col_0 = hash_schema.columns(0);
-      ASSERT_TRUE(col_0.has_id());
-      ASSERT_EQ(ref_col_0_id, col_0.id());
+    const auto ref_col_0_id = int32_t(schema.column_id(0));
+    const auto& col_0 = hash_schema.columns(0);
+    ASSERT_TRUE(col_0.has_id());
+    ASSERT_EQ(ref_col_0_id, col_0.id());
 
-      const auto ref_col_1_id = int32_t(schema.column_id(1));
-      const auto& col_1 = hash_schema.columns(1);
-      ASSERT_TRUE(col_1.has_id());
-      ASSERT_EQ(ref_col_1_id, col_1.id());
-    }
+    const auto ref_col_1_id = int32_t(schema.column_id(1));
+    const auto& col_1 = hash_schema.columns(1);
+    ASSERT_TRUE(col_1.has_id());
+    ASSERT_EQ(ref_col_1_id, col_1.id());
   }
 
-  // Now verify the table's schema: fetch the information on the altered
+  // Verify the table's schema: fetch the information on the altered
   // table and make sure the schema contains information on the newly added
   // range partition with the custom hash schema.
   {
@@ -1200,7 +1204,7 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
     ASSERT_EQ(ref_col_1_id, column_ids[1]);
   }
 
-  // Now verify that everything works as expected when dropping a range
+  // Verify that everything works as expected when dropping a range
   // partition with custom hash schema.
   {
     AlterTableRequestPB req;
@@ -1210,15 +1214,16 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
     // Add the required information on the table's schema:
     // key and non-null columns must be present in the request.
     {
-      ColumnSchemaPB* col0 = req.mutable_schema()->add_columns();
-      col0->set_name(kCol0);
-      col0->set_type(INT32);
-      col0->set_is_key(true);
-
-      ColumnSchemaPB* col1 = req.mutable_schema()->add_columns();
-      col1->set_name(kCol1);
-      col1->set_type(INT64);
-      col0->set_is_key(true);
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol0);
+      col->set_type(INT32);
+      col->set_is_key(true);
+    }
+    {
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol1);
+      col->set_type(INT64);
+      col->set_is_key(true);
     }
 
     AlterTableRequestPB::Step* step = req.add_alter_schema_steps();
@@ -1321,19 +1326,21 @@ TEST_F(MasterTest, 
AlterTableAddDropRangeWithTableWideHashSchema) {
     // Add the required information on the table's schema:
     // key and non-null columns must be present in the request.
     {
-      ColumnSchemaPB* col0 = req.mutable_schema()->add_columns();
-      col0->set_name(kCol0);
-      col0->set_type(INT32);
-      col0->set_is_key(true);
-
-      ColumnSchemaPB* col1 = req.mutable_schema()->add_columns();
-      col1->set_name(kCol1);
-      col1->set_type(INT64);
-      col0->set_is_key(true);
-
-      ColumnSchemaPB* col2 = req.mutable_schema()->add_columns();
-      col2->set_name(kCol2);
-      col2->set_type(STRING);
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol0);
+      col->set_type(INT32);
+      col->set_is_key(true);
+    }
+    {
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol1);
+      col->set_type(INT64);
+      col->set_is_key(true);
+    }
+    {
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol2);
+      col->set_type(STRING);
     }
 
     AlterTableRequestPB::Step* step = req.add_alter_schema_steps();
@@ -1397,19 +1404,21 @@ TEST_F(MasterTest, 
AlterTableAddDropRangeWithTableWideHashSchema) {
     // Add the required information on the table's schema:
     // key and non-null columns must be present in the request.
     {
-      ColumnSchemaPB* col0 = req.mutable_schema()->add_columns();
-      col0->set_name(kCol0);
-      col0->set_type(INT32);
-      col0->set_is_key(true);
-
-      ColumnSchemaPB* col1 = req.mutable_schema()->add_columns();
-      col1->set_name(kCol1);
-      col1->set_type(INT64);
-      col0->set_is_key(true);
-
-      ColumnSchemaPB* col2 = req.mutable_schema()->add_columns();
-      col2->set_name(kCol2);
-      col2->set_type(STRING);
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol0);
+      col->set_type(INT32);
+      col->set_is_key(true);
+    }
+    {
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol1);
+      col->set_type(INT64);
+      col->set_is_key(true);
+    }
+    {
+      auto* col = req.mutable_schema()->add_columns();
+      col->set_name(kCol2);
+      col->set_type(STRING);
     }
 
     AlterTableRequestPB::Step* step = req.add_alter_schema_steps();

Reply via email to