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/06/08 17:42:11 UTC

[2/2] calcite git commit: [CALCITE-2321] A union of CHAR columns of different lengths can now (based on a conformance setting) yield a VARCHAR column (Hequn Cheng)

[CALCITE-2321] A union of CHAR columns of different lengths can now (based on a conformance setting) yield a VARCHAR column (Hequn Cheng)

The previous behavior was to return a CHAR column whose length is the
longest of the inputs. This remains the default behavior, and the
behavior in strict SQL standard mode. The new SqlConformance method is
shouldConvertRaggedUnionTypesToVarying(). Also added
RelDataTypeSystem.shouldConvertRaggedUnionTypesToVarying().

Close apache/calcite#699


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

Branch: refs/heads/master
Commit: 18bfc278839bfbc7c3bde15116e6edab790707a6
Parents: aaf4471
Author: hequn8128 <ch...@gmail.com>
Authored: Fri May 25 17:18:29 2018 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jun 8 10:36:05 2018 -0700

----------------------------------------------------------------------
 .../calcite/jdbc/CalciteConnectionImpl.java     |  13 ++-
 .../java/org/apache/calcite/jdbc/Driver.java    |   8 ++
 .../calcite/rel/type/DelegatingTypeSystem.java  |  99 ++++++++++++++++
 .../rel/type/RelDataTypeFactoryImpl.java        |   1 -
 .../calcite/rel/type/RelDataTypeSystem.java     |   4 +
 .../calcite/rel/type/RelDataTypeSystemImpl.java |   4 +
 .../apache/calcite/sql/fun/SqlCaseOperator.java |   4 +-
 .../calcite/sql/type/SqlTypeFactoryImpl.java    |  18 +--
 .../sql/validate/SqlAbstractConformance.java    |   4 +
 .../calcite/sql/validate/SqlConformance.java    |  28 +++++
 .../sql/validate/SqlConformanceEnum.java        |  14 +++
 .../calcite/sql/test/DefaultSqlTestFactory.java |  25 +++-
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 116 +++++++++++++++++++
 13 files changed, 315 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
index ff6fb9f..4e19554 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -45,6 +45,7 @@ import org.apache.calcite.materialize.Lattice;
 import org.apache.calcite.materialize.MaterializationService;
 import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.DelegatingTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
@@ -119,8 +120,18 @@ abstract class CalciteConnectionImpl
     if (typeFactory != null) {
       this.typeFactory = typeFactory;
     } else {
-      final RelDataTypeSystem typeSystem =
+      RelDataTypeSystem typeSystem =
           cfg.typeSystem(RelDataTypeSystem.class, RelDataTypeSystem.DEFAULT);
+      if (cfg.conformance().shouldConvertRaggedUnionTypesToVarying()) {
+        typeSystem =
+            new DelegatingTypeSystem(typeSystem) {
+              @Override public boolean
+              shouldConvertRaggedUnionTypesToVarying() {
+                return true;
+              }
+            };
+        cfg.typeSystem(RelDataTypeSystem.class, RelDataTypeSystem.DEFAULT);
+      }
       this.typeFactory = new JavaTypeFactoryImpl(typeSystem);
     }
     this.rootSchema =

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/Driver.java b/core/src/main/java/org/apache/calcite/jdbc/Driver.java
index f8b7ca0..09e4d93 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/Driver.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/Driver.java
@@ -176,6 +176,14 @@ public class Driver extends UnregisteredDriver {
         .newConnection(this, factory, CONNECT_STRING_PREFIX, new Properties(),
             rootSchema, typeFactory);
   }
+
+  /** Creates an internal connection. */
+  CalciteConnection connect(CalciteSchema rootSchema,
+      JavaTypeFactory typeFactory, Properties properties) {
+    return (CalciteConnection) ((CalciteFactory) factory)
+        .newConnection(this, factory, CONNECT_STRING_PREFIX, properties,
+            rootSchema, typeFactory);
+  }
 }
 
 // End Driver.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java
