You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2018/05/22 22:17:23 UTC

calcite git commit: [CALCITE-2265] Allow comparison of ROW values (Dylan Adams)

Repository: calcite
Updated Branches:
  refs/heads/master 4e47dfea5 -> da568324a


[CALCITE-2265] Allow comparison of ROW values (Dylan Adams)

StandardConvertletTable.consistentType now checks if structs' members
are pair-wise of the same family.

Added isSameFamily and areSameFamily methods to SqlTypeUtil to determine
whether two types, or a list of types, are of the same family. Added
unit test to verify this new functionality.

Added new case to JdbcTest to verify row comparison are executed in
HSQLDB.

Refactored common code from SqlTypeUtil.isComparable for determining
the family of a RelDataType.

Extracted SqlTypeFixture from SqlTypeFactoryTest.

Close apache/calcite#696


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/da568324
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/da568324
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/da568324

Branch: refs/heads/master
Commit: da568324ad84c5a792f2b9c2bd2c827c1440b19c
Parents: 4e47dfe
Author: dkadams <dy...@gmail.com>
Authored: Tue May 22 10:44:06 2018 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue May 22 13:49:50 2018 -0700

----------------------------------------------------------------------
 .../apache/calcite/sql/type/SqlTypeUtil.java    |  56 +++++++++
 .../sql2rel/StandardConvertletTable.java        |   6 +-
 .../calcite/sql/type/SqlTypeFactoryTest.java    |  44 +------
 .../apache/calcite/sql/type/SqlTypeFixture.java |  65 +++++++++++
 .../calcite/sql/type/SqlTypeUtilTest.java       | 114 +++++++++++++++++++
 .../org/apache/calcite/test/CalciteSuite.java   |   2 +
 .../java/org/apache/calcite/test/JdbcTest.java  |   7 ++
 7 files changed, 250 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 19cdbf9..5b3ddab 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlCollation;
@@ -38,6 +39,7 @@ import org.apache.calcite.util.Util;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 import java.nio.charset.Charset;
 import java.util.AbstractList;
@@ -1309,6 +1311,58 @@ public abstract class SqlTypeUtil {
     return family;
   }
 
