You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2019/02/28 01:13:22 UTC

[kudu] 02/04: schema: Fix ColumnSchema::Equals() usage

This is an automated email from the ASF dual-hosted git repository.

mpercy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit e3da9518787732f84d29ea7e9622745088ecdbba
Author: Mike Percy <mp...@apache.org>
AuthorDate: Tue Feb 26 10:45:36 2019 -0800

    schema: Fix ColumnSchema::Equals() usage
    
    In commit 4be39feef4907550c16f9d2b34430d098b9f1b21 the signature of
    ColumnSchema::Equals() was changed from taking a boolean flag for
    checking whether the defaults matched to taking an enum for describing a
    broader set of comparison options. However, a few call sites were not
    updated when that change was made, and due to type coercion several call
    sites ended up with weaker comparison semantics.
    
    This patch reinstates the behavior prior to the merge of the
    above-mentioned patch for the problematic call sites and updates the
    function signature of ColumnSchema::Equals() to treat the comparison
    enum as strongly typed to avoid similar problems in the future.
    
    Updated tests to add coverage for the relevant call sites.
    
    Change-Id: I416e4a44d0ed259d251d52a663b72acd79b46f33
    Reviewed-on: http://gerrit.cloudera.org:8080/12605
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/client/client-unittest.cc       | 19 +++++++++++++
 src/kudu/client/schema.cc                |  2 +-
 src/kudu/client/schema.h                 |  5 ++++
 src/kudu/common/column_predicate-test.cc | 48 ++++++++++++++++++++++++++++++++
 src/kudu/common/column_predicate.cc      |  4 +--
 src/kudu/common/column_predicate.h       |  9 ++++--
 src/kudu/common/schema-test.cc           |  3 +-
 src/kudu/common/schema.h                 |  8 +++---
 8 files changed, 87 insertions(+), 11 deletions(-)

diff --git a/src/kudu/client/client-unittest.cc b/src/kudu/client/client-unittest.cc
index 3112ca8..3cf76f8 100644
--- a/src/kudu/client/client-unittest.cc
+++ b/src/kudu/client/client-unittest.cc
@@ -312,5 +312,24 @@ TEST(ClientUnitTest, TestKuduSchemaToStringWithColumnIds) {
             kudu_schema.ToString());
 }
 
