You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by li...@apache.org on 2022/03/04 13:24:26 UTC

[calcite] 22/41: [CALCITE-4872] Add UNKNOWN value to enum SqlTypeName, distinct from the NULL type

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

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

commit 67ba00764f822a8b0773f8964c1d846c9fad667c
Author: Will Noble <wn...@google.com>
AuthorDate: Fri Oct 29 11:24:38 2021 -0700

    [CALCITE-4872] Add UNKNOWN value to enum SqlTypeName, distinct from the NULL type
    
    Before this change, the UNKNOWN type would become the NULL type
    when switching nullability.
    
    Close apache/calcite#2595
---
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |  7 +++-
 .../org/apache/calcite/sql/type/BasicSqlType.java  | 29 ++++++++++-------
 .../calcite/sql/type/SqlTypeFactoryImpl.java       | 15 +++++++--
 .../calcite/sql/type/SqlTypeMappingRule.java       |  4 +--
 .../org/apache/calcite/sql/type/SqlTypeName.java   |  1 +
 .../apache/calcite/sql/type/SqlTypeTransforms.java |  2 ++
 .../org/apache/calcite/sql/type/SqlTypeUtil.java   |  5 ++-
 .../calcite/sql/type/SqlTypeFactoryTest.java       | 16 ++++++++++
 .../apache/calcite/sql/type/SqlTypeFixture.java    |  2 ++
 .../apache/calcite/sql/type/SqlTypeUtilTest.java   | 37 ++++++++++++++++++++++
 site/_docs/reference.md                            |  3 +-
 11 files changed, 102 insertions(+), 19 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 48575ba..9155ab7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -801,7 +801,12 @@ public abstract class SqlImplementor {
           SqlNode fieldOperand = field(ordinal);
           return SqlStdOperatorTable.CURSOR.createCall(SqlParserPos.ZERO, fieldOperand);
         }