+  /**
+   * Returns whether all types in a collection have the same family, as
+   * determined by {@link #isSameFamily(RelDataType, RelDataType)}.
+   *
+   * @param types Types to check
+   * @return true if all types are of the same family
+   */
+  public static boolean areSameFamily(Iterable<RelDataType> types) {
+    final List<RelDataType> typeList = ImmutableList.copyOf(types);
+    if (Sets.newHashSet(RexUtil.families(typeList)).size() < 2) {
+      return true;
+    }
+    for (Pair<RelDataType, RelDataType> adjacent : Pair.adjacents(typeList)) {
+      if (!isSameFamily(adjacent.left, adjacent.right)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Returns whether two types are scalar types of the same family, or struct types whose fields
+   * are pairwise of the same family.
+   *
+   * @param type1 First type
+   * @param type2 Second type
+   * @return Whether types have the same family
+   */
+  private static boolean isSameFamily(RelDataType type1, RelDataType type2) {
+    if (type1.isStruct() != type2.isStruct()) {
+      return false;
+    }
+
+    if (type1.isStruct()) {
+      int n = type1.getFieldCount();
+      if (n != type2.getFieldCount()) {
+        return false;
+      }
+      for (Pair<RelDataTypeField, RelDataTypeField> pair
+          : Pair.zip(type1.getFieldList(), type2.getFieldList())) {
+        if (!isSameFamily(pair.left.getType(), pair.right.getType())) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    final RelDataTypeFamily family1 = family(type1);
+    final RelDataTypeFamily family2 = family(type2);
+    return family1 == family2;
+  }
+
   /** Returns whether a character data type can be implicitly converted to a
    * given family in a compare operation. */
   private static boolean canConvertStringInCompare(RelDataTypeFamily family) {
@@ -1371,6 +1425,8 @@ public abstract class SqlTypeUtil {
   public static boolean isArray(RelDataType type) {
     return type.getSqlTypeName() == SqlTypeName.ARRAY;
   }
+
+
 }
 
 // End SqlTypeUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 8c7ffc1..fb22286 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -73,13 +73,11 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 
 import java.math.BigDecimal;
 import java.math.RoundingMode;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 /**
  * Standard implementation of {@link SqlRexConvertletTable}.
@@ -877,9 +875,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       SqlOperandTypeChecker.Consistency consistency, List<RelDataType> types) {
     switch (consistency) {
     case COMPARE:
-      final Set<RelDataTypeFamily> families =
-          Sets.newHashSet(RexUtil.families(types));
-      if (families.size() < 2) {
+      if (SqlTypeUtil.areSameFamily(types)) {
         // All arguments are of same family. No need for explicit casts.
         return null;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
index 8b1a985..f3c5209 100644
--- a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.sql.type;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
 
 import com.google.common.collect.Lists;
 
@@ -34,21 +33,21 @@ import static org.junit.Assert.fail;
 public class SqlTypeFactoryTest {
 
   @Test public void testLeastRestrictiveWithAny() {
-    Fixture f = new Fixture();
+    SqlTypeFixture f = new SqlTypeFixture();
     RelDataType leastRestrictive =
         f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlBigInt, f.sqlAny));
     assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.ANY));
   }
 
   @Test public void testLeastRestrictiveWithNumbers() {
-    Fixture f = new Fixture();
+    SqlTypeFixture f = new SqlTypeFixture();
     RelDataType leastRestrictive =
         f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlBigInt, f.sqlInt));
     assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.BIGINT));
   }
 
   @Test public void testLeastRestrictiveWithNullability() {
-    Fixture f = new Fixture();
+    SqlTypeFixture f = new SqlTypeFixture();
     RelDataType leastRestrictive =
         f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlVarcharNullable, f.sqlAny));
     assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.ANY));
@@ -56,7 +55,7 @@ public class SqlTypeFactoryTest {
   }
 
   @Test public void testLeastRestrictiveWithNull() {
-    Fixture f = new Fixture();
+    SqlTypeFixture f = new SqlTypeFixture();
     RelDataType leastRestrictive =
         f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlNull, f.sqlNull));
     assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.NULL));
@@ -77,7 +76,7 @@ public class SqlTypeFactoryTest {
 
   /** Unit test for {@link ArraySqlType#getPrecedenceList()}. */
   @Test public void testArrayPrecedenceList() {
-    Fixture f = new Fixture();
+    SqlTypeFixture f = new SqlTypeFixture();
     assertThat(checkPrecendenceList(f.arrayBigInt, f.arrayBigInt, f.arrayFloat),
         is(3));
     assertThat(
@@ -114,39 +113,6 @@ public class SqlTypeFactoryTest {
     assertThat(SqlTypeUtil.comparePrecision(p1, p1), is(0));
   }
 
-  /** Sets up data needed by a test. */
-  private static class Fixture {
-    SqlTypeFactoryImpl typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-    final RelDataType sqlBigInt = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.BIGINT), false);
-    final RelDataType sqlBigIntNullable = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.BIGINT), true);
-    final RelDataType sqlInt = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.INTEGER), false);
-    final RelDataType sqlVarcharNullable = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.VARCHAR), true);
-    final RelDataType sqlNull = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.NULL), false);
-    final RelDataType sqlAny = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.ANY), false);
-    final RelDataType sqlFloat = typeFactory.createTypeWithNullability(
-        typeFactory.createSqlType(SqlTypeName.FLOAT), false);
-    final RelDataType arrayFloat = typeFactory.createTypeWithNullability(
-        typeFactory.createArrayType(sqlFloat, -1), false);
-    final RelDataType arrayBigInt = typeFactory.createTypeWithNullability(
-        typeFactory.createArrayType(sqlBigIntNullable, -1), false);
-    final RelDataType multisetFloat = typeFactory.createTypeWithNullability(
-        typeFactory.createMultisetType(sqlFloat, -1), false);
-    final RelDataType multisetBigInt = typeFactory.createTypeWithNullability(
-        typeFactory.createMultisetType(sqlBigIntNullable, -1), false);
-    final RelDataType arrayBigIntNullable = typeFactory.createTypeWithNullability(
-        typeFactory.createArrayType(sqlBigIntNullable, -1), true);
-    final RelDataType arrayOfArrayBigInt = typeFactory.createTypeWithNullability(
-        typeFactory.createArrayType(arrayBigInt, -1), false);
-    final RelDataType arrayOfArrayFloat = typeFactory.createTypeWithNullability(
-        typeFactory.createArrayType(arrayFloat, -1), false);
-  }
-
 }
 
 // End SqlTypeFactoryTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFixture.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFixture.java b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFixture.java