new file mode 100644
index 0000000..a91a2db
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java
@@ -0,0 +1,99 @@
+/*
+ * 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.rel.type;
+
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/** Implementation of {@link org.apache.calcite.rel.type.RelDataTypeSystem}
+ * that sends all methods to an underlying object. */
+public class DelegatingTypeSystem implements RelDataTypeSystem {
+  private final RelDataTypeSystem typeSystem;
+
+  /** Creates a {@code DelegatingTypeSystem}. */
+  protected DelegatingTypeSystem(RelDataTypeSystem typeSystem) {
+    this.typeSystem = typeSystem;
+  }
+
+  public int getMaxScale(SqlTypeName typeName) {
+    return typeSystem.getMaxScale(typeName);
+  }
+
+  public int getDefaultPrecision(SqlTypeName typeName) {
+    return typeSystem.getDefaultPrecision(typeName);
+  }
+
+  public int getMaxPrecision(SqlTypeName typeName) {
+    return typeSystem.getMaxPrecision(typeName);
+  }
+
+  public int getMaxNumericScale() {
+    return typeSystem.getMaxNumericScale();
+  }
+
+  public int getMaxNumericPrecision() {
+    return typeSystem.getMaxNumericPrecision();
+  }
+
+  public String getLiteral(SqlTypeName typeName, boolean isPrefix) {
+    return typeSystem.getLiteral(typeName, isPrefix);
+  }
+
+  public boolean isCaseSensitive(SqlTypeName typeName) {
+    return typeSystem.isCaseSensitive(typeName);
+  }
+
+  public boolean isAutoincrement(SqlTypeName typeName) {
+    return typeSystem.isAutoincrement(typeName);
+  }
+
+  public int getNumTypeRadix(SqlTypeName typeName) {
+    return typeSystem.getNumTypeRadix(typeName);
+  }
+
+  public RelDataType deriveSumType(RelDataTypeFactory typeFactory,
+      RelDataType argumentType) {
+    return typeSystem.deriveSumType(typeFactory, argumentType);
+  }
+
+  public RelDataType deriveAvgAggType(RelDataTypeFactory typeFactory,
+      RelDataType argumentType) {
+    return typeSystem.deriveAvgAggType(typeFactory, argumentType);
+  }
+
+  public RelDataType deriveCovarType(RelDataTypeFactory typeFactory,
+      RelDataType arg0Type, RelDataType arg1Type) {
+    return typeSystem.deriveCovarType(typeFactory, arg0Type, arg1Type);
+  }
+
+  public RelDataType deriveFractionalRankType(RelDataTypeFactory typeFactory) {
+    return typeSystem.deriveFractionalRankType(typeFactory);
+  }
+
+  public RelDataType deriveRankType(RelDataTypeFactory typeFactory) {
+    return typeSystem.deriveRankType(typeFactory);
+  }
+
+  public boolean isSchemaCaseSensitive() {
+    return typeSystem.isSchemaCaseSensitive();
+  }
+
+  public boolean shouldConvertRaggedUnionTypesToVarying() {
+    return typeSystem.shouldConvertRaggedUnionTypesToVarying();
+  }
+}
+
+// End DelegatingTypeSystem.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
index 4c1f474..4151c1a 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
@@ -194,7 +194,6 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
         });
   }
 
-  // implement RelDataTypeFactory
   public RelDataType leastRestrictive(List<RelDataType> types) {
     assert types != null;
     assert types.size() >= 1;

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
index b8a8088..6103a24 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
@@ -96,6 +96,10 @@ public interface RelDataTypeSystem {
   /** Whether two record types are considered distinct if their field names
    * are the same but in different cases. */
   boolean isSchemaCaseSensitive();
+
+  /** Whether the least restrictive type of a number of CHAR types of different
+   * lengths should be a VARCHAR type. And similarly BINARY to VARBINARY. */
+  boolean shouldConvertRaggedUnionTypesToVarying();
 }
 
 // End RelDataTypeSystem.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
index b7b8839..2e9399d 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
@@ -244,6 +244,10 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     return true;
   }
 
+  public boolean shouldConvertRaggedUnionTypesToVarying() {
+    return false;
+  }
+
 }
 
 // End RelDataTypeSystemImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