-        if (ignoreCast) {
+        // Ideally the UNKNOWN type would never exist in a fully-formed, validated rel node, but
+        // it can be useful in certain situations where determining the type of an expression is
+        // infeasible, such as inserting arbitrary user-provided SQL snippets into an otherwise
+        // manually-constructed (as opposed to parsed) rel node.
+        // In such a context, assume that casting anything to UNKNOWN is a no-op.
+        if (ignoreCast || call.getType().getSqlTypeName() == SqlTypeName.UNKNOWN) {
           assert nodeList.size() == 1;
           return nodeList.get(0);
         } else {
diff --git a/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
index d7a9dd6..9428ad4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
@@ -40,7 +40,7 @@ public class BasicSqlType extends AbstractSqlType {
 
   private final int precision;
   private final int scale;
-  private final RelDataTypeSystem typeSystem;
+  protected final RelDataTypeSystem typeSystem;
   private final @Nullable SqlCollation collation;
   private final @Nullable SerializableCharset wrappedCharset;
 
@@ -54,19 +54,14 @@ public class BasicSqlType extends AbstractSqlType {
    * @param typeName Type name
    */
   public BasicSqlType(RelDataTypeSystem typeSystem, SqlTypeName typeName) {
-    this(typeSystem, typeName, false, PRECISION_NOT_SPECIFIED,
-        SCALE_NOT_SPECIFIED, null, null);
-    checkPrecScale(typeName, false, false);
+    this(typeSystem, typeName, false);
   }
 
-  /** Throws if {@code typeName} does not allow the given combination of
-   * precision and scale. */
-  protected static void checkPrecScale(SqlTypeName typeName,
-      boolean precisionSpecified, boolean scaleSpecified) {
-    if (!typeName.allowsPrecScale(precisionSpecified, scaleSpecified)) {
-      throw new AssertionError("typeName.allowsPrecScale("
-          + precisionSpecified + ", " + scaleSpecified + "): " + typeName);
-    }
+  protected BasicSqlType(RelDataTypeSystem typeSystem, SqlTypeName typeName,
+      boolean nullable) {
+    this(typeSystem, typeName, nullable, PRECISION_NOT_SPECIFIED,
+        SCALE_NOT_SPECIFIED, null, null);
+    checkPrecScale(typeName, false, false);
   }
 
   /**
@@ -115,6 +110,16 @@ public class BasicSqlType extends AbstractSqlType {
     computeDigest();
   }
 
+  /** Throws if {@code typeName} does not allow the given combination of
+   * precision and scale. */
+  protected static void checkPrecScale(SqlTypeName typeName,
+      boolean precisionSpecified, boolean scaleSpecified) {
+    if (!typeName.allowsPrecScale(precisionSpecified, scaleSpecified)) {
+      throw new AssertionError("typeName.allowsPrecScale("
+          + precisionSpecified + ", " + scaleSpecified + "): " + typeName);
+    }
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
index 6fd58e6..532ccfe 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
@@ -570,9 +570,20 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
 
   /** The unknown type. Similar to the NULL type, but is only equal to
    * itself. */
-  private static class UnknownSqlType extends BasicSqlType {
+  static class UnknownSqlType extends BasicSqlType {
     UnknownSqlType(RelDataTypeFactory typeFactory) {
-      super(typeFactory.getTypeSystem(), SqlTypeName.NULL);
+      this(typeFactory.getTypeSystem(), false);
+    }
+
+    private UnknownSqlType(RelDataTypeSystem typeSystem, boolean nullable) {
+      super(typeSystem, SqlTypeName.UNKNOWN, nullable);
+    }
+
+    @Override BasicSqlType createWithNullability(boolean nullable) {
+      if (nullable == this.isNullable) {
+        return this;
+      }
+      return new UnknownSqlType(this.typeSystem, nullable);
     }
 
     @Override protected void generateTypeString(StringBuilder sb,
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeMappingRule.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeMappingRule.java
index a9b7afb..a9e36a7 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeMappingRule.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeMappingRule.java
@@ -44,9 +44,9 @@ public interface SqlTypeMappingRule {
     Objects.requireNonNull(to, "to");
     Objects.requireNonNull(from, "from");
 
-    if (to == SqlTypeName.NULL) {
+    if (to == SqlTypeName.NULL || to == SqlTypeName.UNKNOWN) {
       return false;
-    } else if (from == SqlTypeName.NULL) {
+    } else if (from == SqlTypeName.NULL || from == SqlTypeName.UNKNOWN) {
       return true;
     }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index 840cd74..47c57ad 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -106,6 +106,7 @@ public enum SqlTypeName {
   VARBINARY(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.VARBINARY,
       SqlTypeFamily.BINARY),
   NULL(PrecScale.NO_NO, true, Types.NULL, SqlTypeFamily.NULL),
+  UNKNOWN(PrecScale.NO_NO, true, Types.NULL, SqlTypeFamily.NULL),
   ANY(PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES, true,
       Types.JAVA_OBJECT, SqlTypeFamily.ANY),
   SYMBOL(PrecScale.NO_NO, true, Types.OTHER, null),
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
index 92165f5..cc4e637 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
@@ -145,6 +145,8 @@ public abstract class SqlTypeTransforms {
             return SqlTypeName.ANY;
           case NULL:
             return SqlTypeName.NULL;
+          case UNKNOWN:
+            return SqlTypeName.UNKNOWN;
           default:
             throw Util.unexpected(sqlTypeName);
           }
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 a226f77..0fcff9c 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
@@ -824,6 +824,9 @@ public abstract class SqlTypeUtil {
 
     final SqlTypeName fromTypeName = fromType.getSqlTypeName();
     final SqlTypeName toTypeName = toType.getSqlTypeName();
+    if (toTypeName == SqlTypeName.UNKNOWN) {
+      return true;
+    }
     if (toType.isStruct() || fromType.isStruct()) {
       if (toTypeName == SqlTypeName.DISTINCT) {
         if (fromTypeName == SqlTypeName.DISTINCT) {
@@ -1033,7 +1036,7 @@ public abstract class SqlTypeUtil {
     assert typeName != null;
 
     final SqlTypeNameSpec typeNameSpec;
-    if (isAtomic(type) || isNull(type)) {
+    if (isAtomic(type) || isNull(type) || type.getSqlTypeName() == SqlTypeName.UNKNOWN) {
       int precision = typeName.allowsPrec() ? type.getPrecision() : -1;
       // fix up the precision.
       if (maxPrecision > 0 && precision > maxPrecision) {
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 80a8a86..fa07a31 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
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl.UnknownSqlType;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -33,6 +34,7 @@ import java.util.Map;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.Is.isA;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -286,4 +288,18 @@ class SqlTypeFactoryTest {
     assertThat(tsWithPrecision3 == tsWithPrecision8, is(true));
   }
 
+  /** Test that the {code UNKNOWN} type does not does not change class when nullified. */
+  @Test void testUnknownCreateWithNullabilityTypeConsistency() {
+    SqlTypeFixture f = new SqlTypeFixture();
+
+    RelDataType unknownType  = f.typeFactory.createUnknownType();
+    assertThat(unknownType, isA(UnknownSqlType.class));
+    assertThat(unknownType.getSqlTypeName(), is(SqlTypeName.UNKNOWN));
+    assertFalse(unknownType.isNullable());
+
+    RelDataType nullableRelDataType = f.typeFactory.createTypeWithNullability(unknownType, true);
+    assertThat(nullableRelDataType, isA(UnknownSqlType.class));
+    assertThat(nullableRelDataType.getSqlTypeName(), is(SqlTypeName.UNKNOWN));
+    assertTrue(nullableRelDataType.isNullable());
+  }
 }
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
index 6827024..05fa8fe 100644
--- a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFixture.java
+++ b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFixture.java
@@ -44,6 +44,8 @@ class SqlTypeFixture {
       typeFactory.createSqlType(SqlTypeName.VARCHAR), true);
   final RelDataType sqlNull = typeFactory.createTypeWithNullability(
       typeFactory.createSqlType(SqlTypeName.NULL), false);
+  final RelDataType sqlUnknown = typeFactory.createTypeWithNullability(
+      typeFactory.createSqlType(SqlTypeName.UNKNOWN), false);
   final RelDataType sqlAny = typeFactory.createTypeWithNullability(
       typeFactory.createSqlType(SqlTypeName.ANY), false);
   final RelDataType sqlFloat = typeFactory.createTypeWithNullability(
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
index 0c065a5..b6bddfa 100644
--- a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeUtilTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeUtilTest.java
@@ -29,6 +29,7 @@ import org.junit.jupiter.api.Test;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Locale;
 import java.util.stream.Collectors;
 
 import static org.apache.calcite.sql.type.SqlTypeUtil.areSameFamily;
@@ -155,6 +156,10 @@ class SqlTypeUtilTest {
         (SqlBasicTypeNameSpec) convertTypeToSpec(f.sqlNull).getTypeNameSpec();
     assertThat(nullSpec.getTypeName().getSimple(), is("NULL"));
 
+    SqlBasicTypeNameSpec unknownSpec =
+        (SqlBasicTypeNameSpec) convertTypeToSpec(f.sqlUnknown).getTypeNameSpec();
+    assertThat(unknownSpec.getTypeName().getSimple(), is("UNKNOWN"));
+
     SqlBasicTypeNameSpec basicSpec =
         (SqlBasicTypeNameSpec) convertTypeToSpec(f.sqlBigInt).getTypeNameSpec();
     assertThat(basicSpec.getTypeName().getSimple(), is("BIGINT"));
@@ -226,4 +231,36 @@ class SqlTypeUtilTest {
     compareTypesIgnoringNullability("identical types should return true.",
         bigIntType, bigIntType1, true);
   }
+
+  @Test void testCanAlwaysCastToUnknownFromBasic() {
+    RelDataType unknownType = f.typeFactory.createUnknownType();
+    RelDataType nullableUnknownType = f.typeFactory.createTypeWithNullability(unknownType, true);
+
+    for (SqlTypeName fromTypeName : SqlTypeName.values()) {
+      BasicSqlType fromType;
+      try {
+        // This only works for basic types. Ignore the rest.
+        fromType = (BasicSqlType) f.typeFactory.createSqlType(fromTypeName);
+      } catch (AssertionError e) {
+        continue;
+      }
+      BasicSqlType nullableFromType = fromType.createWithNullability(!fromType.isNullable);
+
+      assertCanCast(unknownType, fromType);
+      assertCanCast(unknownType, nullableFromType);
+      assertCanCast(nullableUnknownType, fromType);
+      assertCanCast(nullableUnknownType, nullableFromType);
+    }
+  }
+
+  private static void assertCanCast(RelDataType toType, RelDataType fromType) {
+    assertThat(
+        String.format(Locale.ROOT,
+            "Expected to be able to cast from %s to %s without coercion.", fromType, toType),
+        SqlTypeUtil.canCastFrom(toType, fromType, /* coerce= */ false), is(true));
+    assertThat(
+        String.format(Locale.ROOT,
+            "Expected to be able to cast from %s to %s with coercion.", fromType, toType),
+        SqlTypeUtil.canCastFrom(toType, fromType, /* coerce= */ true), is(true));
+  }
 }
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 6ac180b..4f1de54 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1150,7 +1150,8 @@ Note:
 
 | Type     | Description                | Example literals
 |:-------- |:---------------------------|:---------------
-| ANY      | A value of an unknown type |
+| ANY      | The union of all types |
+| UNKNOWN  | A value of an unknown type; used as a placeholder |
 | ROW      | Row with 1 or more columns | Example: Row(f0 int null, f1 varchar)
 | MAP      | Collection of keys mapped to values |
 | MULTISET | Unordered collection that may contain duplicates | Example: int multiset