new file mode 100644
index 0000000..df70571
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFixture.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+
+/**
+ * Reusable {@link RelDataType} fixtures for tests.
+ */
+class SqlTypeFixture {
+  SqlTypeFactoryImpl typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+  final RelDataType sqlBoolean = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.BOOLEAN), false);
+  final RelDataType sqlBigInt = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.BIGINT), false);
+  final RelDataType sqlBigIntNullable = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.BIGINT), true);
+  final RelDataType sqlInt = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.INTEGER), false);
+  final RelDataType sqlDate = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.DATE), false);
+  final RelDataType sqlVarchar = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.VARCHAR), false);
+  final RelDataType sqlChar = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.CHAR), false);
+  final RelDataType sqlVarcharNullable = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.VARCHAR), true);
+  final RelDataType sqlNull = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.NULL), false);
+  final RelDataType sqlAny = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.ANY), false);
+  final RelDataType sqlFloat = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.FLOAT), false);
+  final RelDataType arrayFloat = typeFactory.createTypeWithNullability(
+      typeFactory.createArrayType(sqlFloat, -1), false);
+  final RelDataType arrayBigInt = typeFactory.createTypeWithNullability(
+      typeFactory.createArrayType(sqlBigIntNullable, -1), false);
+  final RelDataType multisetFloat = typeFactory.createTypeWithNullability(
+      typeFactory.createMultisetType(sqlFloat, -1), false);
+  final RelDataType multisetBigInt = typeFactory.createTypeWithNullability(
+      typeFactory.createMultisetType(sqlBigIntNullable, -1), false);
+  final RelDataType arrayBigIntNullable = typeFactory.createTypeWithNullability(
+      typeFactory.createArrayType(sqlBigIntNullable, -1), true);
+  final RelDataType arrayOfArrayBigInt = typeFactory.createTypeWithNullability(
+      typeFactory.createArrayType(arrayBigInt, -1), false);
+  final RelDataType arrayOfArrayFloat = typeFactory.createTypeWithNullability(
+      typeFactory.createArrayType(arrayFloat, -1), false);
+}
+
+// End SqlTypeFixture.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/test/java/org/apache/calcite/sql/type/SqlTypeUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeUtilTest.java b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeUtilTest.java
new file mode 100644
index 0000000..6707a2b
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeUtilTest.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+
+import static org.apache.calcite.sql.type.SqlTypeUtil.areSameFamily;
+
+import com.google.common.collect.ImmutableList;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test of {@link org.apache.calcite.sql.type.SqlTypeUtil}.
+ */
+public class SqlTypeUtilTest {
+
+  private final SqlTypeFixture f = new SqlTypeFixture();
+
+  @Test
+  public void testTypesIsSameFamilyWithNumberTypes() {
+    assertThat(areSameFamily(ImmutableList.of(f.sqlBigInt, f.sqlBigInt)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlInt, f.sqlBigInt)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlFloat, f.sqlBigInt)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlInt, f.sqlBigIntNullable)),
+        is(true));
+  }
+
+  @Test
+  public void testTypesIsSameFamilyWithCharTypes() {
+    assertThat(areSameFamily(ImmutableList.of(f.sqlVarchar, f.sqlVarchar)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlVarchar, f.sqlChar)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlVarchar, f.sqlVarcharNullable)),
+        is(true));
+  }
+
+  @Test
+  public void testTypesIsSameFamilyWithInconvertibleTypes() {
+    assertThat(areSameFamily(ImmutableList.of(f.sqlBoolean, f.sqlBigInt)), is(false));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlFloat, f.sqlBoolean)), is(false));
+    assertThat(areSameFamily(ImmutableList.of(f.sqlInt, f.sqlDate)), is(false));
+  }
+
+  @Test
+  public void testTypesIsSameFamilyWithNumberStructTypes() {
+    final RelDataType bigIntAndFloat = struct(f.sqlBigInt, f.sqlFloat);
+    final RelDataType floatAndBigInt = struct(f.sqlFloat, f.sqlBigInt);
+
+    assertThat(areSameFamily(ImmutableList.of(bigIntAndFloat, floatAndBigInt)),
+        is(true));
+    assertThat(areSameFamily(ImmutableList.of(bigIntAndFloat, bigIntAndFloat)),
+        is(true));
+    assertThat(areSameFamily(ImmutableList.of(bigIntAndFloat, bigIntAndFloat)),
+        is(true));
+    assertThat(areSameFamily(ImmutableList.of(floatAndBigInt, floatAndBigInt)),
+        is(true));
+  }
+
+  @Test
+  public void testTypesIsSameFamilyWithCharStructTypes() {
+    final RelDataType varCharStruct = struct(f.sqlVarchar);
+    final RelDataType charStruct = struct(f.sqlChar);
+
+    assertThat(areSameFamily(ImmutableList.of(varCharStruct, charStruct)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(charStruct, varCharStruct)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(varCharStruct, varCharStruct)), is(true));
+    assertThat(areSameFamily(ImmutableList.of(charStruct, charStruct)), is(true));
+  }
+
+  @Test
+  public void testTypesIsSameFamilyWithInconvertibleStructTypes() {
+    final RelDataType dateStruct = struct(f.sqlDate);
+    final RelDataType boolStruct = struct(f.sqlBoolean);
+    assertThat(areSameFamily(ImmutableList.of(dateStruct, boolStruct)), is(false));
+
+    final RelDataType charIntStruct = struct(f.sqlChar, f.sqlInt);
+    final RelDataType charDateStruct = struct(f.sqlChar, f.sqlDate);
+    assertThat(areSameFamily(ImmutableList.of(charIntStruct, charDateStruct)),
+        is(false));
+
+    final RelDataType boolDateStruct = struct(f.sqlBoolean, f.sqlDate);
+    final RelDataType floatIntStruct = struct(f.sqlInt, f.sqlFloat);
+    assertThat(areSameFamily(ImmutableList.of(boolDateStruct, floatIntStruct)),
+        is(false));
+  }
+
+  private RelDataType struct(RelDataType...relDataTypes) {
+    final RelDataTypeFactory.Builder builder = f.typeFactory.builder();
+    for (int i = 0; i < relDataTypes.length; i++) {
+      builder.add("field" + i, relDataTypes[i]);
+    }
+    return builder.build();
+  }
+}
+
+// End SqlTypeUtilTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index a1d87a2..d87efb0 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -46,6 +46,7 @@ import org.apache.calcite.sql.test.SqlOperatorTest;
 import org.apache.calcite.sql.test.SqlPrettyWriterTest;
 import org.apache.calcite.sql.test.SqlTypeNameTest;
 import org.apache.calcite.sql.type.SqlTypeFactoryTest;
