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 2022/11/30 17:37:46 UTC

[calcite] branch main updated: [CALCITE-5159] ARRAY string constructor, and implicit cast from string literal to array literal (enabled in Postgres conformance)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new c89bd13cb2 [CALCITE-5159] ARRAY string constructor, and implicit cast from string literal to array literal (enabled in Postgres conformance)
c89bd13cb2 is described below

commit c89bd13cb209a0f4403dd7a82fdf24e2ee247988
Author: dssysolyatin <dm...@gmail.com>
AuthorDate: Tue Nov 29 19:13:01 2022 -0800

    [CALCITE-5159] ARRAY string constructor, and implicit cast from string literal to array literal (enabled in Postgres conformance)
    
    After this change, you can write an ARRAY constructor
    followed by a string literal:
    
      ARRAY '{1,2,3}'
    
    in a connection with Postgres conformance, and it is
    equivalent to
    
      ARRAY [1,2,3]
    
    Also, if you use a string literal in a context where an
    array is needed, the string literal is treated as an array
    literal. For example,
    
      SELECT ARRAY [1,2,3] = '{1,2,3}'
    
    is equivalent to
    
      SELECT ARRAY [1,2,3] = ARRAY '{1,2,3}'
    
    Close apache/calcite#2819
---
 babel/src/main/codegen/config.fmpp                 |   1 +
 .../org/apache/calcite/test/BabelDdlExecutor.java  |  43 ++++
 .../org/apache/calcite/test/BabelParserTest.java   |  28 +++
 .../org/apache/calcite/test/BabelQuidemTest.java   |  10 +
 babel/src/test/resources/sql/postgresql.iq         |  59 +++++
 core/src/main/codegen/default_config.fmpp          |   1 +
 core/src/main/codegen/templates/Parser.jj          |  45 ++++
 .../apache/calcite/runtime/CalciteResource.java    |   3 +
 .../calcite/sql/parser/SqlAbstractParserImpl.java  |   5 +
 .../apache/calcite/sql/parser/SqlParserUtil.java   |  19 ++
 .../org/apache/calcite/sql/type/SqlTypeUtil.java   |   5 +-
 .../sql/validate/SqlAbstractConformance.java       |   3 +
 .../calcite/sql/validate/SqlConformance.java       |  16 ++
 .../calcite/sql/validate/SqlConformanceEnum.java   |   8 +
 .../calcite/sql/validate/SqlValidatorImpl.java     |  18 +-
 .../validate/implicit/AbstractTypeCoercion.java    |  46 +++-
 .../calcite/runtime/CalciteResource.properties     |   1 +
 core/src/test/codegen/config.fmpp                  |   4 +-
 core/src/test/codegen/includes/parserImpls.ftl     |   2 +-
 ...eateTable.java => ExtensionSqlCreateTable.java} |  26 +--
 .../apache/calcite/test/ExtensionDdlExecutor.java  | 257 +--------------------
 site/_docs/reference.md                            |  37 +--
 .../apache/calcite/sql/parser/SqlParserTest.java   |   6 +-
 .../org/apache/calcite/test/MockDdlExecutor.java   |  60 +++--
 24 files changed, 374 insertions(+), 329 deletions(-)

diff --git a/babel/src/main/codegen/config.fmpp b/babel/src/main/codegen/config.fmpp
index c4168665b7..2d1ababb42 100644
--- a/babel/src/main/codegen/config.fmpp
+++ b/babel/src/main/codegen/config.fmpp
@@ -567,6 +567,7 @@ data: {
     ]
 
     includePosixOperators: true
+    includeParsingStringLiteralAsArrayLiteral: true
   }
 }
 
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelDdlExecutor.java b/babel/src/test/java/org/apache/calcite/test/BabelDdlExecutor.java
new file mode 100644
index 0000000000..5002bf131e
--- /dev/null
+++ b/babel/src/test/java/org/apache/calcite/test/BabelDdlExecutor.java
@@ -0,0 +1,43 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.server.DdlExecutor;
+import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
+
+import java.io.Reader;
+
+/** Executes the few DDL commands supported by
+ * {@link SqlBabelParserImpl}. */
+public class BabelDdlExecutor extends MockDdlExecutor {
+  static final BabelDdlExecutor INSTANCE = new BabelDdlExecutor();
+
+  /** Parser factory. */
+  @SuppressWarnings("unused") // used via reflection
+  public static final SqlParserImplFactory PARSER_FACTORY =
+      new SqlParserImplFactory() {
+        @Override public SqlAbstractParserImpl getParser(Reader stream) {
+          return SqlBabelParserImpl.FACTORY.getParser(stream);
+        }
+
+        @Override public DdlExecutor getDdlExecutor() {
+          return BabelDdlExecutor.INSTANCE;
+        }
+      };
+}
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
index ee5c4fb6e9..04e4bb13cf 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
@@ -304,6 +304,34 @@ class BabelParserTest extends SqlParserTest {
     sql(sql).ok(expected);
   }
 
