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/02/16 21:28:01 UTC

[kudu] 02/02: [schema] use operator==() instead of Equals()

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

commit 5557c488e1c4264c20b764235b4830456f73ce3b
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Tue Feb 15 13:43:53 2022 -0800

    [schema] use operator==() instead of Equals()
    
    While looking at the code in src/kudu/common/schema.h in the scope of
    reviewing a prior changelist, I noticed that the comparison options
    for Schema::Equals() were used only in the test scenarios.  I took
    the liberty and moved the Equals() method into the test code and
    replaced it with the operator== at call sites.  This patch includes
    other minor cleanup, but overall the patch doesn't contain any
    functional changes.
    
    Change-Id: Ib0e3d75d216241efec49b64233198eafe52871cb
    Reviewed-on: http://gerrit.cloudera.org:8080/18237
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Abhishek Chennaka <ac...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/client/client.cc                      |  8 +--
 src/kudu/client/schema.cc                      |  3 +-
 src/kudu/common/generic_iterators.cc           |  4 +-
 src/kudu/common/row.h                          |  6 +-
 src/kudu/common/schema-test.cc                 | 88 +++++++++++++++++---------
 src/kudu/common/schema.h                       | 51 ++++++---------
 src/kudu/integration-tests/alter_table-test.cc |  4 +-
 src/kudu/master/master-test.cc                 |  2 +-
 src/kudu/master/sys_catalog.cc                 |  2 +-
 src/kudu/tablet/compaction-test.cc             |  2 +-
 src/kudu/tablet/tablet.cc                      |  2 +-
 src/kudu/tablet/tablet_metadata.cc             |  2 +-
 src/kudu/tserver/tablet_service.cc             |  4 +-
 13 files changed, 98 insertions(+), 80 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index e26a46c..4b22424 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1515,13 +1515,13 @@ KuduTableAlterer* KuduTableAlterer::AddRangePartitionWithDimension(
     data_->status_ = Status::InvalidArgument("range partition bounds may not be null");
     return this;
   }
