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 2016/12/08 06:05:02 UTC

[3/3] calcite git commit: [CALCITE-1529] Support CREATE TABLE in tests (and only in tests)

[CALCITE-1529] Support CREATE TABLE in tests (and only in tests)

Upgrade freemarker.

Add "parserFactory" to config.


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

Branch: refs/heads/master
Commit: e641211fd72a4f4a24f0f4644034c4b2f3c040ca
Parents: 267e2f3
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Dec 5 22:49:22 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 7 18:35:30 2016 -0800

----------------------------------------------------------------------
 core/src/main/codegen/config.fmpp               |  12 +-
 core/src/main/codegen/templates/Parser.jj       |  61 +++++++
 .../enumerable/EnumerableRelImplementor.java    |  13 ++
 .../enumerable/EnumerableTableModify.java       |   5 +-
 .../adapter/enumerable/EnumerableTableScan.java |   7 +
 .../calcite/config/CalciteConnectionConfig.java |   2 +
 .../config/CalciteConnectionConfigImpl.java     |   6 +
 .../config/CalciteConnectionProperty.java       |   6 +
 .../calcite/prepare/CalcitePrepareImpl.java     |  24 ++-
 .../java/org/apache/calcite/sql/SqlCreate.java  |  46 +++++
 .../org/apache/calcite/sql/SqlDataTypeSpec.java |  37 +++-
 .../java/org/apache/calcite/sql/SqlDrop.java    |  34 ++++
 .../calcite/sql/SqlExecutableStatement.java     |  31 ++++
 core/src/test/codegen/config.fmpp               |  17 +-
 core/src/test/codegen/includes/parserImpls.ftl  |  13 ++
 .../calcite/sql/parser/SqlParserTest.java       |   2 +-
 .../ExtensionSqlParserTest.java                 |   5 +
 .../parserextensiontesting/SqlCreateTable.java  | 169 +++++++++++++++++++
 .../org/apache/calcite/test/CalciteAssert.java  |   3 +
 .../org/apache/calcite/test/QuidemTest.java     |  35 ++--
 core/src/test/resources/sql/blank.iq            |  44 +++++
 pom.xml                                         |   2 +-
 site/_docs/reference.md                         |   1 +
 23 files changed, 538 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/config.fmpp b/core/src/main/codegen/config.fmpp
index 9f7a118..41cfeee 100644
--- a/core/src/main/codegen/config.fmpp
+++ b/core/src/main/codegen/config.fmpp
@@ -70,12 +70,22 @@ data: {
     dataTypeParserMethods: [
     ]
 
-    # List of methods for parsing extensions to ALTER SYSTEM calls.
+    # List of methods for parsing extensions to "ALTER <scope>" calls.
     # Each must accept arguments "(SqlParserPos pos, String scope)".
     # Example: "SqlUploadJarNode"
     alterStatementParserMethods: [
     ]
 
+    # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+    # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+    createStatementParserMethods: [
+    ]
+
+    # List of methods for parsing extensions to "DROP" calls.
+    # Each must accept arguments "(SqlParserPos pos)".
+    dropStatementParserMethods: [
+    ]
+
     # List of files in @includes directory that have parser method
     # implementations for parsing custom SQL statements, literals or types
     # given as part of "statementParserMethods", "literalParserMethods" or

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 390c75c..26d40f4 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -928,6 +928,10 @@ SqlNode SqlStmt() :
         |
         stmt = SqlAlter()
         |
+<#if parser.createStatementParserMethods?size != 0>
+        stmt = SqlCreate()
+        |
+</#if>
         stmt = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
         |
         stmt = SqlExplain()
@@ -3088,6 +3092,61 @@ String Scope() :
     ( <SYSTEM> | <SESSION> ) { return token.image.toUpperCase(); }
 }
 