+  @Test void testArrayLiteralFromString() {
+    sql("select array '{1,2,3}'")
+        .ok("SELECT (ARRAY[1, 2, 3])");
+    sql("select array '{{1,2,5}, {3,4,7}}'")
+        .ok("SELECT (ARRAY[(ARRAY[1, 2, 5]), (ARRAY[3, 4, 7])])");
+    sql("select array '{}'")
+        .ok("SELECT (ARRAY[])");
+    sql("select array '{\"1\", \"2\", \"3\"}'")
+        .ok("SELECT (ARRAY['1', '2', '3'])");
+    sql("select array '{null, 1, null, 2}'")
+        .ok("SELECT (ARRAY[NULL, 1, NULL, 2])");
+
+    sql("select array ^'null, 1, null, 2'^")
+        .fails("Illegal array expression 'null, 1, null, 2'");
+  }
+
+  @Test void testArrayLiteralBigQuery() {
+    final SqlParserFixture f = fixture().withDialect(BIG_QUERY);
+    f.sql("select array '{1, 2}'")
+        .ok("SELECT (ARRAY[1, 2])");
+    f.sql("select array \"{1, 2}\"")
+        .ok("SELECT (ARRAY[1, 2])");
+    f.sql("select array '{\"a\", \"b\"}'")
+        .ok("SELECT (ARRAY['a', 'b'])");
+    f.sql("select array \"{\\\"a\\\", \\\"b\\\"}\"")
+        .ok("SELECT (ARRAY['a', 'b'])");
+  }
+
   /** Similar to {@link #testHoist()} but using custom parser. */
   @Test void testHoistMySql() {
     // SQL contains back-ticks, which require MySQL's quoting,
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java b/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java
index 49eab28ddb..5aba653855 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java
@@ -113,6 +113,16 @@ class BabelQuidemTest extends QuidemTest {
                   SqlConformanceEnum.BABEL)
               .with(CalciteConnectionProperty.LENIENT_OPERATOR_LOOKUP, true)
               .connect();
+        case "scott-postgresql":
+          return CalciteAssert.that()
+              .with(CalciteAssert.SchemaSpec.SCOTT)
+              .with(CalciteConnectionProperty.FUN, "standard,postgresql")
+              .with(CalciteConnectionProperty.PARSER_FACTORY,
+                  BabelDdlExecutor.class.getName() + "#PARSER_FACTORY")
+              .with(CalciteConnectionProperty.CONFORMANCE,
+                  SqlConformanceEnum.BABEL)
+              .with(CalciteConnectionProperty.LENIENT_OPERATOR_LOOKUP, true)
+              .connect();
         default:
           return super.connect(name, reference);
         }
diff --git a/babel/src/test/resources/sql/postgresql.iq b/babel/src/test/resources/sql/postgresql.iq
new file mode 100644
index 0000000000..f82b2a91bb
--- /dev/null
+++ b/babel/src/test/resources/sql/postgresql.iq
@@ -0,0 +1,59 @@
+# postgresql.iq - Babel test for PostgreSQL dialect of SQL
+#
+# 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.
+#
+!use scott-postgresql
+!set outputformat csv
+
+# Test string and array comparison
+select array[0,1,2] = '{0,1,2}';
+EXPR$0
+true
+!ok
+
+select array[0,1,2] = '{3,1,2}';
+EXPR$0
+false
+!ok
+
+select array['hello', 'world'] = '{"hello", "world"}';
+EXPR$0
+true
+!ok
+
+select array['hello', 'world'] = '{1,2}';
+EXPR$0
+false
+!ok
+
+# Test coercion string to array inside INSERT
+create table sal_emp (name varchar, pay_by_quarter int array, schedule varchar array array);
+(0 rows modified)
+
+!update
+
+INSERT INTO sal_emp
+    VALUES ('Bill',
+    '{10000, 10000, 10000, 10000}',
+    '{{"meeting", "lunch"}, {"training", "presentation"}}');
+(1 row modified)
+
+!update
+
+SELECT * FROM sal_emp;
+NAME, PAY_BY_QUARTER, SCHEDULE
+Bill, [10000, 10000, 10000, 10000], [[meeting, lunch], [training, presentation]]
+!ok
diff --git a/core/src/main/codegen/default_config.fmpp b/core/src/main/codegen/default_config.fmpp
index 22ca9289ec..ea59d6c48d 100644
--- a/core/src/main/codegen/default_config.fmpp
+++ b/core/src/main/codegen/default_config.fmpp
@@ -434,4 +434,5 @@ parser: {
   includeCompoundIdentifier: true
   includeBraces: true
   includeAdditionalDeclarations: false
+  includeParsingStringLiteralAsArrayLiteral: false
 }
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index c00da30d18..206f7929b2 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -238,6 +238,17 @@ public class ${parser.class} extends SqlAbstractParserImpl
         return SqlStmtList();
     }
 
+    public SqlNode parseArray() throws SqlParseException {
+        switchTo(LexicalState.BQID);
+        try {
+          return ArrayLiteral();
+        } catch (ParseException ex) {
+          throw normalizeException(ex);
+        } catch (TokenMgrError ex) {
+          throw normalizeException(ex);
+        }
+    }
+
     private SqlNode extend(SqlNode table, SqlNodeList extendList) {
         return SqlStdOperatorTable.EXTEND.createCall(
             Span.of(table, extendList).pos(), table, extendList);
@@ -4637,6 +4648,7 @@ SqlNode ArrayConstructor() :
     SqlNodeList args;
     SqlNode e;
     final Span s;
+    final String p;
 }
 {
     <ARRAY> { s = span(); }
@@ -4663,7 +4675,40 @@ SqlNode ArrayConstructor() :
             return SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(
                 s.end(this), args.getList());
         }
