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/09/24 23:12:14 UTC

[kudu] branch master updated: KUDU-3353 [schema] Support to modify immutable attribute on column schema

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 0c71a0707 KUDU-3353 [schema] Support to modify immutable attribute on column schema
0c71a0707 is described below

commit 0c71a070727bc0c645eed1d661822c9286fe461f
Author: Yingchun Lai <la...@apache.org>
AuthorDate: Fri Sep 16 22:00:24 2022 +0800

    KUDU-3353 [schema] Support to modify immutable attribute on column schema
    
    In the previous implementation of adding immutable attribute
    to column on C++ client side ec3a9f75b6924a70ecbf08e3805228ad9b92b9f0,
    it missed to alter column schema to add/remove immutable attribute.
    
    This patch implements it and adds some tests.
    
    Change-Id: I213957bfb3a87b8e1a92cfd340a2e806d192f9ba
    Reviewed-on: http://gerrit.cloudera.org:8080/18996
    Reviewed-by: Alexey Serbin <al...@apache.org>
    Tested-by: Alexey Serbin <al...@apache.org>
---
 src/kudu/client/client-test.cc                 | 31 ++++++++++++++---
 src/kudu/client/client.cc                      |  4 ++-
 src/kudu/client/schema.cc                      |  4 +++
 src/kudu/common/common.proto                   |  1 +
 src/kudu/common/schema.cc                      |  3 ++
 src/kudu/common/schema.h                       |  7 +++-
 src/kudu/common/wire_protocol.cc               |  6 ++++
 src/kudu/integration-tests/alter_table-test.cc | 47 +++++++++++++++++++++++---
 8 files changed, 93 insertions(+), 10 deletions(-)

diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index f8eef831d..071f88ccd 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -9226,7 +9226,7 @@ TEST_P(ClientTestImmutableColumn, TestUpsert) {
   }
 }
 