-  if (!lower_bound->schema()->Equals(*upper_bound->schema())) {
+  if (*lower_bound->schema() != *upper_bound->schema()) {
     data_->status_ = Status::InvalidArgument("range partition bounds must have matching schemas");
     return this;
   }
   if (data_->schema_ == nullptr) {
     data_->schema_ = lower_bound->schema();
-  } else if (!lower_bound->schema()->Equals(*data_->schema_)) {
+  } else if (*lower_bound->schema() != *data_->schema_) {
     data_->status_ = Status::InvalidArgument("range partition bounds must have matching schemas");
     return this;
   }
@@ -1547,13 +1547,13 @@ KuduTableAlterer* KuduTableAlterer::DropRangePartition(
     data_->status_ = Status::InvalidArgument("range partition bounds may not be null");
     return this;
   }
-  if (!lower_bound->schema()->Equals(*upper_bound->schema())) {
+  if (*lower_bound->schema() != *upper_bound->schema()) {
     data_->status_ = Status::InvalidArgument("range partition bounds must have matching schemas");
     return this;
   }
   if (data_->schema_ == nullptr) {
     data_->schema_ = lower_bound->schema();
-  } else if (!lower_bound->schema()->Equals(*data_->schema_)) {
+  } else if (*lower_bound->schema() != *data_->schema_) {
     data_->status_ = Status::InvalidArgument("range partition bounds must have matching schemas");
     return this;
   }
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index 8c97e55..b53b5fe 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -878,8 +878,7 @@ Status KuduSchema::Reset(const vector<KuduColumnSchema>& columns, int key_column
 }
 
 bool KuduSchema::operator==(const KuduSchema& rhs) const {
-  return this == &rhs ||
-      (schema_ && rhs.schema_ && schema_->Equals(*rhs.schema_));
+  return this == &rhs || (schema_ && rhs.schema_ && (*schema_ == *rhs.schema_));
 }
 
 bool KuduSchema::operator!=(const KuduSchema& rhs) const {
diff --git a/src/kudu/common/generic_iterators.cc b/src/kudu/common/generic_iterators.cc
index c2c3851..ad9fa17 100644
--- a/src/kudu/common/generic_iterators.cc
+++ b/src/kudu/common/generic_iterators.cc
@@ -616,7 +616,7 @@ Status MergeIterator::Init(ScanSpec *spec) {
   finished_iter_stats_by_col_.resize(schema_->num_columns());
 #ifndef NDEBUG
   for (const auto& s : states_) {
-    if (!s.schema().Equals(*schema_)) {
+    if (s.schema() != *schema_) {
       return Status::InvalidArgument(
           Substitute("Schemas do not match: $0 vs. $1",
                      schema_->ToString(), s.schema().ToString()));
@@ -1008,7 +1008,7 @@ Status UnionIterator::Init(ScanSpec *spec) {
   finished_iter_stats_by_col_.resize(schema_->num_columns());
 #ifndef NDEBUG
   for (const auto& i : iters_) {
-    if (!i.iter->schema().Equals(*schema_)) {
+    if (i.iter->schema() != *schema_) {
       return Status::InvalidArgument(
           Substitute("Schemas do not match: $0 vs. $1",
                      schema_->ToString(), i.iter->schema().ToString()));
diff --git a/src/kudu/common/row.h b/src/kudu/common/row.h
index a2e3756..aa63241 100644
--- a/src/kudu/common/row.h
+++ b/src/kudu/common/row.h
@@ -144,7 +144,7 @@ class RowProjector {
   // The two Schema pointers must remain valid for the lifetime of this object.
   RowProjector(const Schema* base_schema, const Schema* projection)
     : base_schema_(base_schema), projection_(projection),
-      is_identity_(base_schema->Equals(*projection)) {
+      is_identity_(*base_schema == *projection) {
   }
 
   // Initialize the projection mapping with the specified base_schema and projection
@@ -157,7 +157,7 @@ class RowProjector {
     projection_ = projection;
     base_cols_mapping_.clear();
     projection_defaults_.clear();
-    is_identity_ = base_schema->Equals(*projection);
+    is_identity_ = (*base_schema == *projection);
     return Init();
   }
 
@@ -275,7 +275,7 @@ class DeltaProjector {
   // of the object.
   DeltaProjector(const Schema* delta_schema, const Schema* projection)
     : delta_schema_(delta_schema), projection_(projection),
-      is_identity_(delta_schema->Equals(*projection)) {
+      is_identity_(*delta_schema == *projection) {
   }
 
   Status Init() {
diff --git a/src/kudu/common/schema-test.cc b/src/kudu/common/schema-test.cc
index 251402b..ae79cf8 100644
--- a/src/kudu/common/schema-test.cc
+++ b/src/kudu/common/schema-test.cc
@@ -21,7 +21,6 @@
 #include <cstdint>
 #include <string>
 #include <tuple>  // IWYU pragma: keep
-#include <unordered_map>
 #include <utility>
 #include <vector>
 
@@ -44,19 +43,52 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-namespace kudu {
-namespace tablet {
-
 using std::string;
-using std::unordered_map;
 using std::vector;
 using strings::Substitute;
 
+namespace kudu {
+
+// Return true if the schemas have exactly the same set of columns
+// and respective types.
+bool EqualSchemas(const Schema& lhs, const Schema& rhs) {
+  if (lhs != rhs) {
+    return false;
+  }
+
+  if (lhs.num_key_columns_ != rhs.num_key_columns_) {
+    return false;
+  }
+  if (lhs.num_columns() != rhs.num_columns()) {
+    return false;
+  }
+  for (size_t i = 0; i < rhs.num_columns(); ++i) {
+    if (!lhs.cols_[i].Equals(rhs.cols_[i])) {
+      return false;
+    }
+  }
+
+  if (lhs.has_column_ids() != rhs.has_column_ids()) {
+    return false;
+  }
+  if (lhs.has_column_ids()) {
+    if (lhs.col_ids_ != rhs.col_ids_) {
+      return false;
+    }
+    if (lhs.max_col_id() != rhs.max_col_id()) {
+      return false;
+    }
+  }
+
+  return true;
+}
+
+namespace tablet {
+
 // Copy a row and its referenced data into the given Arena.
-static Status CopyRowToArena(const Slice &row,
-                             const Schema &schema,
-                             Arena *dst_arena,
-                             ContiguousRow *copied) {
+static Status CopyRowToArena(const Slice& row,
+                             Arena* dst_arena,
+                             ContiguousRow* copied) {
   Slice row_data;
 
   // Copy the direct row data to arena
@@ -65,8 +97,7 @@ static Status CopyRowToArena(const Slice &row,
   }
 
   copied->Reset(row_data.mutable_data());
-  RETURN_NOT_OK(RelocateIndirectDataToArena(copied, dst_arena));
-  return Status::OK();
+  return RelocateIndirectDataToArena(copied, dst_arena);
 }
 
 class TestSchema : public KuduTest {};
@@ -159,11 +190,10 @@ TEST_P(ParameterizedSchemaTest, TestCopyAndMove) {
 
   vector<ColumnSchema> cols = { col1, col2, col3 };
   vector<ColumnId> ids = { ColumnId(0), ColumnId(1), ColumnId(2) };
-  const int kNumKeyCols = 1;
+  constexpr int kNumKeyCols = 1;
 
-  Schema schema = GetParam() == INCLUDE_COL_IDS ?
-                      Schema(cols, ids, kNumKeyCols) :
-                      Schema(cols, kNumKeyCols);
+  const auto& schema = GetParam() == INCLUDE_COL_IDS
+      ? Schema(cols, ids, kNumKeyCols) : Schema(cols, kNumKeyCols);
 
   NO_FATALS(check_schema(schema));
 
@@ -172,7 +202,7 @@ TEST_P(ParameterizedSchemaTest, TestCopyAndMove) {
   {
     Schema copied_schema = schema;
     NO_FATALS(check_schema(copied_schema));
-    ASSERT_TRUE(copied_schema.Equals(schema, Schema::COMPARE_ALL));
+    ASSERT_TRUE(EqualSchemas(schema, copied_schema));
 
     // Move-assign to 'moved_to_schema' from 'copied_schema' and then let
     // 'copied_schema' go out of scope to make sure none of the 'moved_schema'
@@ -184,18 +214,18 @@ TEST_P(ParameterizedSchemaTest, TestCopyAndMove) {
     copied_schema.ToString(); // NOLINT(*)
   }
   NO_FATALS(check_schema(moved_schema));
-  ASSERT_TRUE(moved_schema.Equals(schema, Schema::COMPARE_ALL));
+  ASSERT_TRUE(EqualSchemas(schema, moved_schema));
 
   // Check copy- and move-construction.
   {
     Schema copied_schema(schema);
     NO_FATALS(check_schema(copied_schema));
-    ASSERT_TRUE(copied_schema.Equals(schema, Schema::COMPARE_ALL));
+    ASSERT_TRUE(EqualSchemas(schema, copied_schema));
 
     Schema moved_schema(std::move(copied_schema));
     copied_schema.ToString(); // NOLINT(*)
     NO_FATALS(check_schema(moved_schema));
-    ASSERT_TRUE(moved_schema.Equals(schema, Schema::COMPARE_ALL));
+    ASSERT_TRUE(EqualSchemas(schema, moved_schema));
   }
 }
 
@@ -254,10 +284,10 @@ TEST_F(TestSchema, TestSchemaEqualsWithDecimal) {
   Schema schema_17_10({ col1, col_17_10 }, 1);
   Schema schema_17_9({ col1, col_17_9 }, 1);
 
-  EXPECT_TRUE(schema_18_10.Equals(schema_18_10));
-  EXPECT_FALSE(schema_18_10.Equals(schema_18_9));
-  EXPECT_FALSE(schema_18_10.Equals(schema_17_10));
-  EXPECT_FALSE(schema_18_10.Equals(schema_17_9));
+  EXPECT_EQ(schema_18_10, schema_18_10);
+  EXPECT_NE(schema_18_10, schema_18_9);
+  EXPECT_NE(schema_18_10, schema_17_10);
+  EXPECT_NE(schema_18_10, schema_17_9);
 }
 
 TEST_F(TestSchema, TestColumnSchemaEquals) {
@@ -294,14 +324,14 @@ TEST_F(TestSchema, TestSchemaEquals) {
                    ColumnSchema("col2", UINT32),
                    ColumnSchema("col3", UINT32, false) },
                  2);
-  ASSERT_FALSE(schema1.Equals(schema2));
+  ASSERT_NE(schema1, schema2);
   ASSERT_TRUE(schema1.KeyEquals(schema1));
   ASSERT_TRUE(schema1.KeyEquals(schema2, ColumnSchema::COMPARE_TYPE));
   ASSERT_FALSE(schema1.KeyEquals(schema2, ColumnSchema::COMPARE_NAME));
   ASSERT_TRUE(schema1.KeyTypeEquals(schema2));
   ASSERT_FALSE(schema2.KeyTypeEquals(schema3));
-  ASSERT_FALSE(schema3.Equals(schema4));
-  ASSERT_TRUE(schema4.Equals(schema4));
+  ASSERT_NE(schema3, schema4);
+  ASSERT_EQ(schema4, schema4);
   ASSERT_TRUE(schema3.KeyEquals(schema4, ColumnSchema::COMPARE_NAME_AND_TYPE));
 }
 
@@ -470,7 +500,7 @@ TEST_F(TestSchema, TestGetMappedReadProjection) {
   ASSERT_EQ(1, mapped.num_key_columns());
   ASSERT_EQ(2, mapped.num_columns());
   ASSERT_TRUE(mapped.has_column_ids());
-  ASSERT_FALSE(mapped.Equals(projection, Schema::COMPARE_ALL));
+  ASSERT_FALSE(EqualSchemas(mapped, projection));
 
   // The column id for the 'key' column in the mapped projection should match
   // the one from the tablet schema.
@@ -521,7 +551,7 @@ TEST_F(TestSchema, TestRowOperations) {
   rb.AddUint32(3);
   rb.AddInt32(-3);
   ContiguousRow row_a(&schema);
-  ASSERT_OK(CopyRowToArena(rb.data(), schema, &arena, &row_a));
+  ASSERT_OK(CopyRowToArena(rb.data(), &arena, &row_a));
 
   rb.Reset();
   rb.AddString(string("row_b_1"));
@@ -529,7 +559,7 @@ TEST_F(TestSchema, TestRowOperations) {
   rb.AddUint32(3);
   rb.AddInt32(-3);
   ContiguousRow row_b(&schema);
-  ASSERT_OK(CopyRowToArena(rb.data(), schema, &arena, &row_b));
+  ASSERT_OK(CopyRowToArena(rb.data(), &arena, &row_b));
 
   ASSERT_GT(schema.Compare(row_b, row_a), 0);
   ASSERT_LT(schema.Compare(row_a, row_b), 0);
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index c00e66d..725a0e8 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -50,12 +50,12 @@ namespace kudu {
 class Schema;
 }  // namespace kudu
 
-// Check that two schemas are equal, yielding a useful error message in the case that
-// they are not.
+// Check that two schemas are equal, yielding a useful error message
+// if they are not.
 #define DCHECK_SCHEMA_EQ(s1, s2) \
   do { \
-    DCHECK((s1).Equals((s2))) << "Schema " << (s1).ToString() \
-                              << " does not match " << (s2).ToString(); \
+    DCHECK((s1) == (s2)) << "Schema " << (s1).ToString() \
+                           << " does not match " << (s2).ToString(); \
   } while (0)
 
 #define DCHECK_KEY_PROJECTION_SCHEMA_EQ(s1, s2) \
@@ -787,42 +787,28 @@ class Schema {
   // so should only be used when necessary for output.
   std::string ToString(ToStringMode mode = ToStringMode::WITH_COLUMN_IDS) const;
 
-  // Compare column ids in Equals() method.
-  enum SchemaComparisonType {
-    COMPARE_COLUMNS = 1 << 0,
-    COMPARE_COLUMN_IDS = 1 << 1,
-
-    COMPARE_ALL = COMPARE_COLUMNS | COMPARE_COLUMN_IDS
-  };
-
-  // Return true if the schemas have exactly the same set of columns
-  // and respective types.
-  bool Equals(const Schema& other, SchemaComparisonType flags = COMPARE_COLUMNS) const {
-    if (this == &other) return true;
+  bool operator==(const Schema& other) const {
+    if (this == &other) {
+      return true;
+    }
 
-    if (flags & COMPARE_COLUMNS) {
-      if (this->num_key_columns_ != other.num_key_columns_) return false;
-      if (this->num_columns() != other.num_columns()) return false;
-      for (size_t i = 0; i < other.num_columns(); i++) {
-        if (!this->cols_[i].Equals(other.cols_[i])) return false;
-      }
+    if (this->num_key_columns_ != other.num_key_columns_) {
+      return false;
     }
 
-    if (flags & COMPARE_COLUMN_IDS) {
-      if (this->has_column_ids() != other.has_column_ids()) return false;
-      if (this->has_column_ids()) {
-        if (this->col_ids_ != other.col_ids_) return false;
-        if (this->max_col_id() != other.max_col_id()) return false;
+    const size_t num_columns = this->num_columns();
+    if (num_columns != other.num_columns()) {
+      return false;
+    }
+    for (size_t i = 0; i < num_columns; ++i) {
+      if (!this->cols_[i].Equals(other.cols_[i])) {
+        return false;
       }
     }
 
     return true;
   }
 
-  bool operator==(const Schema& other) const {
-    return this->Equals(other);
-  }
-
   bool operator!=(const Schema& other) const {
     return !(*this == other);
   }
@@ -967,6 +953,9 @@ class Schema {
 
   friend class SchemaBuilder;
 
+  // 'Deep' compare two schemas: this is used by Schema's unit tests.
+  friend bool EqualSchemas(const Schema&, const Schema&);
+
   std::vector<ColumnSchema> cols_;
   size_t num_key_columns_;
   std::vector<ColumnId> col_ids_;
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 0c2e79f..c356bda 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -307,8 +307,8 @@ class AlterTableTest : public KuduTest {
         } else {
           ASSERT_EQ(first_node_replica->tablet()->tablet_id(),
             cur_node_replica->tablet()->tablet_id());
-          ASSERT_TRUE(first_node_replica->tablet()->schema()->Equals(
-            *(cur_node_replica->tablet()->schema())));
+          ASSERT_EQ(*(first_node_replica->tablet()->schema()),
+                    *(cur_node_replica->tablet()->schema()));
           if (verify_row_count == VerifyRowCount::kEnable) {
             uint64_t cur_count = 0;
             ASSERT_OK(cur_node_replica->CountLiveRows(&cur_count));
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 289c12f..537686b 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -1398,7 +1398,7 @@ TEST_F(MasterTest, TestGetTableSchemaIsAtomicWithCreateTable) {
       } else {
         Schema receivedSchema;
         CHECK_OK(SchemaFromPB(resp.schema(), &receivedSchema));
-        CHECK(kTableSchema.Equals(receivedSchema)) <<
+        CHECK(kTableSchema == receivedSchema) <<
             strings::Substitute("$0 not equal to $1",
                                 kTableSchema.ToString(), receivedSchema.ToString());
         CHECK_EQ(kTableName, resp.table_name());
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 74662ce..0d417a5 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -252,7 +252,7 @@ Status SysCatalogTable::Load(FsManager *fs_manager) {
   RETURN_NOT_OK(tablet::TabletMetadata::Load(fs_manager, kSysCatalogTabletId, &metadata));
 
   // Verify that the schema is the current one
-  if (!metadata->schema().Equals(BuildTableSchema())) {
+  if (metadata->schema() != BuildTableSchema()) {
     // TODO: In this case we probably should execute the migration step.
     return(Status::Corruption("Unexpected schema", metadata->schema().ToString()));
   }
diff --git a/src/kudu/tablet/compaction-test.cc b/src/kudu/tablet/compaction-test.cc
index 4ccba4c..654b95c 100644
--- a/src/kudu/tablet/compaction-test.cc
+++ b/src/kudu/tablet/compaction-test.cc
@@ -184,7 +184,7 @@ class TestCompaction : public KuduRowSetTest {
                      int row_key,
                      int32_t val) {
     BuildRow(row_key, val);
-    if (!mrs->schema().Equals(*row_builder_.schema())) {
+    if (*row_builder_.schema() != mrs->schema()) {
       // The MemRowSet is not projecting the row, so must be done by the caller
       RowProjector projector(row_builder_.schema(), &mrs->schema());
       uint8_t rowbuf[ContiguousRowHelper::row_size(mrs->schema())];
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index cbeb2ff..28bb0cf 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1565,7 +1565,7 @@ Status Tablet::AlterSchema(AlterSchemaOpState* op_state) {
   std::lock_guard<Semaphore> lock(rowsets_flush_sem_);
 
   // If the current version >= new version, there is nothing to do.
-  bool same_schema = schema()->Equals(*op_state->schema());
+  const bool same_schema = (*schema() == *op_state->schema());
   if (metadata_->schema_version() >= op_state->schema_version()) {
     const string msg =
         Substitute("Skipping requested alter to schema version $0, tablet already "
diff --git a/src/kudu/tablet/tablet_metadata.cc b/src/kudu/tablet/tablet_metadata.cc
index aabcb7d..0a36f1c 100644
--- a/src/kudu/tablet/tablet_metadata.cc
+++ b/src/kudu/tablet/tablet_metadata.cc
@@ -167,7 +167,7 @@ Status TabletMetadata::LoadOrCreate(FsManager* fs_manager,
                                     scoped_refptr<TabletMetadata>* metadata) {
   Status s = Load(fs_manager, tablet_id, metadata);
   if (s.ok()) {
-    if (!(*metadata)->schema().Equals(schema)) {
+    if ((*metadata)->schema() != schema) {
       return Status::Corruption(Substitute("Schema on disk ($0) does not "
         "match expected schema ($1)", (*metadata)->schema().ToString(),
         schema.ToString()));
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index d37bf08..16c73ba 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -1170,8 +1170,8 @@ void TabletServiceAdminImpl::AlterSchema(const AlterSchemaRequestPB* req,
       return;
     }
 
-    Schema tablet_schema = replica->tablet_metadata()->schema();
-    if (req_schema.Equals(tablet_schema)) {
+    const auto& tablet_schema = replica->tablet_metadata()->schema();
+    if (req_schema == tablet_schema) {
       context->RespondSuccess();
       return;
     }