+<#if (parser.includeParsingStringLiteralAsArrayLiteral!default.parser.includeParsingStringLiteralAsArrayLiteral) >
+    |
+        p = SimpleStringLiteral() {
+            try {
+                return SqlParserUtil.parseArrayLiteral(p);
+            } catch (SqlParseException ex) {
+                throw SqlUtil.newContextException(getPos(),
+                    RESOURCE.illegalArrayExpression(p));
+            }
+        }
+</#if>
+    )
+}
+
+SqlCall ArrayLiteral() :
+{
+    final List<SqlNode> list;
+    SqlNode e;
+    final Span s;
+}
+{
+    <LBRACE> { s = span(); }
+    (
+        e = Literal() { list = startList(e); }
+        ( <COMMA> e = Literal() { list.add(e); } )*
+    |
+        e = ArrayLiteral() { list = startList(e); }
+        ( <COMMA> e = ArrayLiteral() { list.add(e); } )*
+    |
+        { list = Collections.emptyList(); }
     )
+    <RBRACE> {
+       return SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(s.end(this), list);
+    }
 }
 
 /** Parses a MAP constructor */
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 88245302bc..47150f5503 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -94,6 +94,9 @@ public interface CalciteResource {
   @BaseMessage("Illegal binary string {0}")
   ExInst<CalciteException> illegalBinaryString(String a0);
 
+  @BaseMessage("Illegal array expression ''{0}''")
+  ExInst<CalciteException> illegalArrayExpression(String a0);
+
   @BaseMessage("''FROM'' without operands preceding it is illegal")
   ExInst<CalciteException> illegalFromEmpty();
 
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
index 9c46a38215..3337442e4a 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
@@ -557,6 +557,11 @@ public abstract class SqlAbstractParserImpl {
    */
   public abstract void setConformance(SqlConformance conformance);
 
+  /**
+   * Parses string to array literal.
+   */
+  public abstract SqlNode parseArray() throws SqlParseException;
+
   /**
    * Sets the SQL text that is being parsed.
    */
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
index 5141c19d2e..c3f69220bb 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
@@ -39,6 +39,7 @@ import org.apache.calcite.sql.SqlTimeLiteral;
 import org.apache.calcite.sql.SqlTimestampLiteral;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.PrecedenceClimbingParser;
 import org.apache.calcite.util.TimeString;
@@ -52,6 +53,7 @@ import com.google.common.collect.ImmutableList;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 
+import java.io.StringReader;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.charset.Charset;
@@ -369,6 +371,23 @@ public final class SqlParserUtil {
     return SqlLiteral.createInterval(sign, s, intervalQualifier, pos);
   }
 
+  /**
+   * Parses string to array literal
+   * using {@link org.apache.calcite.sql.parser.impl.SqlParserImpl} parser.
+   * String format description can be found at the
+   * <a href="https://www.postgresql.org/docs/current/arrays.html#ARRAYS-INPUT">link</a>
+   *
+   * @param s a string to parse
+   * @return a array value
+   *
+   * @throws SqlParseException if there is a parse error
+   */
+  public static SqlNode parseArrayLiteral(String s) throws SqlParseException {
+    SqlAbstractParserImpl parser = SqlParserImpl.FACTORY.getParser(
+        new StringReader(s));
+    return parser.parseArray();
+  }
+
   /**
    * Checks if the date/time format is valid, throws if not.
    *
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 5546dc6ee9..da283526b3 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
@@ -927,10 +927,9 @@ public abstract class SqlTypeUtil {
     RelDataType c1 = toType.getComponentType();
     if (c1 != null) {
       RelDataType c2 = fromType.getComponentType();
-      if (c2 == null) {
-        return false;
+      if (c2 != null) {
+        return canCastFrom(c1, c2, coerce);
       }
-      return canCastFrom(c1, c2, coerce);
     }
     if ((isInterval(fromType) && isExactNumeric(toType))
         || (isInterval(toType) && isExactNumeric(fromType))) {
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 0f00b5985f..5fd1855a1f 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
@@ -137,4 +137,7 @@ public abstract class SqlAbstractConformance implements SqlConformance {
     return SqlConformanceEnum.DEFAULT.semantics();
   }
 
+  @Override public boolean allowCoercionStringToArray() {
+    return SqlConformanceEnum.DEFAULT.allowCoercionStringToArray();
+  }
 }
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 e6493b3a09..a0a7dceefa 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
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.validate;
 
+import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.sql.fun.SqlLibrary;
 
 /**
@@ -548,4 +549,19 @@ public interface SqlConformance {
    * </ul>
    */
   SqlLibrary semantics();
+
+  /**
+   * Whether to allow coercion string literal to array literal
+   *
+   * <p>For example,
+   *
+   * <blockquote><pre>SELECT ARRAY[0,1,2] == '{0,1,2}'
+   * </pre></blockquote>
+   *
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#BABEL},
+   * false otherwise.
+   */
+  @Experimental
+  boolean allowCoercionStringToArray();
 }
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 dd56474103..911f9e675a 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
@@ -418,4 +418,12 @@ public enum SqlConformanceEnum implements SqlConformance {
     }
   }
 