+<#if parser.createStatementParserMethods?size != 0>
+/**
+ * Parses a CREATE statement.
+ */
+SqlCreate SqlCreate() :
+{
+    final SqlParserPos pos;
+    boolean replace = false;
+    final SqlCreate create;
+}
+{
+    <CREATE> { pos = getPos(); }
+    [
+        <OR> <REPLACE> {
+            replace = true;
+        }
+    ]
+    (
+    <#-- additional literal parser methods are included here -->
+    <#list parser.createStatementParserMethods as method>
+        create = ${method}(pos, replace)
+        <#sep>|</#sep>
+    </#list>
+    )
+    {
+        return create;
+    }
+}
+</#if>
+
+<#if parser.dropStatementParserMethods?size != 0>
+/**
+ * Parses a DROP statement.
+ */
+SqlDrop SqlDrop() :
+{
+    final SqlParserPos pos;
+    boolean replace = false;
+    final SqlDrop drop;
+}
+{
+    <DROP> { pos = getPos(); }
+    (
+    <#-- additional literal parser methods are included here -->
+    <#list parser.dropStatementParserMethods as method>
+        drop = ${method}(pos, replace)
+        <#sep>|</#sep>
+    </#list>
+    )
+    {
+        return drop;
+    }
+}
+</#if>
+
 /**
  * Parses a literal expression, allowing continued string literals.
  * Usually returns an SqlLiteral, but a continued string literal
@@ -5333,6 +5392,7 @@ SqlPostfixOperator PostfixRowOperator() :
     | < RELATIVE: "RELATIVE" >
     | < RELEASE: "RELEASE" >
     | < REPEATABLE: "REPEATABLE" >
+    | < REPLACE: "REPLACE" >
     | < RESET: "RESET" >
     | < RESTART: "RESTART" >
     | < RESTRICT: "RESTRICT" >
@@ -5733,6 +5793,7 @@ String CommonNonReservedKeyWord() :
         | <READ>
         | <RELATIVE>
         | <REPEATABLE>
+        | <REPLACE>
         | <RESTART>
         | <RESTRICT>
         | <RETURNED_CARDINALITY>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
index 90105e0..6e9033c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
@@ -66,6 +66,12 @@ import java.util.Set;
  * operators of {@link EnumerableConvention} calling convention.
  */
 public class EnumerableRelImplementor extends JavaRelImplementor {
+  /** Maximum number of arguments to a constructor. See
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1097">[CALCITE-1097]
+   * Exception when executing query with too many aggregation columns</a> for
+   * details. */
+  private static final int MAX_CONSTRUCTOR_ARG_COUNT = 10;
+
   public final Map<String, Object> map;
   private final Map<String, RexToLixTranslator.InputGetter> corrVars =
       Maps.newHashMap();
@@ -482,6 +488,13 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
       types.add(type);
       return super.visit(newArrayExpression);
     }
+
+    @Override public Void visit(ConstantExpression constantExpression) {
+      if (constantExpression.value instanceof Type) {
+        types.add((Type) constantExpression.value);
+      }
+      return super.visit(constantExpression);
+    }
   }
 
   /** Adds a declaration of each synthetic type found in a code block. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
index 07128e4..14ac0de 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
@@ -101,9 +101,9 @@ public class EnumerableTableModify extends TableModify
     if (!getInput().getRowType().equals(getRowType())) {
       final JavaTypeFactory typeFactory =
           (JavaTypeFactory) getCluster().getTypeFactory();
+      final JavaRowFormat format = EnumerableTableScan.deduceFormat(table);
       PhysType physType =
-          PhysTypeImpl.of(typeFactory, table.getRowType(),
-              JavaRowFormat.CUSTOM);
+          PhysTypeImpl.of(typeFactory, table.getRowType(), format);
       List<Expression> expressionList = new ArrayList<Expression>();
       final PhysType childPhysType = result.physType;
       final ParameterExpression o_ =
@@ -167,6 +167,7 @@ public class EnumerableTableModify extends TableModify
                 ? JavaRowFormat.ARRAY : JavaRowFormat.SCALAR);
     return implementor.result(physType, builder.toBlock());
   }
+
 }
 
 // End EnumerableTableModify.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
index d7529ce..9fd5d34 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
@@ -127,6 +127,13 @@ public class EnumerableTableScan
     }
   }
 
+  public static JavaRowFormat deduceFormat(RelOptTable table) {
+    final Class elementType = deduceElementType(table.unwrap(Table.class));
+    return elementType == Object[].class
+        ? JavaRowFormat.ARRAY
+        : JavaRowFormat.CUSTOM;
+  }
+
   private Expression getExpression(PhysType physType) {
     final Expression expression = table.getExpression(Queryable.class);
     final Expression expression2 = toEnumerable(expression);

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
index 1c3b0d6..6eac17d 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
@@ -48,6 +48,8 @@ public interface CalciteConnectionConfig extends ConnectionConfig {
   Casing quotedCasing();
   /** @see CalciteConnectionProperty#CASE_SENSITIVE */
   boolean caseSensitive();