+import org.apache.calcite.sql.type.SqlTypeUtilTest;
 import org.apache.calcite.sql.validate.LexCaseSensitiveTest;
 import org.apache.calcite.sql.validate.SqlValidatorUtilTest;
 import org.apache.calcite.test.enumerable.EnumerableCorrelateTest;
@@ -113,6 +114,7 @@ import org.junit.runners.Suite;
     RelOptPlanReaderTest.class,
     RexBuilderTest.class,
     SqlTypeFactoryTest.class,
+    SqlTypeUtilTest.class,
     SqlValidatorUtilTest.class,
 
     // medium tests (above 0.1s)

http://git-wip-us.apache.org/repos/asf/calcite/blob/da568324/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 7a3f259..f013f3e 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -5890,6 +5890,13 @@ public class JdbcTest {
             });
   }
 
+  @Test public void testRowComparison() {
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_SCOTT)
+        .query("SELECT empno FROM JDBC_SCOTT.emp WHERE (ename, job) < ('Blake', 'Manager')")
+        .returnsUnordered("EMPNO=7876", "EMPNO=7499", "EMPNO=7698");
+  }
+
   @Test public void testUnicode() throws Exception {
     CalciteAssert.AssertThat with =
         CalciteAssert.that().with(CalciteAssert.Config.FOODMART_CLONE);