+  @Override public boolean allowCoercionStringToArray() {
+    switch (this) {
+    case BABEL:
+      return true;
+    default:
+      return false;
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 0db4eaa8a4..bcc1dc9776 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -320,7 +320,23 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     @SuppressWarnings("argument.type.incompatible")
     TypeCoercion typeCoercion = config.typeCoercionFactory().create(typeFactory, this);
     this.typeCoercion = typeCoercion;
-    if (config.typeCoercionRules() != null) {
+
+    if (config.conformance().allowCoercionStringToArray()) {
+      SqlTypeCoercionRule rules = requireNonNull(config.typeCoercionRules() != null
+          ? config.typeCoercionRules() : SqlTypeCoercionRule.THREAD_PROVIDERS.get());
+
+      ImmutableSet<SqlTypeName> arrayMapping = ImmutableSet.<SqlTypeName>builder()
+          .addAll(rules.getTypeMapping().getOrDefault(SqlTypeName.ARRAY, ImmutableSet.of()))
+          .add(SqlTypeName.VARCHAR)
+          .add(SqlTypeName.CHAR)
+          .build();
+
+      Map<SqlTypeName, ImmutableSet<SqlTypeName>> mapping = new HashMap(rules.getTypeMapping());
+      mapping.replace(SqlTypeName.ARRAY, arrayMapping);
+      rules = SqlTypeCoercionRule.instance(mapping);
+
+      SqlTypeCoercionRule.THREAD_PROVIDERS.set(rules);
+    } else if (config.typeCoercionRules() != null) {
       SqlTypeCoercionRule.THREAD_PROVIDERS.set(config.typeCoercionRules());
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java b/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java
index d4ca0eae29..bf20f221a5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlDynamicParam;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -30,7 +31,9 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
@@ -103,12 +106,17 @@ public abstract class AbstractTypeCoercion implements TypeCoercion {
       return false;
     }
     requireNonNull(scope, "scope");
+    RelDataType operandType = validator.deriveType(scope, operand);
+    if (coerceStringToArray(call, operand, index, operandType, targetType)) {
+      return true;
+    }
+
     // Check it early.
     if (!needToCast(scope, operand, targetType)) {
       return false;
     }
     // Fix up nullable attr.
-    RelDataType targetType1 = syncAttributes(validator.deriveType(scope, operand), targetType);
+    RelDataType targetType1 = syncAttributes(operandType, targetType);
     SqlNode desired = castTo(operand, targetType1);
     call.setOperand(index, desired);
     updateInferredType(desired, targetType1);
@@ -525,6 +533,16 @@ public abstract class AbstractTypeCoercion implements TypeCoercion {
       return type2;
     }
 
+    if (validator.config().conformance().allowCoercionStringToArray()) {
+      if (SqlTypeUtil.isString(type1) && SqlTypeUtil.isArray(type2)) {
+        return type2;
+      }
+
+      if (SqlTypeUtil.isString(type2) && SqlTypeUtil.isArray(type1)) {
+        return type1;
+      }
+    }
+
     return null;
   }
 
@@ -725,4 +743,30 @@ public abstract class AbstractTypeCoercion implements TypeCoercion {
     }
     return null;
   }
+
+  /**
+   * Coerce STRING type to ARRAY type.
+   */
+  protected Boolean coerceStringToArray(
+      SqlCall call,
+      SqlNode operand,
+      int index,
+      RelDataType fromType,
+      RelDataType targetType) {
+    if (validator.config().conformance().allowCoercionStringToArray()
+        && SqlTypeUtil.isString(fromType)
+        && SqlTypeUtil.isArray(targetType)
+        && operand instanceof SqlCharStringLiteral) {
+      try {
+        SqlNode arrayValue = SqlParserUtil.parseArrayLiteral(
+            ((SqlCharStringLiteral) operand).getValueAs(String.class));
+        call.setOperand(index, arrayValue);
+        updateInferredType(arrayValue, targetType);
+      } catch (SqlParseException e) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
 }
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index cd8135ff43..d8992d57af 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -39,6 +39,7 @@ IllegalOrderBy=ORDER BY unexpected
 IllegalJoinExpression=Join expression encountered in illegal context
 ExpectedQueryOrJoinExpression=Expected query or join
 IllegalBinaryString=Illegal binary string {0}
+IllegalArrayExpression=Illegal array expression ''{0}''
 IllegalFromEmpty=''FROM'' without operands preceding it is illegal
 IllegalRowExpression=ROW expression encountered in illegal context
 IllegalColon=Illegal identifier '':''. Was expecting ''VALUE''
diff --git a/core/src/test/codegen/config.fmpp b/core/src/test/codegen/config.fmpp
index 53d8d20db1..99ecb93da9 100644
--- a/core/src/test/codegen/config.fmpp
+++ b/core/src/test/codegen/config.fmpp
@@ -28,8 +28,8 @@ data: {
     # Example: "org.apache.calcite.sql.*", "java.util.List".
     imports: [
       "org.apache.calcite.sql.SqlCreate",
-      "org.apache.calcite.sql.ddl.SqlDdlNodes"
-      "org.apache.calcite.sql.parser.parserextensiontesting.SqlCreateTable",
+      "org.apache.calcite.sql.ddl.SqlDdlNodes",
+      "org.apache.calcite.sql.parser.parserextensiontesting.ExtensionSqlCreateTable",
       "org.apache.calcite.sql.parser.parserextensiontesting.SqlUploadJarNode"
     ]
 
diff --git a/core/src/test/codegen/includes/parserImpls.ftl b/core/src/test/codegen/includes/parserImpls.ftl
index 866fbffac1..482867753e 100644
--- a/core/src/test/codegen/includes/parserImpls.ftl
+++ b/core/src/test/codegen/includes/parserImpls.ftl
@@ -53,7 +53,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) :
     |   { query = null; }
     )
     {
-        return new SqlCreateTable(s.end(this), id, columnList, query);
+        return new ExtensionSqlCreateTable(s.end(this), id, columnList, query);
     }
 }
 
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlCreateTable.java
similarity index 75%
rename from core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
rename to core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlCreateTable.java
index ab250c96cc..24148655e5 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlCreateTable.java
@@ -16,44 +16,28 @@
  */
 package org.apache.calcite.sql.parser.parserextensiontesting;
 
-import org.apache.calcite.sql.SqlCreate;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.ddl.SqlCreateTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import org.checkerframework.checker.nullness.qual.Nullable;
-
 import java.util.List;
-import java.util.Objects;
 import java.util.function.BiConsumer;
 
 /**
  * Simple test example of a CREATE TABLE statement.
  */
-public class SqlCreateTable extends SqlCreate {
-  public final SqlIdentifier name;
-  public final @Nullable SqlNodeList columnList;
-  public final @Nullable SqlNode query;
-
-  private static final SqlOperator OPERATOR =
-      new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
-
+public class ExtensionSqlCreateTable extends SqlCreateTable {
   /** Creates a SqlCreateTable. */
-  public SqlCreateTable(SqlParserPos pos, SqlIdentifier name,
-      @Nullable SqlNodeList columnList, @Nullable SqlNode query) {
-    super(OPERATOR, pos, false, false);
-    this.name = Objects.requireNonNull(name, "name");
-    this.columnList = columnList; // may be null
-    this.query = query; // for "CREATE TABLE ... AS query"; may be null
+  public ExtensionSqlCreateTable(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList, SqlNode query) {
+    super(pos, false, false, name, columnList, query);
   }
 
   @Override public List<SqlNode> getOperandList() {
diff --git a/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java b/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
index c757ee3200..1466663cb9 100644
--- a/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
+++ b/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
@@ -16,73 +16,21 @@
  */
 package org.apache.calcite.test;
 
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.ContextSqlValidator;
-import org.apache.calcite.linq4j.Enumerator;
-import org.apache.calcite.linq4j.Linq4j;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.linq4j.Queryable;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.rel.RelRoot;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeImpl;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.impl.AbstractTableQueryable;
-import org.apache.calcite.schema.impl.ViewTable;
-import org.apache.calcite.schema.impl.ViewTableMacro;
 import org.apache.calcite.server.DdlExecutor;
-import org.apache.calcite.server.DdlExecutorImpl;
-import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.SqlUtil;
-import org.apache.calcite.sql.ddl.SqlCreateView;
-import org.apache.calcite.sql.dialect.CalciteSqlDialect;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.ddl.SqlCreateTable;
 import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
-import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParserImplFactory;
-import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.parserextensiontesting.ExtensionSqlCreateTable;
 import org.apache.calcite.sql.parser.parserextensiontesting.ExtensionSqlParserImpl;
-import org.apache.calcite.sql.parser.parserextensiontesting.SqlCreateTable;
-import org.apache.calcite.sql.pretty.SqlPrettyWriter;
-import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
-
-import com.google.common.collect.ImmutableList;
-
-import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.io.Reader;
-import java.lang.reflect.Type;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import static org.apache.calcite.util.Static.RESOURCE;
-
-import static java.util.Objects.requireNonNull;
+import java.util.function.BiConsumer;
 
 /** Executes the few DDL commands supported by
  * {@link ExtensionSqlParserImpl}. */
-public class ExtensionDdlExecutor extends DdlExecutorImpl {
+public class ExtensionDdlExecutor extends MockDdlExecutor {
   static final ExtensionDdlExecutor INSTANCE = new ExtensionDdlExecutor();
 
   /** Parser factory. */
@@ -98,197 +46,8 @@ public class ExtensionDdlExecutor extends DdlExecutorImpl {
         }
       };
 
-  /** Returns the schema in which to create an object. */
-  static Pair<CalciteSchema, String> schema(CalcitePrepare.Context context,
-      boolean mutable, SqlIdentifier id) {
-    final String name;
-    final List<String> path;
-    if (id.isSimple()) {
-      path = context.getDefaultSchemaPath();
-      name = id.getSimple();
-    } else {
-      path = Util.skipLast(id.names);
-      name = Util.last(id.names);
-    }
-    CalciteSchema schema = mutable ? context.getMutableRootSchema()
-        : context.getRootSchema();
-    for (String p : path) {
-      schema = requireNonNull(schema.getSubSchema(p, true));
-    }
-    return Pair.of(schema, name);
-  }
-
-  /** Wraps a query to rename its columns. Used by CREATE VIEW and CREATE
-   * MATERIALIZED VIEW. */
-  static SqlNode renameColumns(@Nullable SqlNodeList columnList,
-      SqlNode query) {
-    if (columnList == null) {
-      return query;
-    }
-    final SqlParserPos p = query.getParserPosition();
-    final SqlNodeList selectList = SqlNodeList.SINGLETON_STAR;
-    final SqlCall from =
-        SqlStdOperatorTable.AS.createCall(p,
-            ImmutableList.<SqlNode>builder()
-                .add(query)
-                .add(new SqlIdentifier("_", p))
-                .addAll(columnList)
-                .build());
-    return new SqlSelect(p, null, selectList, from, null, null, null, null,
-        null, null, null, null);
-  }
-
-  /** Executes a {@code CREATE TABLE} command. Called via reflection. */
-  public void execute(SqlCreateTable create, CalcitePrepare.Context context) {
-    final CalciteSchema schema =
-        Schemas.subSchema(context.getRootSchema(),
-            context.getDefaultSchemaPath());
-    final JavaTypeFactory typeFactory = context.getTypeFactory();
-    final RelDataType queryRowType;
-    if (create.query != null) {
-      // A bit of a hack: pretend it's a view, to get its row type
-      final String sql =
-          create.query.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
-      final ViewTableMacro viewTableMacro =
-          ViewTable.viewMacro(schema.plus(), sql, schema.path(null),
-              context.getObjectPath(), false);
-      final TranslatableTable x = viewTableMacro.apply(ImmutableList.of());
-      queryRowType = x.getRowType(typeFactory);
-
-      if (create.columnList != null
-          && queryRowType.getFieldCount() != create.columnList.size()) {
-        throw SqlUtil.newContextException(create.columnList.getParserPosition(),
-            RESOURCE.columnCountMismatch());
-      }
-    } else {
-      queryRowType = null;
-    }
-    final RelDataTypeFactory.Builder builder = typeFactory.builder();
-    if (create.columnList != null) {
-      final SqlValidator validator = new ContextSqlValidator(context, false);
-      create.forEachNameType((name, typeSpec) ->
-          builder.add(name.getSimple(), typeSpec.deriveType(validator, true)));
-    } else {
-      if (queryRowType == null) {
-        // "CREATE TABLE t" is invalid; because there is no "AS query" we need
-        // a list of column names and types, "CREATE TABLE t (INT c)".
-        throw SqlUtil.newContextException(create.name.getParserPosition(),
-            RESOURCE.createTableRequiresColumnList());
-      }
-      builder.addAll(queryRowType.getFieldList());
-    }
-    final RelDataType rowType = builder.build();
-    schema.add(create.name.getSimple(),
-        new MutableArrayTable(create.name.getSimple(),
-            RelDataTypeImpl.proto(rowType)));
-    if (create.query != null) {
-      populate(create.name, create.query, context);
-    }
-  }
-
-  /** Executes a {@code CREATE VIEW} command. */
-  public void execute(SqlCreateView create,
-      CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        schema(context, true, create.name);
-    final SchemaPlus schemaPlus = pair.left.plus();
-    for (Function function : schemaPlus.getFunctions(pair.right)) {
-      if (function.getParameters().isEmpty()) {
-        if (!create.getReplace()) {
-          throw SqlUtil.newContextException(create.name.getParserPosition(),
-              RESOURCE.viewExists(pair.right));
-        }
-        pair.left.removeFunction(pair.right);
-      }
-    }
-    final SqlNode q = renameColumns(create.columnList, create.query);
-    final String sql = q.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
-    final ViewTableMacro viewTableMacro =
-        ViewTable.viewMacro(schemaPlus, sql, pair.left.path(null),
-            context.getObjectPath(), false);
-    final TranslatableTable x = viewTableMacro.apply(ImmutableList.of());
-    Util.discard(x);
-    schemaPlus.add(pair.right, viewTableMacro);
-  }
-
-  /** Populates the table called {@code name} by executing {@code query}. */
-  protected static void populate(SqlIdentifier name, SqlNode query,
-      CalcitePrepare.Context context) {
-    // Generate, prepare and execute an "INSERT INTO table query" statement.
-    // (It's a bit inefficient that we convert from SqlNode to SQL and back
-    // again.)
-    final FrameworkConfig config = Frameworks.newConfigBuilder()
-        .defaultSchema(
-            requireNonNull(
-                Schemas.subSchema(context.getRootSchema(),
-                    context.getDefaultSchemaPath())).plus())
-        .build();
-    final Planner planner = Frameworks.getPlanner(config);
-    try {
-      final StringBuilder buf = new StringBuilder();
-      final SqlPrettyWriter w =
-          new SqlPrettyWriter(
-              SqlPrettyWriter.config()
-                  .withDialect(CalciteSqlDialect.DEFAULT)
-                  .withAlwaysUseParentheses(false),
-              buf);
-      buf.append("INSERT INTO ");
-      name.unparse(w, 0, 0);
-      buf.append(" ");
-      query.unparse(w, 0, 0);
-      final String sql = buf.toString();
-      final SqlNode query1 = planner.parse(sql);
-      final SqlNode query2 = planner.validate(query1);
-      final RelRoot r = planner.rel(query2);
-      final PreparedStatement prepare =
-          context.getRelRunner().prepareStatement(r.rel);
-      int rowCount = prepare.executeUpdate();
-      Util.discard(rowCount);
-      prepare.close();
-    } catch (SqlParseException | ValidationException
-        | RelConversionException | SQLException e) {
-      throw Util.throwAsRuntime(e);
-    }
-  }
-
-  /** Table backed by a Java list. */
-  private static class MutableArrayTable
-      extends AbstractModifiableTable {
-    final List list = new ArrayList();
-    private final RelProtoDataType protoRowType;
-
-    MutableArrayTable(String name, RelProtoDataType protoRowType) {
-      super(name);
-      this.protoRowType = protoRowType;
-    }
-
-    public Collection getModifiableCollection() {
-      return list;
-    }
-
-    public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
-        SchemaPlus schema, String tableName) {
-      return new AbstractTableQueryable<T>(queryProvider, schema, this,
-          tableName) {
-        public Enumerator<T> enumerator() {
-          //noinspection unchecked
-          return (Enumerator<T>) Linq4j.enumerator(list);
-        }
-      };
-    }
-
-    public Type getElementType() {
-      return Object[].class;
-    }
-
-    public Expression getExpression(SchemaPlus schema, String tableName,
-        Class clazz) {
-      return Schemas.tableExpression(schema, getElementType(),
-          tableName, clazz);
-    }
-
-    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-      return protoRowType.apply(typeFactory);
-    }
+  @Override protected void forEachNameType(SqlCreateTable createTable, BiConsumer<SqlIdentifier,
+      SqlDataTypeSpec> consumer) {
+    ((ExtensionSqlCreateTable) createTable).forEachNameType(consumer);
   }
 }
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index fe11d27e3c..edd4537977 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1521,24 +1521,25 @@ Calcite type conversions. The table shows all possible conversions,
 without regard to the context in which it is made. The rules governing
 these details follow the table.
 
-| FROM - TO           | NULL | BOOLEAN | TINYINT | SMALLINT | INT | BIGINT | DECIMAL | FLOAT or REAL | DOUBLE | INTERVAL | DATE | TIME | TIMESTAMP | CHAR or VARCHAR | BINARY or VARBINARY | GEOMETRY
-|:--------------------|:---- |:------- |:------- |:-------- |:--- |:------ |:------- |:------------- |:------ |:-------- |:-----|:-----|:----------|:--------------- |:--------------------|:--------
-| NULL                | i    | i       | i       | i        | i   | i      | i       | i             | i      | i        | i    | i    | i         | i               | i                   | i
-| BOOLEAN             | x    | i       | e       | e        | e   | e      | e       | e             | e      | x        | x    | x    | x         | i               | x                   | x
-| TINYINT             | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x
-| SMALLINT            | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x
-| INT                 | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x
-| BIGINT              | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x
-| DECIMAL             | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x
-| FLOAT/REAL          | x    | e       | i       | i        | i   | i      | i       | i             | i      | x        | x    | x    | e         | i               | x                   | x
-| DOUBLE              | x    | e       | i       | i        | i   | i      | i       | i             | i      | x        | x    | x    | e         | i               | x                   | x
-| INTERVAL            | x    | x       | e       | e        | e   | e      | e       | x             | x      | i        | x    | x    | x         | e               | x                   | x
-| DATE                | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | i    | x    | i         | i               | x                   | x
-| TIME                | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | x    | i    | e         | i               | x                   | x
-| TIMESTAMP           | x    | x       | e       | e        | e   | e      | e       | e             | e      | x        | i    | e    | i         | i               | x                   | x
-| CHAR or VARCHAR     | x    | e       | i       | i        | i   | i      | i       | i             | i      | i        | i    | i    | i         | i               | i                   | i
-| BINARY or VARBINARY | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | e    | e    | e         | i               | i                   | x
-| GEOMETRY            | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | x    | x    | x         | i               | x                   | i
+| FROM - TO           | NULL | BOOLEAN | TINYINT | SMALLINT | INT | BIGINT | DECIMAL | FLOAT or REAL | DOUBLE | INTERVAL | DATE | TIME | TIMESTAMP | CHAR or VARCHAR | BINARY or VARBINARY | GEOMETRY | ARRAY |
+|:--------------------|:-----|:--------|:--------|:---------|:----|:-------|:--------|:--------------|:-------|:---------|:-----|:-----|:----------|:----------------|:--------------------|:---------|:------|
+| NULL                | i    | i       | i       | i        | i   | i      | i       | i             | i      | i        | i    | i    | i         | i               | i                   | i        | x     |
+| BOOLEAN             | x    | i       | e       | e        | e   | e      | e       | e             | e      | x        | x    | x    | x         | i               | x                   | x        | x     |
+| TINYINT             | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x        | x     |
+| SMALLINT            | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x        | x     |
+| INT                 | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x        | x     |
+| BIGINT              | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x        | x     |
+| DECIMAL             | x    | e       | i       | i        | i   | i      | i       | i             | i      | e        | x    | x    | e         | i               | x                   | x        | x     |
+| FLOAT/REAL          | x    | e       | i       | i        | i   | i      | i       | i             | i      | x        | x    | x    | e         | i               | x                   | x        | x     |
+| DOUBLE              | x    | e       | i       | i        | i   | i      | i       | i             | i      | x        | x    | x    | e         | i               | x                   | x        | x     |
+| INTERVAL            | x    | x       | e       | e        | e   | e      | e       | x             | x      | i        | x    | x    | x         | e               | x                   | x        | x     |
+| DATE                | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | i    | x    | i         | i               | x                   | x        | x     |
+| TIME                | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | x    | i    | e         | i               | x                   | x        | x     |
+| TIMESTAMP           | x    | x       | e       | e        | e   | e      | e       | e             | e      | x        | i    | e    | i         | i               | x                   | x        | x     |
+| CHAR or VARCHAR     | x    | e       | i       | i        | i   | i      | i       | i             | i      | i        | i    | i    | i         | i               | i                   | i        | i     |
+| BINARY or VARBINARY | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | e    | e    | e         | i               | i                   | x        | x     |
+| GEOMETRY            | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | x    | x    | x         | i               | x                   | i        | x     |
+| ARRAY               | x    | x       | x       | x        | x   | x      | x       | x             | x      | x        | x    | x    | x         | x               | x                   | x        | i     |
 
 i: implicit cast / e: explicit cast / x: not allowed
 
diff --git a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
index e95dc05631..92c4ec6e21 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -44,6 +44,7 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSortedSet;
 
@@ -582,7 +583,7 @@ public class SqlParserTest {
           .withFromFolding(SqlWriterConfig.LineFolding.TALL)
           .withIndentation(0);
 
-  private static final SqlDialect BIG_QUERY =
+  protected static final SqlDialect BIG_QUERY =
       SqlDialect.DatabaseProduct.BIG_QUERY.getDialect();
   private static final SqlDialect CALCITE =
       SqlDialect.DatabaseProduct.CALCITE.getDialect();
@@ -5026,7 +5027,8 @@ public class SqlParserTest {
     };
   }
 
-  @Test void testCaseExpression() {
+  @VisibleForTesting
+  @Test public void testCaseExpression() {
     // implicit simple "ELSE NULL" case
     expr("case \t col1 when 1 then 'one' end")
         .ok("(CASE WHEN (`COL1` = 1) THEN 'one' ELSE NULL END)");
diff --git a/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java b/testkit/src/main/java/org/apache/calcite/test/MockDdlExecutor.java
similarity index 87%
copy from core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
copy to testkit/src/main/java/org/apache/calcite/test/MockDdlExecutor.java
index c757ee3200..1b29f7a767 100644
--- a/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
+++ b/testkit/src/main/java/org/apache/calcite/test/MockDdlExecutor.java
@@ -37,23 +37,21 @@ import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.schema.impl.ViewTable;
 import org.apache.calcite.schema.impl.ViewTableMacro;
-import org.apache.calcite.server.DdlExecutor;
 import org.apache.calcite.server.DdlExecutorImpl;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
+import org.apache.calcite.sql.ddl.SqlCreateTable;
 import org.apache.calcite.sql.ddl.SqlCreateView;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
 import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.parser.parserextensiontesting.ExtensionSqlParserImpl;
-import org.apache.calcite.sql.parser.parserextensiontesting.SqlCreateTable;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.tools.FrameworkConfig;
@@ -68,35 +66,20 @@ import com.google.common.collect.ImmutableList;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
-import java.io.Reader;
 import java.lang.reflect.Type;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.function.BiConsumer;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
 import static java.util.Objects.requireNonNull;
 
-/** Executes the few DDL commands supported by
- * {@link ExtensionSqlParserImpl}. */
-public class ExtensionDdlExecutor extends DdlExecutorImpl {
-  static final ExtensionDdlExecutor INSTANCE = new ExtensionDdlExecutor();
-
-  /** Parser factory. */
-  @SuppressWarnings("unused") // used via reflection
-  public static final SqlParserImplFactory PARSER_FACTORY =
-      new SqlParserImplFactory() {
-        @Override public SqlAbstractParserImpl getParser(Reader stream) {
-          return ExtensionSqlParserImpl.FACTORY.getParser(stream);
-        }
-
-        @Override public DdlExecutor getDdlExecutor() {
-          return ExtensionDdlExecutor.INSTANCE;
-        }
-      };
+/** Executes the few DDL commands. */
+public class MockDdlExecutor extends DdlExecutorImpl {
 
   /** Returns the schema in which to create an object. */
   static Pair<CalciteSchema, String> schema(CalcitePrepare.Context context,
@@ -166,7 +149,7 @@ public class ExtensionDdlExecutor extends DdlExecutorImpl {
     final RelDataTypeFactory.Builder builder = typeFactory.builder();
     if (create.columnList != null) {
       final SqlValidator validator = new ContextSqlValidator(context, false);
-      create.forEachNameType((name, typeSpec) ->
+      forEachNameType(create, (name, typeSpec) ->
           builder.add(name.getSimple(), typeSpec.deriveType(validator, true)));
     } else {
       if (queryRowType == null) {
@@ -246,11 +229,26 @@ public class ExtensionDdlExecutor extends DdlExecutorImpl {
       Util.discard(rowCount);
       prepare.close();
     } catch (SqlParseException | ValidationException
-        | RelConversionException | SQLException e) {
+             | RelConversionException | SQLException e) {
       throw Util.throwAsRuntime(e);
     }
   }
 
+  /** Calls an action for each (name, type) pair from {@code SqlCreateTable::columnList}, in which
+   * they alternate. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  protected void forEachNameType(SqlCreateTable createTable,
+      BiConsumer<SqlIdentifier, SqlDataTypeSpec> consumer) {
+    createTable.columnList.forEach(sqlNode -> {
+      if (sqlNode instanceof SqlColumnDeclaration) {
+        final SqlColumnDeclaration d = (SqlColumnDeclaration) sqlNode;
+        consumer.accept(d.name, d.dataType);
+      } else {
+        throw new AssertionError(sqlNode.getClass());
+      }
+    });
+  }
+
   /** Table backed by a Java list. */
   private static class MutableArrayTable
       extends AbstractModifiableTable {
@@ -262,32 +260,32 @@ public class ExtensionDdlExecutor extends DdlExecutorImpl {
       this.protoRowType = protoRowType;
     }
 
-    public Collection getModifiableCollection() {
+    @Override public Collection getModifiableCollection() {
       return list;
     }
 
-    public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
+    @Override public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
         SchemaPlus schema, String tableName) {
       return new AbstractTableQueryable<T>(queryProvider, schema, this,
           tableName) {
-        public Enumerator<T> enumerator() {
+        @Override public Enumerator<T> enumerator() {
           //noinspection unchecked
           return (Enumerator<T>) Linq4j.enumerator(list);
         }
       };
     }
 
-    public Type getElementType() {
+    @Override public Type getElementType() {
       return Object[].class;
     }
 
-    public Expression getExpression(SchemaPlus schema, String tableName,
+    @Override public Expression getExpression(SchemaPlus schema, String tableName,
         Class clazz) {
       return Schemas.tableExpression(schema, getElementType(),
           tableName, clazz);
     }
 
-    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+    @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
       return protoRowType.apply(typeFactory);
     }
   }