+TEST(KuduColumnSchemaTest, TestEquals) {
+  KuduColumnSchema a32("a", KuduColumnSchema::INT32);
+  ASSERT_TRUE(a32.Equals(a32));
+
+  KuduColumnSchema a32_2(a32);
+  ASSERT_TRUE(a32.Equals(a32_2));
+
+  KuduColumnSchema b32("b", KuduColumnSchema::INT32);
+  ASSERT_FALSE(a32.Equals(b32));
+
+  KuduColumnSchema a16("a", KuduColumnSchema::INT16);
+  ASSERT_FALSE(a32.Equals(a16));
+
+  const int kDefaultOf7 = 7;
+  KuduColumnSchema a32_dflt("a", KuduColumnSchema::INT32, /*is_nullable=*/false,
+                              /*default_value=*/&kDefaultOf7);
+  ASSERT_FALSE(a32.Equals(a32_dflt));
+}
+
 } // namespace client
 } // namespace kudu
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index cdc510c..c75a293 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -647,7 +647,7 @@ void KuduColumnSchema::CopyFrom(const KuduColumnSchema& other) {
 bool KuduColumnSchema::Equals(const KuduColumnSchema& other) const {
   return this == &other ||
     col_ == other.col_ ||
-    (col_ != nullptr && col_->Equals(*other.col_, true));
+    (col_ != nullptr && col_->Equals(*other.col_, ColumnSchema::COMPARE_ALL));
 }
 
 const std::string& KuduColumnSchema::name() const {
diff --git a/src/kudu/client/schema.h b/src/kudu/client/schema.h
index 217cf1f..5495ab7 100644
--- a/src/kudu/client/schema.h
+++ b/src/kudu/client/schema.h
@@ -27,6 +27,7 @@
 #include <vector>
 
 #ifdef KUDU_HEADERS_NO_STUBS
+#include <gtest/gtest_prod.h>
 #include "kudu/gutil/port.h"
 #else
 #include "kudu/client/stubs.h"
@@ -248,6 +249,10 @@ class KUDU_EXPORT KuduColumnSchema {
   // is transitive to nested classes. See https://s.apache.org/inner-class-friends
   friend class KuduTableAlterer;
 
+#ifdef KUDU_HEADERS_NO_STUBS
+  FRIEND_TEST(KuduColumnSchemaTest, TestEquals);
+#endif
+
   KuduColumnSchema();
 
 #if defined(__clang__) || \
diff --git a/src/kudu/common/column_predicate-test.cc b/src/kudu/common/column_predicate-test.cc
index e0b6b63..e40e1a4 100644
--- a/src/kudu/common/column_predicate-test.cc
+++ b/src/kudu/common/column_predicate-test.cc
@@ -31,6 +31,7 @@
 #include "kudu/common/types.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/bloom_filter.h"
+#include "kudu/util/hash.pb.h"
 #include "kudu/util/int128.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/random.h"
@@ -1448,4 +1449,51 @@ TEST_F(TestColumnPredicate, TestBloomFilterMerge) {
   TestMergeBloomFilterCombinations(ColumnSchema("c", STRING, true), &bfs, binary_keys);
 }
 
+// Test ColumnPredicate operator (in-)equality.
+TEST_F(TestColumnPredicate, TestEquals) {
+  ColumnSchema c1("c1", INT32, true);
+  ASSERT_EQ(ColumnPredicate::None(c1), ColumnPredicate::None(c1));
+
+  ColumnSchema c1a("c1", INT32, true);
+  ASSERT_EQ(ColumnPredicate::None(c1), ColumnPredicate::None(c1a));
+
+  ColumnSchema c2("c2", INT32, true);
+  ASSERT_NE(ColumnPredicate::None(c1), ColumnPredicate::None(c2));
+
+  ColumnSchema c1string("c1", STRING, true);
+  ASSERT_NE(ColumnPredicate::None(c1), ColumnPredicate::None(c1string));
+
+  const int kDefaultOf3 = 3;
+  ColumnSchema c1dflt("c1", INT32, /*is_nullable=*/false, /*read_default=*/&kDefaultOf3);
+  ASSERT_NE(ColumnPredicate::None(c1), ColumnPredicate::None(c1dflt));
+}
+
+using TestColumnPredicateDeathTest = TestColumnPredicate;
+
+// Ensure that ColumnPredicate::Merge(other) requires the 'other' predicate to
+// have the same column name and type as 'this'.
+TEST_F(TestColumnPredicateDeathTest, TestMergeRequiresNameAndType) {
+
+  ColumnSchema c1int32("c1", INT32, true);
+  ColumnSchema c2int32("c2", INT32, true);
+  vector<int32_t> values = { 0, 1, 2, 3 };
+
+  EXPECT_DEATH({
+    // This should crash because the columns have different names.
+    TestMerge(ColumnPredicate::Equality(c1int32, &values[0]),
+              ColumnPredicate::Equality(c2int32, &values[0]),
+              ColumnPredicate::None(c1int32), // unused
+              PredicateType::None);
+  }, "COMPARE_NAME_AND_TYPE");
+
+  ColumnSchema c1int16("c1", INT16, true);
+  EXPECT_DEATH({
+    // This should crash because the columns have different types.
+    TestMerge(ColumnPredicate::Equality(c1int32, &values[0]),
+              ColumnPredicate::Equality(c1int16, &values[0]),
+              ColumnPredicate::None(c1int32), // unused
+              PredicateType::None);
+  }, "COMPARE_NAME_AND_TYPE");
+}
+
 } // namespace kudu
diff --git a/src/kudu/common/column_predicate.cc b/src/kudu/common/column_predicate.cc
index b923ff4..5f9e7be 100644
--- a/src/kudu/common/column_predicate.cc
+++ b/src/kudu/common/column_predicate.cc
@@ -291,7 +291,7 @@ void ColumnPredicate::Simplify() {
 }
 
 void ColumnPredicate::Merge(const ColumnPredicate& other) {
-  CHECK(column_.Equals(other.column_, false));
+  CHECK(column_.Equals(other.column_, ColumnSchema::COMPARE_NAME_AND_TYPE));
   switch (predicate_type_) {
     case PredicateType::None: return;
     case PredicateType::Range: {
@@ -822,7 +822,7 @@ string ColumnPredicate::ToString() const {
 }
 
 bool ColumnPredicate::operator==(const ColumnPredicate& other) const {
-  if (!column_.Equals(other.column_, false)) { return false; }
+  if (!column_.Equals(other.column_, ColumnSchema::COMPARE_NAME_AND_TYPE)) { return false; }
   if (predicate_type_ != other.predicate_type_) {
     return false;
   }
diff --git a/src/kudu/common/column_predicate.h b/src/kudu/common/column_predicate.h
index ea8385b..2527fe1 100644
--- a/src/kudu/common/column_predicate.h
+++ b/src/kudu/common/column_predicate.h
@@ -17,10 +17,9 @@
 
 #pragma once
 
+#include <algorithm>
 #include <cstddef>
 #include <cstdint>
-
-#include <algorithm>
 #include <ostream>
 #include <string>
 #include <vector>
@@ -32,6 +31,7 @@
 #include "kudu/common/schema.h"
 #include "kudu/common/types.h"
 #include "kudu/util/bloom_filter.h"
+#include "kudu/util/hash.pb.h"
 #include "kudu/util/slice.h"
 
 namespace kudu {
@@ -233,6 +233,11 @@ class ColumnPredicate {
   // Predicates over different columns are not equal.
   bool operator==(const ColumnPredicate& other) const;
 
+  // Negation of operator==.
+  bool operator!=(const ColumnPredicate& other) const {
+    return !(*this == other);
+  }
+
   // Returns the raw lower bound value if this is a range predicate, or the
   // equality value if this is an equality predicate.
   const void* raw_lower() const {
diff --git a/src/kudu/common/schema-test.cc b/src/kudu/common/schema-test.cc
index a35e3be..69f70ca 100644
--- a/src/kudu/common/schema-test.cc
+++ b/src/kudu/common/schema-test.cc
@@ -303,8 +303,7 @@ TEST_F(TestSchema, TestSchemaEquals) {
   ASSERT_FALSE(schema2.KeyTypeEquals(schema3));
   ASSERT_FALSE(schema3.Equals(schema4));
   ASSERT_TRUE(schema4.Equals(schema4));
-  ASSERT_TRUE(schema3.KeyEquals(schema4,
-              ColumnSchema::COMPARE_NAME | ColumnSchema::COMPARE_TYPE));
+  ASSERT_TRUE(schema3.KeyEquals(schema4, ColumnSchema::COMPARE_NAME_AND_TYPE));
 }
 
 TEST_F(TestSchema, TestReset) {
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index fd6b40e..0eb2d67 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -300,16 +300,17 @@ class ColumnSchema {
   }
 
   // compare types in Equals function
-  enum {
+  enum CompareFlags {
     COMPARE_NAME = 1 << 0,
     COMPARE_TYPE = 1 << 1,
     COMPARE_DEFAULTS = 1 << 2,
 
+    COMPARE_NAME_AND_TYPE = COMPARE_NAME | COMPARE_TYPE,
     COMPARE_ALL = COMPARE_NAME | COMPARE_TYPE | COMPARE_DEFAULTS
   };
 
   bool Equals(const ColumnSchema &other,
-              int flags = COMPARE_ALL) const {
+              CompareFlags flags = COMPARE_ALL) const {
     if (this == &other) return true;
 
     if ((flags & COMPARE_NAME) && this->name_ != other.name_)
@@ -782,8 +783,7 @@ class Schema {
   // Return true if the key projection schemas have exactly the same set of
   // columns and respective types.
   bool KeyEquals(const Schema& other,
-                 int flags
-                    = ColumnSchema::COMPARE_NAME | ColumnSchema::COMPARE_TYPE) const {
+                 ColumnSchema::CompareFlags flags = ColumnSchema::COMPARE_NAME_AND_TYPE) const {
     if (this == &other) return true;
     if (this->num_key_columns_ != other.num_key_columns_) return false;
     for (size_t i = 0; i < this->num_key_columns_; i++) {