index 1e5e738..ec92b48 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
@@ -245,9 +245,7 @@ public class SqlCaseOperator extends SqlOperator {
       nullList.add(elseOp);
     }
 
-    RelDataType ret =
-        callBinding.getTypeFactory().leastRestrictive(
-            argTypes);
+    RelDataType ret = callBinding.getTypeFactory().leastRestrictive(argTypes);
     if (null == ret) {
       throw callBinding.newValidationError(RESOURCE.illegalMixingOfTypes());
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
----------------------------------------------------------------------
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 e512333..91556ca 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
@@ -23,7 +23,6 @@ import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlIntervalQualifier;
-import org.apache.calcite.util.Glossary;
 import org.apache.calcite.util.Util;
 
 import java.nio.charset.Charset;
@@ -315,7 +314,7 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
 
           SqlTypeName newTypeName = type.getSqlTypeName();
 
-          if (shouldRaggedFixedLengthValueUnionBeVariable()) {
+          if (typeSystem.shouldConvertRaggedUnionTypesToVarying()) {
             if (resultType.getPrecision() != type.getPrecision()) {
               if (newTypeName == SqlTypeName.CHAR) {
                 newTypeName = SqlTypeName.VARCHAR;
@@ -489,21 +488,6 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
     return resultType;
   }
 
-  /**
-   * Controls behavior discussed <a
-   * href="http://sf.net/mailarchive/message.php?msg_id=13337379">here</a>.
-   *
-   * @return false (the default) to provide strict SQL:2003 behavior; true to
-   * provide pragmatic behavior
-   *
-   * @see Glossary#SQL2003 SQL:2003 Part 2 Section 9.3 Syntax Rule 3.a.iii.3
-   */
-  protected boolean shouldRaggedFixedLengthValueUnionBeVariable() {
-    // TODO jvs 30-Nov-2006:  implement SQL-Flagger support
-    // for warning about non-standard usage
-    return false;
-  }
-
   private RelDataType createDoublePrecisionType() {
     return createSqlType(SqlTypeName.DOUBLE);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
index 85465b8..c4d0f2d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
@@ -90,6 +90,10 @@ public abstract class SqlAbstractConformance implements SqlConformance {
   public boolean allowGeometry() {
     return SqlConformanceEnum.DEFAULT.allowGeometry();
   }
+
+  public boolean shouldConvertRaggedUnionTypesToVarying() {
+    return SqlConformanceEnum.DEFAULT.shouldConvertRaggedUnionTypesToVarying();
+  }
 }
 
 // End SqlAbstractConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index 6ba9a6f..84eee35 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -324,6 +324,34 @@ public interface SqlConformance {
    * false otherwise.
    */
   boolean allowGeometry();
+
+  /**
+   * Whether the least restrictive type of a number of CHAR types of different
+   * lengths should be a VARCHAR type. And similarly BINARY to VARBINARY.
+   *
+   * <p>For example, consider the query
+   *
+   * <blockquote><pre>SELECT 'abcde' UNION SELECT 'xyz'</pre></blockquote>
+   *
+   * <p>The input columns have types {@code CHAR(5)} and {@code CHAR(3)}, and
+   * we need a result type that is large enough for both:
+   * <ul>
+   * <li>Under strict SQL:2003 behavior, its column has type {@code CHAR(5)},
+   *     and the value in the second row will have trailing spaces.
+   * <li>With lenient behavior, its column has type {@code VARCHAR(5)}, and the
+   *     values have no trailing spaces.
+   * </ul>
+   *
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#PRAGMATIC_99},
+   * {@link SqlConformanceEnum#PRAGMATIC_2003},
+   * {@link SqlConformanceEnum#MYSQL_5};
+   * {@link SqlConformanceEnum#ORACLE_10};
+   * {@link SqlConformanceEnum#ORACLE_12};
+   * {@link SqlConformanceEnum#SQL_SERVER_2008};
+   * false otherwise.
+   */
+  boolean shouldConvertRaggedUnionTypesToVarying();
 }
 
 // End SqlConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
index e24450a..c52d39c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
@@ -249,6 +249,20 @@ public enum SqlConformanceEnum implements SqlConformance {
       return false;
     }
   }
+
+  public boolean shouldConvertRaggedUnionTypesToVarying() {
+    switch (this) {
+    case PRAGMATIC_99:
+    case PRAGMATIC_2003:
+    case MYSQL_5:
+    case ORACLE_10:
+    case ORACLE_12:
+    case SQL_SERVER_2008:
+      return true;
+    default:
+      return false;
+    }
+  }
 }
 
 // End SqlConformanceEnum.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
index 075cac7..ffdf886 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.DelegatingTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.advise.SqlAdvisor;
@@ -79,8 +80,18 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
                     throws Exception {
                   final SqlOperatorTable operatorTable =
                       factory.createOperatorTable(factory);
+                  RelDataTypeSystem typeSystem = RelDataTypeSystem.DEFAULT;
+                  final SqlConformance conformance =
+                      (SqlConformance) factory.get("conformance");
+                  if (conformance.shouldConvertRaggedUnionTypesToVarying()) {
+                    typeSystem = new DelegatingTypeSystem(typeSystem) {
+                      public boolean shouldConvertRaggedUnionTypesToVarying() {
+                        return true;
+                      }
+                    };
+                  }
                   final JavaTypeFactory typeFactory =
-                      new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+                      new JavaTypeFactoryImpl(typeSystem);
                   final MockCatalogReader catalogReader =
                       factory.createCatalogReader(factory, typeFactory);
                   return new Xyz(operatorTable, typeFactory, catalogReader);
@@ -145,6 +156,18 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
       this.typeFactory = typeFactory;
       this.catalogReader = catalogReader;
     }
+
+    public SqlOperatorTable getOperatorTable() {
+      return operatorTable;
+    }
+
+    public JavaTypeFactory getTypeFactory() {
+      return typeFactory;
+    }
+
+    public MockCatalogReader getCatalogReader() {
+      return catalogReader;
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/18bfc278/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index 19c722d..0a1926d 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -44,6 +44,7 @@ import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.util.SqlString;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
@@ -313,6 +314,23 @@ public abstract class SqlOperatorBaseTest {
                 .with("fun", "oracle"));
   }
 
+  protected SqlTester oracleTester(SqlConformance conformance) {
+    if (conformance == null) {
+      conformance = SqlConformanceEnum.DEFAULT;
+    }
+    return tester
+        .withConformance(conformance)
+        .withOperatorTable(
+            ChainedSqlOperatorTable.of(OracleSqlOperatorTable.instance(),
+                SqlStdOperatorTable.instance()))
+        .withConnectionFactory(
+            CalciteAssert.EMPTY_CONNECTION_FACTORY
+                .with(new CalciteAssert
+                    .AddSchemaSpecPostProcessor(CalciteAssert.SchemaSpec.HR))
+                .with("fun", "oracle")
+                .with("conformance", conformance));
+  }
+
   //--- Tests -----------------------------------------------------------
 
   /**
@@ -1578,6 +1596,77 @@ public abstract class SqlOperatorBaseTest {
         "none of the above",
         "CHAR(17) NOT NULL");
 
+    // tests with SqlConformance
+    final SqlTester tester2 =
+        tester.withConformance(SqlConformanceEnum.PRAGMATIC_2003);
+    tester2.checkString(
+        "case 2 when 1 then 'a' when 2 then 'bcd' end",
+        "bcd",
+        "VARCHAR(3)");
+    tester2.checkString(
+        "case 1 when 1 then 'a' when 2 then 'bcd' end",
+        "a",
+        "VARCHAR(3)");
+    tester2.checkString(
+        "case 1 when 1 then cast('a' as varchar(1)) "
+            + "when 2 then cast('bcd' as varchar(3)) end",
+        "a",
+        "VARCHAR(3)");
+
+    tester2.checkString(
+        "case cast(null as int) when cast(null as int)"
+            + " then 'nulls match'"
+            + " else 'nulls do not match' end",
+        "nulls do not match",
+        "VARCHAR(18) NOT NULL");
+    tester2.checkScalarExact(
+        "case when 'a'=cast(null as varchar(1)) then 1 else 2 end",
+        "2");
+
+    // equivalent to "nullif('a',cast(null as varchar(1)))"
+    tester2.checkString(
+        "case when 'a' = cast(null as varchar(1)) then null else 'a' end",
+        "a",
+        "CHAR(1)");
+
+    // multiple values in some cases (introduced in SQL:2011)
+    tester2.checkString(
+        "case 1 "
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
+        "1 or 2",
+        "VARCHAR(17) NOT NULL");
+    tester2.checkString(
+        "case 2 "
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
+        "1 or 2",
+        "VARCHAR(17) NOT NULL");
+    tester2.checkString(
+        "case 3 "
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
+        "3",
+        "VARCHAR(17) NOT NULL");
+    tester2.checkString(
+        "case 4 "
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
+        "none of the above",
+        "VARCHAR(17) NOT NULL");
+
     // TODO: Check case with multisets
   }
 
@@ -5018,6 +5107,11 @@ public abstract class SqlOperatorBaseTest {
         "CHAR(5) NOT NULL");
     tester1.checkScalar("greatest(12, CAST(NULL AS INTEGER), 3)", null, "INTEGER");
     tester1.checkScalar("greatest(false, true)", true, "BOOLEAN NOT NULL");
+
+    final SqlTester tester2 = oracleTester(SqlConformanceEnum.ORACLE_12);
+    tester2.checkString("greatest('on', 'earth')", "on", "VARCHAR(5) NOT NULL");
+    tester2.checkString("greatest('show', 'on', 'earth')", "show",
+        "VARCHAR(5) NOT NULL");
   }
 
   @Test public void testLeastFunc() {
@@ -5028,6 +5122,11 @@ public abstract class SqlOperatorBaseTest {
         "CHAR(5) NOT NULL");
     tester1.checkScalar("least(12, CAST(NULL AS INTEGER), 3)", null, "INTEGER");
     tester1.checkScalar("least(false, true)", false, "BOOLEAN NOT NULL");
+
+    final SqlTester tester2 = oracleTester(SqlConformanceEnum.ORACLE_12);
+    tester2.checkString("least('on', 'earth')", "earth", "VARCHAR(5) NOT NULL");
+    tester2.checkString("least('show', 'on', 'earth')", "earth",
+        "VARCHAR(5) NOT NULL");
   }
 
   @Test public void testNvlFunc() {
@@ -5046,6 +5145,16 @@ public abstract class SqlOperatorBaseTest {
         "VARCHAR(20) NOT NULL");
     tester1.checkNull(
         "nvl(CAST(NULL AS VARCHAR(6)), cast(NULL AS VARCHAR(4)))");
+
+    final SqlTester tester2 = oracleTester(SqlConformanceEnum.ORACLE_12);
+    tester2.checkString("nvl('abc', 'de')", "abc", "VARCHAR(3) NOT NULL");
+    tester2.checkString("nvl('abc', 'defg')", "abc", "VARCHAR(4) NOT NULL");
+    tester2.checkString("nvl('abc', CAST(NULL AS VARCHAR(20)))", "abc",
+        "VARCHAR(20) NOT NULL");
+    tester2.checkString("nvl(CAST(NULL AS VARCHAR(20)), 'abc')", "abc",
+        "VARCHAR(20) NOT NULL");
+    tester2.checkNull(
+        "nvl(CAST(NULL AS VARCHAR(6)), cast(NULL AS VARCHAR(4)))");
   }
 
   @Test public void testDecodeFunc() {
@@ -5844,6 +5953,13 @@ public abstract class SqlOperatorBaseTest {
         "map['washington', 1, 'obama', 44]",
         "(CHAR(10) NOT NULL, INTEGER NOT NULL) MAP NOT NULL",
         "{washington=1, obama=44}");
+
+    final SqlTester tester2 =
+        tester.withConformance(SqlConformanceEnum.PRAGMATIC_2003);
+    tester2.checkScalarExact(
+        "map['washington', 1, 'obama', 44]",
+        "(VARCHAR(10) NOT NULL, INTEGER NOT NULL) MAP NOT NULL",
+        "{washington=1, obama=44}");
   }
 
   @Test public void testCeilFunc() {