+  /** @see CalciteConnectionProperty#PARSER_FACTORY */
+  <T> T parserFactory(Class<T> parserFactoryClass, T defaultParserFactory);
   /** @see CalciteConnectionProperty#SCHEMA_FACTORY */
   <T> T schemaFactory(Class<T> schemaFactoryClass, T defaultSchemaFactory);
   /** @see CalciteConnectionProperty#SCHEMA_TYPE */

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
index 5262059..c169d9d 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
@@ -120,6 +120,12 @@ public class CalciteConnectionConfigImpl extends ConnectionConfigImpl
         .getBoolean(lex().caseSensitive);
   }
 
+  public <T> T parserFactory(Class<T> parserFactoryClass,
+      T defaultParserFactory) {
+    return CalciteConnectionProperty.PARSER_FACTORY.wrap(properties)
+        .getPlugin(parserFactoryClass, defaultParserFactory);
+  }
+
   public <T> T schemaFactory(Class<T> schemaFactoryClass,
       T defaultSchemaFactory) {
     return CalciteConnectionProperty.SCHEMA_FACTORY.wrap(properties)

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
index e17740d..20baf7f 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
@@ -78,6 +78,12 @@ public enum CalciteConnectionProperty implements ConnectionProperty {
    *  If not specified, value from {@link #LEX} is used. */
   CASE_SENSITIVE("caseSensitive", Type.BOOLEAN, null, false),
 
+  /** Parser factory.
+   *
+   * <p>The name of a class that implements
+   * {@link org.apache.calcite.sql.parser.SqlParserImplFactory}. */
+  PARSER_FACTORY("parserFactory", Type.PLUGIN, null, false),
+
   /** Name of initial schema. */
   SCHEMA("schema", Type.STRING, null, false),
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 9125d0b..ee7d4ee 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -111,6 +111,7 @@ import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlExecutableStatement;
 import org.apache.calcite.sql.SqlExplainFormat;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlKind;
@@ -121,6 +122,7 @@ import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlConformance;
@@ -431,6 +433,11 @@ public class CalcitePrepareImpl implements CalcitePrepare {
   }
 
   @Override public void executeDdl(Context context, SqlNode node) {
+    if (node instanceof SqlExecutableStatement) {
+      SqlExecutableStatement statement = (SqlExecutableStatement) node;
+      statement.execute(context);
+      return;
+    }
     throw new UnsupportedOperationException();
   }
 
@@ -708,12 +715,17 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     final Meta.StatementType statementType;
     if (query.sql != null) {
       final CalciteConnectionConfig config = context.config();
-      SqlParser parser = createParser(query.sql,
-          createParserConfig()
-              .setQuotedCasing(config.quotedCasing())
-              .setUnquotedCasing(config.unquotedCasing())
-              .setQuoting(config.quoting())
-              .setConformance(config.conformance()));
+      final SqlParser.ConfigBuilder parserConfig = createParserConfig()
+          .setQuotedCasing(config.quotedCasing())
+          .setUnquotedCasing(config.unquotedCasing())
+          .setQuoting(config.quoting())
+          .setConformance(config.conformance());
+      final SqlParserImplFactory parserFactory =
+          config.parserFactory(SqlParserImplFactory.class, null);
+      if (parserFactory != null) {
+        parserConfig.setParserFactory(parserFactory);
+      }
+      SqlParser parser = createParser(query.sql,  parserConfig);
       SqlNode sqlNode;
       try {
         sqlNode = parser.parseStmt();

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/sql/SqlCreate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCreate.java b/core/src/main/java/org/apache/calcite/sql/SqlCreate.java
new file mode 100644
index 0000000..57e9a71
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCreate.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql;
+
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/**
+ * Base class for an CREATE statements parse tree nodes. The portion of the
+ * statement covered by this class is "CREATE [ OR REPLACE ]". Subclasses handle
+ * whatever comes afterwards.
+ */
+public abstract class SqlCreate extends SqlCall {
+
+  /** Whether "OR REPLACE" was specified. */
+  boolean replace;
+
+  public SqlCreate(SqlParserPos pos, boolean replace) {
+    super(pos);
+    this.replace = replace;
+  }
+
+  public boolean getReplace() {
+    return replace;
+  }
+
+  public void setReplace(boolean replace) {
+    this.replace = replace;
+  }
+
+}
+
+// End SqlCreate.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
index c53b832..7c4e2cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
@@ -28,6 +28,8 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+
 import java.nio.charset.Charset;
 import java.util.Objects;
 import java.util.TimeZone;
@@ -158,6 +160,10 @@ public class SqlDataTypeSpec extends SqlNode {
     return timeZone;
   }
 
+  public Boolean getNullable() {
+    return nullable;
+  }
+
   /** Returns a copy of this data type specification with a given
    * nullability. */
   public SqlDataTypeSpec withNullable(Boolean nullable) {
@@ -289,9 +295,22 @@ public class SqlDataTypeSpec extends SqlNode {
    * Does not throw an error if the type is not built-in.
    */
   public RelDataType deriveType(RelDataTypeFactory typeFactory) {
-    String name = typeName.getSimple();
+    return deriveType(typeFactory, false);
+  }
 
-    SqlTypeName sqlTypeName = SqlTypeName.get(name);
+  /**
+   * Converts this type specification to a {@link RelDataType}.
+   *
+   * <p>Does not throw an error if the type is not built-in.
+   *
+   * @param nullable Whether the type is nullable if the type specification
+   *                 does not explicitly state
+   */
+  public RelDataType deriveType(RelDataTypeFactory typeFactory,
+      boolean nullable) {
+    final String name = typeName.getSimple();
+    final SqlTypeName sqlTypeName =
+        Preconditions.checkNotNull(SqlTypeName.get(name));
 
     // NOTE jvs 15-Jan-2009:  earlier validation is supposed to
     // have caught these, which is why it's OK for them
@@ -322,7 +341,8 @@ public class SqlDataTypeSpec extends SqlNode {
         charset = typeFactory.getDefaultCharset();
       } else {
         String javaCharSetName =
-            SqlUtil.translateCharacterSetName(charSetName);
+            Preconditions.checkNotNull(
+                SqlUtil.translateCharacterSetName(charSetName), charSetName);
         charset = Charset.forName(javaCharSetName);
       }
       type =
@@ -334,9 +354,9 @@ public class SqlDataTypeSpec extends SqlNode {
 
     if (null != collectionsTypeName) {
       final String collectionName = collectionsTypeName.getSimple();
-
-      SqlTypeName collectionsSqlTypeName =
-          SqlTypeName.get(collectionName);
+      final SqlTypeName collectionsSqlTypeName =
+          Preconditions.checkNotNull(SqlTypeName.get(collectionName),
+              collectionName);
 
       switch (collectionsSqlTypeName) {
       case MULTISET:
@@ -348,9 +368,10 @@ public class SqlDataTypeSpec extends SqlNode {
       }
     }
 
-    if (nullable != null) {
-      type = typeFactory.createTypeWithNullability(type, nullable);
+    if (this.nullable != null) {
+      nullable = this.nullable;
     }
+    type = typeFactory.createTypeWithNullability(type, nullable);
 
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/sql/SqlDrop.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDrop.java b/core/src/main/java/org/apache/calcite/sql/SqlDrop.java
new file mode 100644
index 0000000..1716d32
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDrop.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql;
+
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/**
+ * Base class for an DROP statements parse tree nodes. The portion of the
+ * statement covered by this class is "DROP". Subclasses handle
+ * whatever comes afterwards.
+ */
+public abstract class SqlDrop extends SqlCall {
+
+  public SqlDrop(SqlParserPos pos) {
+    super(pos);
+  }
+
+}
+
+// End SqlDrop.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java b/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
new file mode 100644
index 0000000..06ca55f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql;
+
+import org.apache.calcite.jdbc.CalcitePrepare;
+
+/**
+ * Mix-in interface for {@link SqlNode} that allows DDL commands to be
+ * executed directly.
+ *
+ * <p>NOTE: Subject to change without notice.
+ */
+public interface SqlExecutableStatement {
+  void execute(CalcitePrepare.Context context);
+}
+
+// End SqlExecutableStatement.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/core/src/test/codegen/config.fmpp b/core/src/test/codegen/config.fmpp
index 7b8b8e2..bd168b1 100644
--- a/core/src/test/codegen/config.fmpp
+++ b/core/src/test/codegen/config.fmpp
@@ -21,6 +21,8 @@ data: {
 
       # List of import statements.
       imports: [
+        "org.apache.calcite.sql.SqlCreate",
+        "org.apache.calcite.sql.parser.parserextensiontesting.SqlCreateTable",
         "org.apache.calcite.sql.parser.parserextensiontesting.SqlUploadJarNode"
       ]
 
@@ -47,14 +49,25 @@ data: {
       dataTypeParserMethods: [
       ]
 
-      # List of methods for parsing extensions to ALTER SYSTEM calls.
+      # List of methods for parsing extensions to "ALTER <scope>" calls.
       # Each must accept arguments "(SqlParserPos pos, String scope)".
       alterStatementParserMethods: [
         "SqlUploadJarNode"
       ]
 
+      # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+      # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+      createStatementParserMethods: [
+        "SqlCreateTable"
+      ]
+
+      # List of methods for parsing extensions to "DROP" calls.
+      # Each must accept arguments "(SqlParserPos pos)".
+      dropStatementParserMethods: [
+      ]
+
       # List of files in @includes directory that have parser method
-      # implementations for custom SQL statements, literals or types
+      # implementations for parsing custom SQL statements, literals or types
       # given as part of "statementParserMethods", "literalParserMethods" or
       # "dataTypeParserMethods".
       implementationFiles: [

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/core/src/test/codegen/includes/parserImpls.ftl b/core/src/test/codegen/includes/parserImpls.ftl
index 2305fc1..56a4099 100644
--- a/core/src/test/codegen/includes/parserImpls.ftl
+++ b/core/src/test/codegen/includes/parserImpls.ftl
@@ -35,3 +35,16 @@ SqlAlter SqlUploadJarNode(SqlParserPos pos, String scope) :
         return new SqlUploadJarNode(pos.plus(getPos()), scope, jarPathsList);
     }
 }
+
+SqlCreate SqlCreateTable(SqlParserPos pos, boolean replace) :
+{
+    final SqlIdentifier id;
+    final SqlNodeList columnList;
+}
+{
+    <TABLE> id = CompoundIdentifier() columnList = ExtendList() {
+        return new SqlCreateTable(pos, id, columnList);
+    }
+}
+
+// End parserImpls.ftl

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 294a2f3..fa12def 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -7308,7 +7308,7 @@ public class SqlParserTest {
 
   /** Helper class for building fluent code such as
    * {@code sql("values 1").ok();}. */
-  private class Sql {
+  protected class Sql {
     private final String sql;
 
     Sql(String sql) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlParserTest.java
index 51e24bd..9d6fcfa 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/ExtensionSqlParserTest.java
@@ -44,6 +44,11 @@ public class ExtensionSqlParserTest extends SqlParserTest {
     checkFails("^upload^ jar '/path/to/jar'",
       "(?s).*Encountered \"upload\" at .*");
   }
+
+  @Test public void testCreateTable() {
+    sql("CREATE TABLE foo.baz(i INTEGER, j VARCHAR(10) NOT NULL)")
+        .ok("CREATE TABLE `FOO`.`BAZ` (`I` INTEGER, `J` VARCHAR(10) NOT NULL)");
+  }
 }
 
 // End ExtensionSqlParserTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
----------------------------------------------------------------------
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/SqlCreateTable.java
new file mode 100644
index 0000000..3889c12
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.parser.parserextensiontesting;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+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.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.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.SqlCreate;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlExecutableStatement;
+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.parser.SqlParserPos;
+import org.apache.calcite.test.JdbcTest;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Simple test example of a CREATE TABLE statement.
+ */
+public class SqlCreateTable extends SqlCreate
+    implements SqlExecutableStatement {
+  private final SqlIdentifier name;
+  private final SqlNodeList columnList;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("CREATE TABLE", SqlKind.OTHER_DDL);
+
+  /** Creates a SqlCreateTable. */
+  public SqlCreateTable(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList) {
+    super(pos, false);
+    this.name = name;
+    this.columnList = columnList;
+  }
+
+  @Override public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return ImmutableList.of(name, columnList);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword("CREATE");
+    writer.keyword("TABLE");
+    name.unparse(writer, leftPrec, rightPrec);
+    SqlWriter.Frame frame = writer.startList("(", ")");
+    for (Pair<SqlIdentifier, SqlDataTypeSpec> pair : nameTypes()) {
+      writer.sep(",");
+      pair.left.unparse(writer, leftPrec, rightPrec); // name
+      pair.right.unparse(writer, leftPrec, rightPrec); // type
+      if (Boolean.FALSE.equals(pair.right.getNullable())) {
+        writer.keyword("NOT NULL");
+      }
+    }
+    writer.endList(frame);
+  }
+
+  /** Creates a list of (name, type) pairs from {@link #columnList}, in which
+   * they alternate. */
+  private List<Pair<SqlIdentifier, SqlDataTypeSpec>> nameTypes() {
+    final List list = columnList.getList();
+    //noinspection unchecked
+    return Pair.zip((List<SqlIdentifier>) Util.quotientList(list, 2, 0),
+        Util.quotientList((List<SqlDataTypeSpec>) list, 2, 1));
+  }
+
+  public void execute(CalcitePrepare.Context context) {
+    final List<String> path = context.getDefaultSchemaPath();
+    CalciteSchema schema = context.getRootSchema();
+    for (String p : path) {
+      schema = schema.getSubSchema(p, true);
+    }
+    final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl();
+    final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder();
+    for (Pair<SqlIdentifier, SqlDataTypeSpec> pair : nameTypes()) {
+      builder.add(pair.left.getSimple(),
+          pair.right.deriveType(typeFactory, true));
+    }
+    final RelDataType rowType = builder.build();
+    schema.add(name.getSimple(),
+        new MutableArrayTable(name.getSimple(),
+            RelDataTypeImpl.proto(rowType)));
+  }
+
+  /** Table backed by a Java list. */
+  private static class MutableArrayTable
+      extends JdbcTest.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);
+    }
+  }
+}
+
+// End SqlCreateTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 2eb14f4..a6ff27f 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -684,6 +684,8 @@ public class CalciteAssert {
     case LINGUAL:
       return rootSchema.add("SALES",
           new ReflectiveSchema(new JdbcTest.LingualSchema()));
+    case BLANK:
+      return rootSchema.add("BLANK", new AbstractSchema());
     case ORINOCO:
       final SchemaPlus orinoco = rootSchema.add("ORINOCO", new AbstractSchema());
       orinoco.add("ORDERS",
@@ -1632,6 +1634,7 @@ public class CalciteAssert {
     HR,
     JDBC_SCOTT,
     SCOTT,
+    BLANK,
     LINGUAL,
     POST,
     ORINOCO

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
index 78b202c..88479c6 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -265,46 +265,48 @@ public class QuidemTest {
         }
         return null;
       }
-      if (name.equals("hr")) {
+      switch (name) {
+      case "hr":
         return CalciteAssert.hr()
             .connect();
-      }
-      if (name.equals("foodmart")) {
+      case "foodmart":
         return CalciteAssert.that()
             .with(CalciteAssert.Config.FOODMART_CLONE)
             .connect();
-      }
-      if (name.equals("scott")) {
+      case "scott":
         return CalciteAssert.that()
             .with(CalciteAssert.Config.SCOTT)
             .connect();
-      }
-      if (name.equals("jdbc_scott")) {
+      case "jdbc_scott":
         return CalciteAssert.that()
             .with(CalciteAssert.Config.JDBC_SCOTT)
             .connect();
-      }
-      if (name.equals("post")) {
+      case "post":
         return CalciteAssert.that()
             .with(CalciteAssert.Config.REGULAR)
             .with(CalciteAssert.SchemaSpec.POST)
             .withDefaultSchema("POST")
             .connect();
-      }
-      if (name.equals("catchall")) {
+      case "catchall":
         return CalciteAssert.that()
             .withSchema("s",
                 new ReflectiveSchema(
                     new ReflectiveSchemaTest.CatchallSchema()))
             .connect();
-      }
-      if (name.equals("orinoco")) {
+      case "orinoco":
         return CalciteAssert.that()
             .with(CalciteAssert.SchemaSpec.ORINOCO)
             .withDefaultSchema("ORINOCO")
             .connect();
-      }
-      if (name.equals("seq")) {
+      case "blank":
+        return CalciteAssert.that()
+            .with("parserFactory",
+                "org.apache.calcite.sql.parser.parserextensiontesting"
+                    + ".ExtensionSqlParserImpl#FACTORY")
+            .with(CalciteAssert.SchemaSpec.BLANK)
+            .withDefaultSchema("BLANK")
+            .connect();
+      case "seq":
         final Connection connection = CalciteAssert.that()
             .withSchema("s", new AbstractSchema())
             .connect();
@@ -323,8 +325,9 @@ public class QuidemTest {
                   }
                 });
         return connection;
+      default:
+        throw new RuntimeException("unknown connection '" + name + "'");
       }
-      throw new RuntimeException("unknown connection '" + name + "'");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/core/src/test/resources/sql/blank.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/blank.iq b/core/src/test/resources/sql/blank.iq
new file mode 100644
index 0000000..e2ce690
--- /dev/null
+++ b/core/src/test/resources/sql/blank.iq
@@ -0,0 +1,44 @@
+# blank.iq - Queries that start from a blank schema and create their own tables
+#
+# 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 blank
+!set outputformat mysql
+
+create table foo (i int not null, j int);
+(-1 rows modified)
+
+!update
+insert into foo values (1, 0);
+(1 row modified)
+
+!update
+insert into foo values (0, 2);
+(1 row modified)
+
+!update
+select * from foo;
++---+---+
+| I | J |
++---+---+
+| 0 | 2 |
+| 1 | 0 |
++---+---+
+(2 rows)
+
+!ok
+
+# End blank.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2551d53..b342535 100644
--- a/pom.xml
+++ b/pom.xml
@@ -65,7 +65,7 @@ limitations under the License.
     <fmpp-maven-plugin.version>1.0</fmpp-maven-plugin.version>
     <foodmart-data-hsqldb.version>0.3</foodmart-data-hsqldb.version>
     <foodmart-queries.version>0.4.1</foodmart-queries.version>
-    <freemarker.version>2.3.19</freemarker.version>
+    <freemarker.version>2.3.25-incubating</freemarker.version>
     <git-commit-id-plugin.version>2.1.9</git-commit-id-plugin.version>
 
     <!-- We support (and test against) Guava versions between

http://git-wip-us.apache.org/repos/asf/calcite/blob/e641211f/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 2aec223..6b4c4ab 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -606,6 +606,7 @@ READ,
 RELATIVE,
 **RELEASE**,
 REPEATABLE,
+REPLACE,
 **RESET**,
 RESTART,
 RESTRICT,