-class ClientTestImmutableColumnCompatablity : public ClientTest {
+class ClientTestImmutableColumnCompatibility : public ClientTest {
  public:
   void SetUp() override {
     // Disable the immutable column attribute feature in master for testing.
@@ -9243,7 +9243,7 @@ class ClientTestImmutableColumnCompatablity : public ClientTest {
   }
 };
 
-TEST_F(ClientTestImmutableColumnCompatablity, CreateTable) {
+TEST_F(ClientTestImmutableColumnCompatibility, CreateTable) {
   const string kTableName = "create_table_with_immutable_attribute_column";
   KuduSchemaBuilder b;
   b.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
@@ -9263,8 +9263,8 @@ TEST_F(ClientTestImmutableColumnCompatablity, CreateTable) {
                  "CreateTable with feature(s) IMMUTABLE_COLUMN_ATTRIBUTE", kTableName));
 }
 
-TEST_F(ClientTestImmutableColumnCompatablity, AlterTable) {
-  const string kTableName = "alter_table_with_immutable_attribute_column";
+TEST_F(ClientTestImmutableColumnCompatibility, AlterTableAddColumn) {
+  const string kTableName = "alter_table_adding_column_with_immutable_attribute";
   KuduSchemaBuilder b;
   b.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
   b.AddColumn("int_val")->Type(KuduColumnSchema::INT32)->Nullable();
@@ -9286,6 +9286,29 @@ TEST_F(ClientTestImmutableColumnCompatablity, AlterTable) {
       "cluster does not support AlterTable with feature(s) IMMUTABLE_COLUMN_ATTRIBUTE");
 }
 
+TEST_F(ClientTestImmutableColumnCompatibility, AlterTableAlterColumn) {
+  const string kTableName = "alter_table_altering_column_with_immutable_attribute";
+  KuduSchemaBuilder b;
+  b.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
+  b.AddColumn("int_val")->Type(KuduColumnSchema::INT32)->Nullable();
+  ASSERT_OK(b.Build(&schema_));
+
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  ASSERT_OK(table_creator->table_name(kTableName)
+                .schema(&schema_)
+                .add_hash_partitions({"key"}, 2)
+                .num_replicas(1)
+                .Create());
+
+  unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+  table_alterer->AlterColumn("int_val")->Immutable();
+  Status s = table_alterer->Alter();
+  ASSERT_TRUE(s.IsNotSupported()) << s.ToString();
+  ASSERT_STR_CONTAINS(
+      s.ToString(),
+      "cluster does not support AlterTable with feature(s) IMMUTABLE_COLUMN_ATTRIBUTE");
+}
+
 class ClientTestUnixSocket : public ClientTest {
  public:
   void SetUp() override {
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 1c1dd95b5..0706277ee 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1684,7 +1684,9 @@ Status KuduTableAlterer::Alter() {
 
   bool has_immutable_column_schema = false;
   for (const auto& step : data_->steps_) {
-    if (step.step_type == AlterTableRequestPB::ADD_COLUMN && step.spec->data_->immutable) {
+    if ((step.step_type == AlterTableRequestPB::ADD_COLUMN ||
+         step.step_type == AlterTableRequestPB::ALTER_COLUMN) &&
+        step.spec->data_->immutable) {
       has_immutable_column_schema = true;
       break;
     }
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index 5576137bf..203f360e2 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -528,6 +528,10 @@ Status KuduColumnSpec::ToColumnSchemaDelta(ColumnSchemaDelta* col_delta) const {
     col_delta->compression = ToInternalCompressionType(data_->compression.value());
   }
 
+  if (data_->immutable) {
+    col_delta->immutable = data_->immutable.value();
+  }
+
   col_delta->new_name = std::move(data_->rename_to);
   col_delta->cfile_block_size = std::move(data_->block_size);
   col_delta->new_comment = std::move(data_->comment);
diff --git a/src/kudu/common/common.proto b/src/kudu/common/common.proto
index 2d43328c0..8117b65e4 100644
--- a/src/kudu/common/common.proto
+++ b/src/kudu/common/common.proto
@@ -153,6 +153,7 @@ message ColumnSchemaDeltaPB {
   optional int32 block_size = 8;
 
   optional string new_comment = 9;
+  optional bool immutable = 10 [default = false];
 }
 
 message SchemaPB {
diff --git a/src/kudu/common/schema.cc b/src/kudu/common/schema.cc
index 7fa54cbfa..dc7adf0fd 100644
--- a/src/kudu/common/schema.cc
+++ b/src/kudu/common/schema.cc
@@ -153,6 +153,9 @@ Status ColumnSchema::ApplyDelta(const ColumnSchemaDelta& col_delta) {
   if (col_delta.new_comment) {
     comment_ = col_delta.new_comment.value();
   }
+  if (col_delta.immutable) {
+    is_immutable_ = col_delta.immutable.value();
+  }
   return Status::OK();
 }
 
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index b72bca505..21ea34506 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -197,6 +197,8 @@ public:
   std::optional<int32_t> cfile_block_size;
 
   std::optional<std::string> new_comment;
+
+  std::optional<bool> immutable;
 };
 
 // The schema for a given column.
@@ -338,7 +340,6 @@ class ColumnSchema {
   bool EqualsType(const ColumnSchema& other) const {
     if (this == &other) return true;
     return is_nullable_ == other.is_nullable_ &&
-           is_immutable_ == other.is_immutable_ &&
            type_info()->type() == other.type_info()->type() &&
            type_attributes().EqualsForType(other.type_attributes(), type_info()->type());
   }
@@ -382,6 +383,10 @@ class ColumnSchema {
       if (comment_ != other.comment_) {
         return false;
       }
+
+      if (is_immutable_ != other.is_immutable_) {
+        return false;
+      }
     }
     return true;
   }
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index c82a85049..1c7625ce5 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -367,6 +367,9 @@ void ColumnSchemaDeltaToPB(const ColumnSchemaDelta& col_delta, ColumnSchemaDelta
   if (col_delta.new_comment) {
     pb->set_new_comment(*col_delta.new_comment);
   }
+  if (col_delta.immutable) {
+    pb->set_immutable(*col_delta.immutable);
+  }
 }
 
 ColumnSchemaDelta ColumnSchemaDeltaFromPB(const ColumnSchemaDeltaPB& pb) {
@@ -392,6 +395,9 @@ ColumnSchemaDelta ColumnSchemaDeltaFromPB(const ColumnSchemaDeltaPB& pb) {
   if (pb.has_new_comment()) {
     col_delta.new_comment = pb.new_comment();
   }
+  if (pb.has_immutable()) {
+    col_delta.immutable = pb.immutable();
+  }
   return col_delta;
 }
 
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 9e05280a1..9d2e93fd0 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -332,7 +332,7 @@ class AlterTableTest : public KuduTest {
 
   void VerifyRows(int start_row, int num_rows, VerifyPattern pattern);
 
-  void InsertRows(int start_row, int num_rows);
+  void InsertRows(int start_row, int num_rows, bool only_set_first_col = true);
   void DeleteRow(int row_key);
 
   Status InsertRowsSequential(const string& table_name, int start_row, int num_rows);
@@ -709,7 +709,8 @@ TEST_F(AlterTableTest, TestGetSchemaAfterAlterTable) {
   ASSERT_OK(client_->GetTableSchema(kTableName, &s));
 }
 
-void AlterTableTest::InsertRows(int start_row, int num_rows) {
+void AlterTableTest::InsertRows(int start_row, int num_rows,
+                                bool only_set_first_col) {
   shared_ptr<KuduSession> session = client_->NewSession();
   shared_ptr<KuduTable> table;
   CHECK_OK(session->SetFlushMode(KuduSession::AUTO_FLUSH_BACKGROUND));
@@ -725,8 +726,13 @@ void AlterTableTest::InsertRows(int start_row, int num_rows) {
     int32_t key = bswap_32(i);
     CHECK_OK(insert->mutable_row()->SetInt32(0, key));
 
-    if (table->schema().num_columns() > 1) {
-      CHECK_OK(insert->mutable_row()->SetInt32(1, i));
+    for (int cid = 1; cid < table->schema().num_columns(); cid++) {
+      if (table->schema().Column(cid).type() == KuduColumnSchema::INT32) {
+        CHECK_OK(insert->mutable_row()->SetInt32(cid, i));
+        if (only_set_first_col) {
+          break;
+        }
+      }
     }
 
     CHECK_OK(session->Apply(insert.release()));
@@ -2562,6 +2568,14 @@ TEST_F(AlterTableTest, AddAndRemoveImmutableAttribute) {
     ASSERT_OK(table_alterer->Alter());
   }
 
+  shared_ptr<KuduTable> table;
+  ASSERT_OK(client_->OpenTable(kTableName, &table));
+  // Here we use the first column to initialize an object of KuduColumnSchema
+  // for there is no default constructor for it.
+  KuduColumnSchema col_schema = table->schema().Column(0);
+  ASSERT_TRUE(table->schema().HasColumn("c1", &col_schema));
+  ASSERT_TRUE(col_schema.is_immutable());
+
   InsertRows(1, 1);
   ASSERT_OK(tablet_replica_->tablet()->Flush());
 
@@ -2577,6 +2591,10 @@ TEST_F(AlterTableTest, AddAndRemoveImmutableAttribute) {
     ASSERT_OK(table_alterer->Alter());
   }
 
+  ASSERT_OK(client_->OpenTable(kTableName, &table));
+  ASSERT_TRUE(table->schema().HasColumn("c1", &col_schema));
+  ASSERT_FALSE(col_schema.is_immutable());
+
   InsertRows(2, 1);
   ASSERT_OK(tablet_replica_->tablet()->Flush());
 
@@ -2585,6 +2603,27 @@ TEST_F(AlterTableTest, AddAndRemoveImmutableAttribute) {
   EXPECT_EQ("(int32 c0=0, int32 c1=0)", rows[0]);
   EXPECT_EQ("(int32 c0=16777216, int32 c1=1)", rows[1]);
   EXPECT_EQ("(int32 c0=33554432, int32 c1=2)", rows[2]);
+
+  {
+    // Add a column with immutable attribute.
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    table_alterer->AddColumn("c2")->Type(KuduColumnSchema::INT32)->Immutable();
+    ASSERT_OK(table_alterer->Alter());
+  }
+
+  ASSERT_OK(client_->OpenTable(kTableName, &table));
+  ASSERT_TRUE(table->schema().HasColumn("c2", &col_schema));
+  ASSERT_TRUE(col_schema.is_immutable());
+
+  InsertRows(3, 1, false);
+  ASSERT_OK(tablet_replica_->tablet()->Flush());
+
+  ScanToStrings(&rows);
+  ASSERT_EQ(4, rows.size());
+  EXPECT_EQ("(int32 c0=0, int32 c1=0, int32 c2=NULL)", rows[0]);
+  EXPECT_EQ("(int32 c0=16777216, int32 c1=1, int32 c2=NULL)", rows[1]);
+  EXPECT_EQ("(int32 c0=33554432, int32 c1=2, int32 c2=NULL)", rows[2]);
+  EXPECT_EQ("(int32 c0=50331648, int32 c1=3, int32 c2=3)", rows[3]);
 }
 
 } // namespace kudu