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/08/12 04:05:06 UTC

[kudu] branch master updated: KUDU-3306: fix PartitionSchema::DecodeRangeKey

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 77d3ea465 KUDU-3306: fix PartitionSchema::DecodeRangeKey
77d3ea465 is described below

commit 77d3ea465bb1a1fa778cccd8432553032f51bd27
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Thu Jul 14 20:36:24 2022 +0800

    KUDU-3306: fix PartitionSchema::DecodeRangeKey
    
    Prior to this patch, we didn't copy BINARY values into KuduPartialRow
    in PartitionSchema::DecodeRangeKey, which lead to issues while copying
    tables with string, binary and varchar columns in their range key.
    
    Change-Id: I5abf7ff9713ee391199c47710ccd81db8b97653e
    Reviewed-on: http://gerrit.cloudera.org:8080/18733
    Reviewed-by: Alexey Serbin <al...@apache.org>
    Tested-by: Alexey Serbin <al...@apache.org>
---
 src/kudu/common/partition.cc     |  6 ++++++
 src/kudu/tools/kudu-tool-test.cc | 46 ++++++++++++++++++++++++++++++++++++++--
 2 files changed, 50 insertions(+), 2 deletions(-)

diff --git a/src/kudu/common/partition.cc b/src/kudu/common/partition.cc
index f666566d6..475d9d0ed 100644
--- a/src/kudu/common/partition.cc
+++ b/src/kudu/common/partition.cc
@@ -825,6 +825,12 @@ Status PartitionSchema::DecodeRangeKey(Slice* encoded_key,
                                      column.name()));
     // Mark the column as set.
     BitmapSet(partial_row->isset_bitmap_, column_idx);
+
+    if (column.type_info()->physical_type() == BINARY) {
+      // Copy cell value into the 'partial_row', because in the decoding process above, we just make
+      // row data a pointer to the memory allocated by arena.
+      partial_row->Set(column_idx, cont_row.cell_ptr(column_idx));
+    }
   }
   if (!encoded_key->empty()) {
     return Status::InvalidArgument("unable to fully decode range key",
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index e636dcd36..c668b648c 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -777,7 +777,8 @@ enum RunCopyTableCheckArgsType {
   kTestCopyTableSchemaOnly,
   kTestCopyTableComplexSchema,
   kTestCopyUnpartitionedTable,
-  kTestCopyTablePredicates
+  kTestCopyTablePredicates,
+  kTestCopyTableWithStringBounds
 };
 // Subclass of ToolTest that allows running individual test cases with different parameters to run
 // 'kudu table copy' CLI tool.
@@ -810,6 +811,13 @@ class ToolTestCopyTableParameterized :
       KuduSchema schema;
       ASSERT_OK(CreateUnpartitionedTable(&schema));
       ww.set_schema(schema);
+    } else if (test_case_ == kTestCopyTableWithStringBounds) {
+      // Regression for KUDU-3306, verify copying a table with string columns in its range key.
+      KuduSchema schema;
+      ASSERT_OK(CreateTableWithStringBounds(&schema));
+      ww.set_schema(schema);
+      ww.Setup();
+      return;
     }
     ww.Setup();
     ww.Start();
@@ -977,6 +985,10 @@ class ToolTestCopyTableParameterized :
         }
         return multi_args;
       }
+      case kTestCopyTableWithStringBounds:
+        args.mode = TableCopyMode::COPY_SCHEMA_ONLY;
+        args.columns = "";
+        return {args};
       default:
         LOG(FATAL) << "Unknown type " << test_case_;
     }
@@ -1064,6 +1076,35 @@ class ToolTestCopyTableParameterized :
     return Status::OK();
   }
 
+  Status CreateTableWithStringBounds(KuduSchema* schema) {
+    shared_ptr<KuduClient> client;
+    RETURN_NOT_OK(cluster_->CreateClient(nullptr, &client));
+    unique_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
+    *schema = KuduSchema::FromSchema(
+        Schema({ColumnSchema("int_key", INT32), ColumnSchema("string_key", STRING)}, 2));
+
+    unique_ptr<KuduPartialRow> first_lower_bound(schema->NewRow());
+    unique_ptr<KuduPartialRow> first_upper_bound(schema->NewRow());
+    RETURN_NOT_OK(first_lower_bound->SetStringNoCopy("string_key", "2020-01-01"));
+    RETURN_NOT_OK(first_upper_bound->SetStringNoCopy("string_key", "2020-01-01"));
+
+    unique_ptr<KuduPartialRow> second_lower_bound(schema->NewRow());
+    unique_ptr<KuduPartialRow> second_upper_bound(schema->NewRow());
+    RETURN_NOT_OK(second_lower_bound->SetStringNoCopy("string_key", "2021-01-01"));
+    RETURN_NOT_OK(second_upper_bound->SetStringNoCopy("string_key", "2021-01-01"));
+    KuduTableCreator::RangePartitionBound bound_type = KuduTableCreator::INCLUSIVE_BOUND;
+
+    return table_creator->table_name(kTableName)
+        .schema(schema)
+        .set_range_partition_columns({"string_key"})
+        .add_range_partition(
+            first_lower_bound.release(), first_upper_bound.release(), bound_type, bound_type)
+        .add_range_partition(
+            second_lower_bound.release(), second_upper_bound.release(), bound_type, bound_type)
+        .num_replicas(1)
+        .Create();
+  }
+
   void InsertOneRowWithNullCell() {
     shared_ptr<KuduClient> client;
     ASSERT_OK(cluster_->CreateClient(nullptr, &client));
@@ -1099,7 +1140,8 @@ INSTANTIATE_TEST_SUITE_P(CopyTableParameterized,
                                            kTestCopyTableSchemaOnly,
                                            kTestCopyTableComplexSchema,
                                            kTestCopyUnpartitionedTable,
-                                           kTestCopyTablePredicates));
+                                           kTestCopyTablePredicates,
+                                           kTestCopyTableWithStringBounds));
 
 void ToolTest::StartExternalMiniCluster(ExternalMiniClusterOptions opts) {
   cluster_.reset(new ExternalMiniCluster(std::move(opts)));