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 2020/06/01 17:16:52 UTC

[calcite] 03/06: Refactor DDL execution code out of SqlNode sub-classes and into DdlExecutor

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

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

commit c0f102d2013a8f4d2903ec259449f5bd668eec30
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat May 30 15:22:44 2020 -0700

    Refactor DDL execution code out of SqlNode sub-classes and into DdlExecutor
    
    In the previous commit we moved DDL classes from server to
    core, but in this commit we move the execution logic back to
    server.
    
    DdlExecutor is acquired via the parser factory. Not ideal, but keeps
    everything working and can be improved later.
    
    Remove duplicate package-info files, and suppress a checkstyle
    complaint that the org.apache.calcite.server package in the
    server module should have a package-info. (It already has one
    in the core module.)
---
 .../apache/calcite/prepare/CalcitePrepareImpl.java |  15 +-
 .../org/apache/calcite/server/DdlExecutor.java     |  23 +-
 .../org/apache/calcite/server/DdlExecutorImpl.java |  52 ++
 .../java/org/apache/calcite/sql/SqlCreate.java     |   2 +-
 .../main/java/org/apache/calcite/sql/SqlDrop.java  |   2 +-
 .../apache/calcite/sql/SqlExecutableStatement.java |   1 +
 .../calcite/sql/ddl/SqlAttributeDefinition.java    |   4 +-
 .../calcite/sql/ddl/SqlColumnDeclaration.java      |   8 +-
 .../calcite/sql/ddl/SqlCreateForeignSchema.java    |  87 +--
 .../apache/calcite/sql/ddl/SqlCreateFunction.java  |   9 +-
 .../calcite/sql/ddl/SqlCreateMaterializedView.java |  85 +--
 .../apache/calcite/sql/ddl/SqlCreateSchema.java    |  29 +-
 .../org/apache/calcite/sql/ddl/SqlCreateTable.java | 275 +---------
 .../org/apache/calcite/sql/ddl/SqlCreateType.java  |  37 +-
 .../org/apache/calcite/sql/ddl/SqlCreateView.java  |  49 +-
 .../org/apache/calcite/sql/ddl/SqlDdlNodes.java    | 102 ----
 .../calcite/sql/ddl/SqlDropMaterializedView.java   |  25 -
 .../org/apache/calcite/sql/ddl/SqlDropObject.java  |  50 +-
 .../org/apache/calcite/sql/ddl/SqlDropSchema.java  |  28 +-
 .../calcite/sql/parser/SqlParserImplFactory.java   |  23 +
 .../java/org/apache/calcite/util/ReflectUtil.java  |  12 +-
 .../parserextensiontesting/SqlCreateTable.java     | 180 +------
 .../ExtensionDdlExecutor.java}                     | 119 ++---
 .../java/org/apache/calcite/test/QuidemTest.java   |   3 +-
 .../apache/calcite/adapter/file/package-info.java  |   8 +-
 .../calcite/linq4j/function/package-info.java      |  21 -
 .../apache/calcite/linq4j/tree/package-info.java   |  21 -
 .../calcite/server/AbstractModifiableTable.java    |  50 ++
 .../calcite/server/MaterializedViewTable.java      |  47 ++
 .../apache/calcite/server/MutableArrayTable.java   | 100 ++++
 .../apache/calcite/server/ServerDdlExecutor.java   | 593 +++++++++++++++++++++
 .../java/org/apache/calcite/test/ServerTest.java   |  45 +-
 src/main/config/checkstyle/suppressions.xml        |   5 +-
 33 files changed, 1037 insertions(+), 1073 deletions(-)

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 4b270ca..e826ed6 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -80,8 +80,8 @@ import org.apache.calcite.runtime.Typed;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.server.DdlExecutor;
 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;
@@ -93,6 +93,7 @@ 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.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.type.ExtraSqlTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
@@ -147,7 +148,6 @@ public class CalcitePrepareImpl implements CalcitePrepare {
   public static final List<RelOptRule> ENUMERABLE_RULES =
       EnumerableRules.ENUMERABLE_RULES;
 
-
   /** Whether the bindable convention should be the root convention of any
    * plan. If not, enumerable convention is the default. */
   public final boolean enableBindable = Hook.ENABLE_BINDABLE.get(false);
@@ -357,12 +357,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();
+    final CalciteConnectionConfig config = context.config();
+    final SqlParserImplFactory parserFactory =
+        config.parserFactory(SqlParserImplFactory.class, SqlParserImpl.FACTORY);
+    final DdlExecutor ddlExecutor = parserFactory.getDdlExecutor();
+    ddlExecutor.executeDdl(context, node);
   }
 
   /** Factory method for default SQL parser. */
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/package-info.java b/core/src/main/java/org/apache/calcite/server/DdlExecutor.java
similarity index 57%
rename from file/src/test/java/org/apache/calcite/adapter/file/package-info.java
rename to core/src/main/java/org/apache/calcite/server/DdlExecutor.java
index 2a726e9..75f5194 100644
--- a/file/src/test/java/org/apache/calcite/adapter/file/package-info.java
+++ b/core/src/main/java/org/apache/calcite/server/DdlExecutor.java
@@ -14,12 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.calcite.server;
+
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.sql.SqlNode;
 
 /**
- * Query provider that reads from files and web pages in various formats.
- *
- * <p>A Calcite schema that maps onto multiple URLs / HTML Tables.  Each HTML
- * table appears as a table.  Full select SQL operations are available on those
- * tables.
+ * Executes DDL commands.
  */
-package org.apache.calcite.adapter.file;
+public interface DdlExecutor {
+  /** DDL executor that cannot handle any DDL. */
+  DdlExecutor USELESS = (context, node) -> {
+    throw new UnsupportedOperationException("DDL not supported: " + node);
+  };
+
+  /** Executes a DDL statement.
+   *
+   * <p>The statement identified itself as DDL in the
+   * {@link org.apache.calcite.jdbc.CalcitePrepare.ParseResult#kind} field. */
+  void executeDdl(CalcitePrepare.Context context, SqlNode node);
+}
diff --git a/core/src/main/java/org/apache/calcite/server/DdlExecutorImpl.java b/core/src/main/java/org/apache/calcite/server/DdlExecutorImpl.java
new file mode 100644
index 0000000..b5a8e2f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/server/DdlExecutorImpl.java
@@ -0,0 +1,52 @@
+/*
+ * 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.server;
+
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.ReflectUtil;
+import org.apache.calcite.util.ReflectiveVisitor;
+
+/** Abstract implementation of {@link org.apache.calcite.server.DdlExecutor}. */
+public class DdlExecutorImpl implements DdlExecutor, ReflectiveVisitor {
+  /** Creates a DdlExecutorImpl.
+   * Protected only to allow sub-classing;
+   * use a singleton instance where possible. */
+  protected DdlExecutorImpl() {
+  }
+
+  /** Dispatches calls to the appropriate method based on the type of the
+   * first argument. */
+  private final ReflectUtil.MethodDispatcher<Void> dispatcher =
+      ReflectUtil.createMethodDispatcher(void.class, this, "execute",
+          SqlNode.class, CalcitePrepare.Context.class);
+
+  @Override public void executeDdl(CalcitePrepare.Context context,
+      SqlNode node) {
+    dispatcher.invoke(node, context);
+  }
+
+  /** Template for methods that execute DDL commands.
+   *
+   * <p>The base implementation throws {@link UnsupportedOperationException}
+   * because a {@link SqlNode} is not DDL, but overloaded methods such as
+   * {@code public void execute(SqlCreateFoo, CalcitePrepare.Context)} are
+   * called via reflection. */
+  public void execute(SqlNode node, CalcitePrepare.Context context) {
+    throw new UnsupportedOperationException("DDL not supported: " + node);
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCreate.java b/core/src/main/java/org/apache/calcite/sql/SqlCreate.java
index 007e48d..a6d04e4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCreate.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCreate.java
@@ -29,7 +29,7 @@ public abstract class SqlCreate extends SqlDdl {
   boolean replace;
 
   /** Whether "IF NOT EXISTS" was specified. */
-  protected final boolean ifNotExists;
+  public final boolean ifNotExists;
 
   /** Creates a SqlCreate. */
   public SqlCreate(SqlOperator operator, SqlParserPos pos, boolean replace,
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDrop.java b/core/src/main/java/org/apache/calcite/sql/SqlDrop.java
index ffe1f36..ce8ebde 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDrop.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDrop.java
@@ -26,7 +26,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 public abstract class SqlDrop extends SqlDdl {
 
   /** Whether "IF EXISTS" was specified. */
-  protected final boolean ifExists;
+  public final boolean ifExists;
 
   /** Creates a SqlDrop. */
   public SqlDrop(SqlOperator operator, SqlParserPos pos, boolean ifExists) {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java b/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
index 05ac1c5..56cf8d9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
@@ -26,6 +26,7 @@ import org.apache.calcite.linq4j.function.Experimental;
  * <p>NOTE: Subject to change without notice.
  */
 @Experimental
+@Deprecated // to be removed before 1.25
 public interface SqlExecutableStatement {
   void execute(CalcitePrepare.Context context);
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlAttributeDefinition.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlAttributeDefinition.java
index 5e266a9..1d7c88b 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlAttributeDefinition.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlAttributeDefinition.java
@@ -39,8 +39,8 @@ public class SqlAttributeDefinition extends SqlCall {
   private static final SqlSpecialOperator OPERATOR =
       new SqlSpecialOperator("ATTRIBUTE_DEF", SqlKind.ATTRIBUTE_DEF);
 
-  final SqlIdentifier name;
-  final SqlDataTypeSpec dataType;
+  public final SqlIdentifier name;
+  public final SqlDataTypeSpec dataType;
   final SqlNode expression;
   final SqlCollation collation;
 
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java
index 7e0be32..83cb037 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java
@@ -40,10 +40,10 @@ public class SqlColumnDeclaration extends SqlCall {
   private static final SqlSpecialOperator OPERATOR =
       new SqlSpecialOperator("COLUMN_DECL", SqlKind.COLUMN_DECL);
 
-  final SqlIdentifier name;
-  final SqlDataTypeSpec dataType;
-  final SqlNode expression;
-  final ColumnStrategy strategy;
+  public final SqlIdentifier name;
+  public final SqlDataTypeSpec dataType;
+  public final SqlNode expression;
+  public final ColumnStrategy strategy;
 
   /** Creates a SqlColumnDeclaration; use {@link SqlDdlNodes#column}. */
   SqlColumnDeclaration(SqlParserPos pos, SqlIdentifier name,
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java
index 6f4dd1c..9c6b667 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java
@@ -16,51 +16,31 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.adapter.jdbc.JdbcSchema;
-import org.apache.calcite.avatica.AvaticaUtils;
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.model.JsonSchema;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaFactory;
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlCreate;
-import org.apache.calcite.sql.SqlExecutableStatement;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
 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.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 
 import java.util.AbstractList;
-import java.util.Arrays;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Locale;
-import java.util.Map;
 import java.util.Objects;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Parse tree for {@code CREATE FOREIGN SCHEMA} statement.
  */
-public class SqlCreateForeignSchema extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNode type;
-  private final SqlNode library;
+public class SqlCreateForeignSchema extends SqlCreate {
+  public final SqlIdentifier name;
+  public final SqlNode type;
+  public final SqlNode library;
   private final SqlNodeList optionList;
 
   private static final SqlOperator OPERATOR =
@@ -107,7 +87,7 @@ public class SqlCreateForeignSchema extends SqlCreate
       writer.keyword("OPTIONS");
       SqlWriter.Frame frame = writer.startList("(", ")");
       int i = 0;
-      for (Pair<SqlIdentifier, SqlNode> c : options(optionList)) {
+      for (Pair<SqlIdentifier, SqlNode> c : options()) {
         if (i++ > 0) {
           writer.sep(",");
         }
@@ -118,60 +98,9 @@ public class SqlCreateForeignSchema extends SqlCreate
     }
   }
 
-  public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    final SchemaPlus subSchema0 = pair.left.plus().getSubSchema(pair.right);
-    if (subSchema0 != null) {
-      if (!getReplace() && !ifNotExists) {
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.schemaExists(pair.right));
-      }
-    }
-    final Schema subSchema;
-    final String libraryName;
-    if (type != null) {
-      Preconditions.checkArgument(library == null);
-      final String typeName = (String) value(this.type);
-      final JsonSchema.Type type =
-          Util.enumVal(JsonSchema.Type.class,
-              typeName.toUpperCase(Locale.ROOT));
-      if (type != null) {
-        switch (type) {
-        case JDBC:
-          libraryName = JdbcSchema.Factory.class.getName();
-          break;
-        default:
-          libraryName = null;
-        }
-      } else {
-        libraryName = null;
-      }
-      if (libraryName == null) {
-        throw SqlUtil.newContextException(this.type.getParserPosition(),
-            RESOURCE.schemaInvalidType(typeName,
-                Arrays.toString(JsonSchema.Type.values())));
-      }
-    } else {
-      Preconditions.checkArgument(library != null);
-      libraryName = (String) value(library);
-    }
-    final SchemaFactory schemaFactory =
-        AvaticaUtils.instantiatePlugin(SchemaFactory.class, libraryName);
-    final Map<String, Object> operandMap = new LinkedHashMap<>();
-    for (Pair<SqlIdentifier, SqlNode> option : options(optionList)) {
-      operandMap.put(option.left.getSimple(), value(option.right));
-    }
-    subSchema =
-        schemaFactory.create(pair.left.plus(), pair.right, operandMap);
-    pair.left.add(pair.right, subSchema);
-  }
-
-  /** Returns the value of a literal, converting
-   * {@link NlsString} into String. */
-  private static Comparable value(SqlNode node) {
-    final Comparable v = SqlLiteral.value(node);
-    return v instanceof NlsString ? ((NlsString) v).getValue() : v;
+  /** Returns options as a list of (name, value) pairs. */
+  public List<Pair<SqlIdentifier, SqlNode>> options() {
+    return options(optionList);
   }
 
   private static List<Pair<SqlIdentifier, SqlNode>> options(
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java
index 627d0be..5e4929d 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java
@@ -16,9 +16,7 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
 import org.apache.calcite.sql.SqlCreate;
-import org.apache.calcite.sql.SqlExecutableStatement;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
@@ -40,8 +38,7 @@ import java.util.Objects;
 /**
  * Parse tree for {@code CREATE FUNCTION} statement.
  */
-public class SqlCreateFunction extends SqlCreate
-    implements SqlExecutableStatement {
+public class SqlCreateFunction extends SqlCreate {
   private final SqlIdentifier name;
   private final SqlNode className;
   private final SqlNodeList usingList;
@@ -83,10 +80,6 @@ public class SqlCreateFunction extends SqlCreate
     }
   }
 
-  @Override public void execute(CalcitePrepare.Context context) {
-    throw new UnsupportedOperationException("CREATE FUNCTION is not supported yet.");
-  }
-
   @SuppressWarnings("unchecked")
   private List<Pair<SqlLiteral, SqlLiteral>> pairs() {
     return Util.pairs((List) usingList.getList());
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java
index c653a0a..26fd933 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java
@@ -16,48 +16,27 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.materialize.MaterializationKey;
-import org.apache.calcite.materialize.MaterializationService;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeImpl;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.impl.ViewTable;
-import org.apache.calcite.schema.impl.ViewTableMacro;
 import org.apache.calcite.sql.SqlCreate;
-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.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
 import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.Pair;
-
-import com.google.common.collect.ImmutableList;
 
 import java.util.List;
 import java.util.Objects;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Parse tree for {@code CREATE MATERIALIZED VIEW} statement.
  */
-public class SqlCreateMaterializedView extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNodeList columnList;
-  private final SqlNode query;
+public class SqlCreateMaterializedView extends SqlCreate {
+  public final SqlIdentifier name;
+  public final SqlNodeList columnList;
+  public final SqlNode query;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE MATERIALIZED VIEW",
@@ -96,60 +75,4 @@ public class SqlCreateMaterializedView extends SqlCreate
     writer.newlineAndIndent();
     query.unparse(writer, 0, 0);
   }
-
-  public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    if (pair.left.plus().getTable(pair.right) != null) {
-      // Materialized view exists.
-      if (!ifNotExists) {
-        // They did not specify IF NOT EXISTS, so give error.
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.tableExists(pair.right));
-      }
-      return;
-    }
-    final SqlNode q = SqlDdlNodes.renameColumns(columnList, query);
-    final String sql = q.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
-    final List<String> schemaPath = pair.left.path(null);
-    final ViewTableMacro viewTableMacro =
-        ViewTable.viewMacro(pair.left.plus(), sql, schemaPath,
-            context.getObjectPath(), false);
-    final TranslatableTable x = viewTableMacro.apply(ImmutableList.of());
-    final RelDataType rowType = x.getRowType(context.getTypeFactory());
-
-    // Table does not exist. Create it.
-    final MaterializedViewTable table =
-        new MaterializedViewTable(pair.right, RelDataTypeImpl.proto(rowType));
-    pair.left.add(pair.right, table);
-    SqlDdlNodes.populate(name, query, context);
-    table.key =
-        MaterializationService.instance().defineMaterialization(pair.left, null,
-            sql, schemaPath, pair.right, true, true);
-  }
-
-  /** A table that implements a materialized view. */
-  private static class MaterializedViewTable
-      extends SqlCreateTable.MutableArrayTable {
-    /** The key with which this was stored in the materialization service,
-     * or null if not (yet) materialized. */
-    MaterializationKey key;
-
-    MaterializedViewTable(String name, RelProtoDataType protoRowType) {
-      super(name, protoRowType, protoRowType,
-          NullInitializerExpressionFactory.INSTANCE);
-    }
-
-    @Override public Schema.TableType getJdbcTableType() {
-      return Schema.TableType.MATERIALIZED_VIEW;
-    }
-
-    @Override public <C> C unwrap(Class<C> aClass) {
-      if (MaterializationKey.class.isAssignableFrom(aClass)
-          && aClass.isInstance(key)) {
-        return aClass.cast(key);
-      }
-      return super.unwrap(aClass);
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java
index 4175998..f7ab2e3 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java
@@ -16,35 +16,24 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.sql.SqlCreate;
-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.SqlOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.Pair;
 
 import java.util.List;
 import java.util.Objects;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Parse tree for {@code CREATE SCHEMA} statement.
  */
-public class SqlCreateSchema extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
+public class SqlCreateSchema extends SqlCreate {
+  public final SqlIdentifier name;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE SCHEMA", SqlKind.CREATE_SCHEMA);
@@ -72,18 +61,4 @@ public class SqlCreateSchema extends SqlCreate
     }
     name.unparse(writer, leftPrec, rightPrec);
   }
-
-  public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    final SchemaPlus subSchema0 = pair.left.plus().getSubSchema(pair.right);
-    if (subSchema0 != null) {
-      if (!getReplace() && !ifNotExists) {
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.schemaExists(pair.right));
-      }
-    }
-    final Schema subSchema = new AbstractSchema();
-    pair.left.add(pair.right, subSchema);
-  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java
index c0f2992..0773004 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java
@@ -16,75 +16,27 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.linq4j.Enumerator;
-import org.apache.calcite.linq4j.Linq4j;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.linq4j.Queryable;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeImpl;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.ColumnStrategy;
-import org.apache.calcite.schema.ModifiableTable;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.Wrapper;
-import org.apache.calcite.schema.impl.AbstractTable;
-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.sql.SqlCreate;
-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.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.sql2rel.InitializerContext;
-import org.apache.calcite.sql2rel.InitializerExpressionFactory;
-import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
 import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.Pair;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Parse tree for {@code CREATE TABLE} statement.
  */
-public class SqlCreateTable extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNodeList columnList;
-  private final SqlNode query;
+public class SqlCreateTable extends SqlCreate {
+  public final SqlIdentifier name;
+  public final SqlNodeList columnList;
+  public final SqlNode query;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
@@ -123,223 +75,4 @@ public class SqlCreateTable extends SqlCreate
       query.unparse(writer, 0, 0);
     }
   }
-
-  public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    final JavaTypeFactory typeFactory = context.getTypeFactory();
-    final RelDataType queryRowType;
-    if (query != null) {
-      // A bit of a hack: pretend it's a view, to get its row type
-      final String sql = query.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
-      final ViewTableMacro viewTableMacro =
-          ViewTable.viewMacro(pair.left.plus(), sql, pair.left.path(null),
-              context.getObjectPath(), false);
-      final TranslatableTable x = viewTableMacro.apply(ImmutableList.of());
-      queryRowType = x.getRowType(typeFactory);
-
-      if (columnList != null
-          && queryRowType.getFieldCount() != columnList.size()) {
-        throw SqlUtil.newContextException(columnList.getParserPosition(),
-            RESOURCE.columnCountMismatch());
-      }
-    } else {
-      queryRowType = null;
-    }
-    final List<SqlNode> columnList;
-    if (this.columnList != null) {
-      columnList = this.columnList.getList();
-    } 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(name.getParserPosition(),
-            RESOURCE.createTableRequiresColumnList());
-      }
-      columnList = new ArrayList<>();
-      for (String name : queryRowType.getFieldNames()) {
-        columnList.add(new SqlIdentifier(name, SqlParserPos.ZERO));
-      }
-    }
-    final ImmutableList.Builder<ColumnDef> b = ImmutableList.builder();
-    final RelDataTypeFactory.Builder builder = typeFactory.builder();
-    final RelDataTypeFactory.Builder storedBuilder = typeFactory.builder();
-    // REVIEW 2019-08-19 Danny Chan: Should we implement the
-    // #validate(SqlValidator) to get the SqlValidator instance?
-    final SqlValidator validator = SqlDdlNodes.validator(context, true);
-    for (Ord<SqlNode> c : Ord.zip(columnList)) {
-      if (c.e instanceof SqlColumnDeclaration) {
-        final SqlColumnDeclaration d = (SqlColumnDeclaration) c.e;
-        final RelDataType type = d.dataType.deriveType(validator, true);
-        builder.add(d.name.getSimple(), type);
-        if (d.strategy != ColumnStrategy.VIRTUAL) {
-          storedBuilder.add(d.name.getSimple(), type);
-        }
-        b.add(ColumnDef.of(d.expression, type, d.strategy));
-      } else if (c.e instanceof SqlIdentifier) {
-        final SqlIdentifier id = (SqlIdentifier) c.e;
-        if (queryRowType == null) {
-          throw SqlUtil.newContextException(id.getParserPosition(),
-              RESOURCE.createTableRequiresColumnTypes(id.getSimple()));
-        }
-        final RelDataTypeField f = queryRowType.getFieldList().get(c.i);
-        final ColumnStrategy strategy = f.getType().isNullable()
-            ? ColumnStrategy.NULLABLE
-            : ColumnStrategy.NOT_NULLABLE;
-        b.add(ColumnDef.of(c.e, f.getType(), strategy));
-        builder.add(id.getSimple(), f.getType());
-        storedBuilder.add(id.getSimple(), f.getType());
-      } else {
-        throw new AssertionError(c.e.getClass());
-      }
-    }
-    final RelDataType rowType = builder.build();
-    final RelDataType storedRowType = storedBuilder.build();
-    final List<ColumnDef> columns = b.build();
-    final InitializerExpressionFactory ief =
-        new NullInitializerExpressionFactory() {
-          @Override public ColumnStrategy generationStrategy(RelOptTable table,
-              int iColumn) {
-            return columns.get(iColumn).strategy;
-          }
-
-          @Override public RexNode newColumnDefaultValue(RelOptTable table,
-              int iColumn, InitializerContext context) {
-            final ColumnDef c = columns.get(iColumn);
-            if (c.expr != null) {
-              // REVIEW Danny 2019-10-09: Should we support validation for DDL nodes?
-              final SqlNode validated = context.validateExpression(storedRowType, c.expr);
-              // The explicit specified type should have the same nullability
-              // with the column expression inferred type,
-              // actually they should be exactly the same.
-              return context.convertExpression(validated);
-            }
-            return super.newColumnDefaultValue(table, iColumn, context);
-          }
-        };
-    if (pair.left.plus().getTable(pair.right) != null) {
-      // Table exists.
-      if (!ifNotExists) {
-        // They did not specify IF NOT EXISTS, so give error.
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.tableExists(pair.right));
-      }
-      return;
-    }
-    // Table does not exist. Create it.
-    pair.left.add(pair.right,
-        new MutableArrayTable(pair.right,
-            RelDataTypeImpl.proto(storedRowType),
-            RelDataTypeImpl.proto(rowType), ief));
-    if (query != null) {
-      SqlDdlNodes.populate(name, query, context);
-    }
-  }
-
-  /** Column definition. */
-  private static class ColumnDef {
-    final SqlNode expr;
-    final RelDataType type;
-    final ColumnStrategy strategy;
-
-    private ColumnDef(SqlNode expr, RelDataType type,
-        ColumnStrategy strategy) {
-      this.expr = expr;
-      this.type = type;
-      this.strategy = Objects.requireNonNull(strategy);
-      Preconditions.checkArgument(
-          strategy == ColumnStrategy.NULLABLE
-              || strategy == ColumnStrategy.NOT_NULLABLE
-              || expr != null);
-    }
-
-    static ColumnDef of(SqlNode expr, RelDataType type,
-        ColumnStrategy strategy) {
-      return new ColumnDef(expr, type, strategy);
-    }
-  }
-
-  /** Abstract base class for implementations of {@link ModifiableTable}. */
-  abstract static class AbstractModifiableTable
-      extends AbstractTable implements ModifiableTable {
-    AbstractModifiableTable(String tableName) {
-      super();
-    }
-
-    public TableModify toModificationRel(
-        RelOptCluster cluster,
-        RelOptTable table,
-        Prepare.CatalogReader catalogReader,
-        RelNode child,
-        TableModify.Operation operation,
-        List<String> updateColumnList,
-        List<RexNode> sourceExpressionList,
-        boolean flattened) {
-      return LogicalTableModify.create(table, catalogReader, child, operation,
-          updateColumnList, sourceExpressionList, flattened);
-    }
-  }
-
-  /** Table backed by a Java list. */
-  static class MutableArrayTable extends AbstractModifiableTable
-      implements Wrapper {
-    final List rows = new ArrayList();
-    private final RelProtoDataType protoStoredRowType;
-    private final RelProtoDataType protoRowType;
-    private final InitializerExpressionFactory initializerExpressionFactory;
-
-    /** Creates a MutableArrayTable.
-     *
-     * @param name Name of table within its schema
-     * @param protoStoredRowType Prototype of row type of stored columns (all
-     *     columns except virtual columns)
-     * @param protoRowType Prototype of row type (all columns)
-     * @param initializerExpressionFactory How columns are populated
-     */
-    MutableArrayTable(String name, RelProtoDataType protoStoredRowType,
-        RelProtoDataType protoRowType,
-        InitializerExpressionFactory initializerExpressionFactory) {
-      super(name);
-      this.protoStoredRowType = Objects.requireNonNull(protoStoredRowType);
-      this.protoRowType = Objects.requireNonNull(protoRowType);
-      this.initializerExpressionFactory =
-          Objects.requireNonNull(initializerExpressionFactory);
-    }
-
-    public Collection getModifiableCollection() {
-      return rows;
-    }
-
-    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(rows);
-        }
-      };
-    }
-
-    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 public <C> C unwrap(Class<C> aClass) {
-      if (aClass.isInstance(initializerExpressionFactory)) {
-        return aClass.cast(initializerExpressionFactory);
-      }
-      return super.unwrap(aClass);
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateType.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateType.java
index b34a100..40e0689 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateType.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateType.java
@@ -16,13 +16,8 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 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;
@@ -31,9 +26,7 @@ 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.sql.validate.SqlValidator;
 import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.Pair;
 
 import java.util.List;
 import java.util.Objects;
@@ -41,11 +34,10 @@ import java.util.Objects;
 /**
  * Parse tree for {@code CREATE TYPE} statement.
  */
-public class SqlCreateType extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNodeList attributeDefs;
-  private final SqlDataTypeSpec dataType;
+public class SqlCreateType extends SqlCreate {
+  public final SqlIdentifier name;
+  public final SqlNodeList attributeDefs;
+  public final SqlDataTypeSpec dataType;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE TYPE", SqlKind.CREATE_TYPE);
@@ -59,27 +51,6 @@ public class SqlCreateType extends SqlCreate
     this.dataType = dataType; // may be null
   }
 
-  @Override public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    final SqlValidator validator = SqlDdlNodes.validator(context, false);
-    pair.left.add(pair.right, typeFactory -> {
-      if (dataType != null) {
-        return dataType.deriveType(validator);
-      } else {
-        final RelDataTypeFactory.Builder builder = typeFactory.builder();
-        for (SqlNode def : attributeDefs) {
-          final SqlAttributeDefinition attributeDef =
-              (SqlAttributeDefinition) def;
-          final SqlDataTypeSpec typeSpec = attributeDef.dataType;
-          final RelDataType type = typeSpec.deriveType(validator);
-          builder.add(attributeDef.name.getSimple(), type);
-        }
-        return builder.build();
-      }
-    });
-  }
-
   @Override public List<SqlNode> getOperandList() {
     return ImmutableNullableList.of(name, attributeDefs);
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java
index cbd286a..ab05ecf 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java
@@ -16,44 +16,27 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.impl.ViewTable;
-import org.apache.calcite.schema.impl.ViewTableMacro;
 import org.apache.calcite.sql.SqlCreate;
-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.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 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 com.google.common.collect.ImmutableList;
 
 import java.util.List;
 import java.util.Objects;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Parse tree for {@code CREATE VIEW} statement.
  */
-public class SqlCreateView extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNodeList columnList;
-  private final SqlNode query;
+public class SqlCreateView extends SqlCreate {
+  public final SqlIdentifier name;
+  public final SqlNodeList columnList;
+  public final SqlNode query;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE VIEW", SqlKind.CREATE_VIEW);
@@ -91,28 +74,4 @@ public class SqlCreateView extends SqlCreate
     writer.newlineAndIndent();
     query.unparse(writer, 0, 0);
   }
-
-  public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    final SchemaPlus schemaPlus = pair.left.plus();
-    for (Function function : schemaPlus.getFunctions(pair.right)) {
-      if (function.getParameters().isEmpty()) {
-        if (!getReplace()) {
-          throw SqlUtil.newContextException(name.getParserPosition(),
-              RESOURCE.viewExists(pair.right));
-        }
-        pair.left.removeFunction(pair.right);
-      }
-    }
-    final SqlNode q = SqlDdlNodes.renameColumns(columnList, 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);
-  }
-
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
index d5ac495..421e389 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
@@ -16,12 +16,7 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.ContextSqlValidator;
-import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.schema.ColumnStrategy;
-import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDrop;
@@ -29,26 +24,9 @@ import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.SqlWriterConfig;
-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.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 java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.List;
 
 /**
  * Utilities concerning {@link SqlNode} for DDL.
@@ -177,86 +155,6 @@ public class SqlDdlNodes {
     };
   }
 
-  /** 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 = schema.getSubSchema(p, true);
-    }
-    return Pair.of(schema, name);
-  }
-
-  /**
-   * Returns the SqlValidator with the given {@code context} schema
-   * and type factory.
-   * */
-  static SqlValidator validator(CalcitePrepare.Context context, boolean mutable) {
-    return new ContextSqlValidator(context, mutable);
-  }
-
-  /** Wraps a query to rename its columns. Used by CREATE VIEW and CREATE
-   * MATERIALIZED VIEW. */
-  static SqlNode renameColumns(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);
-  }
-
-  /** 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(context.getRootSchema().plus())
-        .build();
-    final Planner planner = Frameworks.getPlanner(config);
-    try {
-      final StringBuilder buf = new StringBuilder();
-      final SqlWriterConfig writerConfig =
-          SqlPrettyWriter.config().withAlwaysUseParentheses(false);
-      final SqlPrettyWriter w = new SqlPrettyWriter(writerConfig, 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().prepare(r.rel);
-      int rowCount = prepare.executeUpdate();
-      Util.discard(rowCount);
-      prepare.close();
-    } catch (SqlParseException | ValidationException
-        | RelConversionException | SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
   /** File type for CREATE FUNCTION. */
   public enum FileType {
     FILE,
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java
index 547757e..3741674 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java
@@ -16,18 +16,11 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.materialize.MaterializationKey;
-import org.apache.calcite.materialize.MaterializationService;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.Wrapper;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.Pair;
 
 /**
  * Parse tree for {@code DROP MATERIALIZED VIEW} statement.
@@ -42,22 +35,4 @@ public class SqlDropMaterializedView extends SqlDropObject {
       SqlIdentifier name) {
     super(OPERATOR, pos, ifExists, name);
   }
-
-  @Override public void execute(CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair =
-        SqlDdlNodes.schema(context, true, name);
-    final Table table = pair.left.plus().getTable(pair.right);
-    if (table != null) {
-      // Materialized view exists.
-      super.execute(context);
-      if (table instanceof Wrapper) {
-        final MaterializationKey materializationKey =
-            ((Wrapper) table).unwrap(MaterializationKey.class);
-        if (materializationKey != null) {
-          MaterializationService.instance()
-              .removeMaterialization(materializationKey);
-        }
-      }
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
index c459d44..8ca5585 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
@@ -17,13 +17,10 @@
 package org.apache.calcite.sql.ddl;
 
 import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.sql.SqlDrop;
-import org.apache.calcite.sql.SqlExecutableStatement;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 
@@ -31,15 +28,13 @@ import com.google.common.collect.ImmutableList;
 
 import java.util.List;
 
-import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Base class for parse trees of {@code DROP TABLE}, {@code DROP VIEW},
  * {@code DROP MATERIALIZED VIEW} and {@code DROP TYPE} statements.
  */
-abstract class SqlDropObject extends SqlDrop
-    implements SqlExecutableStatement {
-  protected final SqlIdentifier name;
+public abstract class SqlDropObject extends SqlDrop {
+  public final SqlIdentifier name;
 
   /** Creates a SqlDropObject. */
   SqlDropObject(SqlOperator operator, SqlParserPos pos, boolean ifExists,
@@ -61,46 +56,5 @@ abstract class SqlDropObject extends SqlDrop
   }
 
   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 boolean existed;
-    switch (getKind()) {
-    case DROP_TABLE:
-    case DROP_MATERIALIZED_VIEW:
-      existed = schema.removeTable(name.getSimple());
-      if (!existed && !ifExists) {
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.tableNotFound(name.getSimple()));
-      }
-      break;
-    case DROP_VIEW:
-      // Not quite right: removes any other functions with the same name
-      existed = schema.removeFunction(name.getSimple());
-      if (!existed && !ifExists) {
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.viewNotFound(name.getSimple()));
-      }
-      break;
-    case DROP_TYPE:
-      existed = schema.removeType(name.getSimple());
-      if (!existed && !ifExists) {
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.typeNotFound(name.getSimple()));
-      }
-      break;
-    case DROP_FUNCTION:
-      existed = schema.removeFunction(name.getSimple());
-      if (!existed && !ifExists) {
-        throw SqlUtil.newContextException(name.getParserPosition(),
-            RESOURCE.functionNotFound(name.getSimple()));
-      }
-      break;
-    case OTHER_DDL:
-    default:
-      throw new AssertionError(getKind());
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java
index 877e7a3..73c9493 100644
--- a/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java
+++ b/core/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java
@@ -16,17 +16,13 @@
  */
 package org.apache.calcite.sql.ddl;
 
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.sql.SqlDrop;
-import org.apache.calcite.sql.SqlExecutableStatement;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 
@@ -34,18 +30,15 @@ import com.google.common.collect.ImmutableList;
 
 import java.util.List;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
- * Parse tree for {@code DROP TABLE} statement.
+ * Parse tree for {@code DROP SCHEMA} statement.
  */
-public class SqlDropSchema extends SqlDrop
-    implements SqlExecutableStatement {
+public class SqlDropSchema extends SqlDrop {
   private final boolean foreign;
-  private final SqlIdentifier name;
+  public final SqlIdentifier name;
 
   private static final SqlOperator OPERATOR =
-      new SqlSpecialOperator("DROP SCHEMA", SqlKind.DROP_TABLE);
+      new SqlSpecialOperator("DROP SCHEMA", SqlKind.DROP_SCHEMA);
 
   /** Creates a SqlDropSchema. */
   SqlDropSchema(SqlParserPos pos, boolean foreign, boolean ifExists,
@@ -71,17 +64,4 @@ public class SqlDropSchema extends SqlDrop
     }
     name.unparse(writer, leftPrec, rightPrec);
   }
-
-  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 boolean existed = schema.removeSubSchema(name.getSimple());
-    if (!existed && !ifExists) {
-      throw SqlUtil.newContextException(name.getParserPosition(),
-          RESOURCE.schemaNotFound(name.getSimple()));
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
index 3d6d348..4ad1d33 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
@@ -16,6 +16,9 @@
  */
 package org.apache.calcite.sql.parser;
 
+import org.apache.calcite.linq4j.function.Experimental;
+import org.apache.calcite.server.DdlExecutor;
+
 import java.io.Reader;
 
 /**
@@ -26,6 +29,7 @@ import java.io.Reader;
  * {@link org.apache.calcite.tools.Planner} created through
  * {@link org.apache.calcite.tools.Frameworks}.</p>
  */
+@FunctionalInterface
 public interface SqlParserImplFactory {
 
   /**
@@ -34,4 +38,23 @@ public interface SqlParserImplFactory {
    * @return {@link SqlAbstractParserImpl} object.
    */
   SqlAbstractParserImpl getParser(Reader stream);
+
+  /**
+   * Returns a DDL executor.
+   *
+   * <p>The default implementation returns {@link DdlExecutor#USELESS},
+   * which cannot handle any DDL commands.
+   *
+   * <p>DDL execution is related to parsing but it is admittedly a stretch to
+   * control them in the same factory. Therefore this is marked 'experimental'.
+   * We are bundling them because they are often overridden at the same time. In
+   * particular, we want a way to refine the behavior of the "server" module,
+   * which supports DDL parsing and execution, and we're not yet ready to define
+   * a new {@link java.sql.Driver} or
+   * {@link org.apache.calcite.server.CalciteServer}.
+   */
+  @Experimental
+  default DdlExecutor getDdlExecutor() {
+    return DdlExecutor.USELESS;
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/util/ReflectUtil.java b/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
index 1d9a703..5c3e649 100644
--- a/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
@@ -523,7 +523,17 @@ public abstract class ReflectUtil {
         try {
           final Object o = method.invoke(visitor, args);
           return returnClazz.cast(o);
-        } catch (IllegalAccessException | InvocationTargetException e) {
+        } catch (IllegalAccessException e) {
+          throw new RuntimeException("While invoking method '" + method + "'",
+              e);
+        } catch (InvocationTargetException e) {
+          final Throwable target = e.getTargetException();
+          if (target instanceof RuntimeException) {
+            throw (RuntimeException) target;
+          }
+          if (target instanceof Error) {
+            throw (Error) target;
+          }
           throw new RuntimeException("While invoking method '" + method + "'",
               e);
         }
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
index 9717af0..176f05a 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/SqlCreateTable.java
@@ -16,73 +16,31 @@
  */
 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.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.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.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.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.dialect.CalciteSqlDialect;
-import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.pretty.SqlPrettyWriter;
-import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.test.JdbcTest;
-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.ImmutableNullableList;
 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.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
 import java.util.function.BiConsumer;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Simple test example of a CREATE TABLE statement.
  */
-public class SqlCreateTable extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNodeList columnList;
-  private final SqlNode query;
+public class SqlCreateTable extends SqlCreate {
+  public final SqlIdentifier name;
+  public final SqlNodeList columnList;
+  public final SqlNode query;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
@@ -106,7 +64,7 @@ public class SqlCreateTable extends SqlCreate
     name.unparse(writer, leftPrec, rightPrec);
     if (columnList != null) {
       SqlWriter.Frame frame = writer.startList("(", ")");
-      nameTypes((name, typeSpec) -> {
+      forEachNameType((name, typeSpec) -> {
         writer.sep(",");
         name.unparse(writer, leftPrec, rightPrec);
         typeSpec.unparse(writer, leftPrec, rightPrec);
@@ -126,135 +84,9 @@ public class SqlCreateTable extends SqlCreate
   /** Calls an action for each (name, type) pair from {@code columnList}, in which
    * they alternate. */
   @SuppressWarnings({"unchecked"})
-  private void nameTypes(BiConsumer<SqlIdentifier, SqlDataTypeSpec> consumer) {
+  public void forEachNameType(BiConsumer<SqlIdentifier, SqlDataTypeSpec> consumer) {
     final List list = columnList.getList();
     Pair.forEach((List<SqlIdentifier>) Util.quotientList(list, 2, 0),
         Util.quotientList((List<SqlDataTypeSpec>) list, 2, 1), consumer);
   }
-
-  public void execute(CalcitePrepare.Context context) {
-    final CalciteSchema schema =
-        Schemas.subSchema(context.getRootSchema(),
-            context.getDefaultSchemaPath());
-    final JavaTypeFactory typeFactory = context.getTypeFactory();
-    final RelDataType queryRowType;
-    if (query != null) {
-      // A bit of a hack: pretend it's a view, to get its row type
-      final String sql = 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 (columnList != null
-          && queryRowType.getFieldCount() != columnList.size()) {
-        throw SqlUtil.newContextException(columnList.getParserPosition(),
-            RESOURCE.columnCountMismatch());
-      }
-    } else {
-      queryRowType = null;
-    }
-    final RelDataTypeFactory.Builder builder = typeFactory.builder();
-    if (columnList != null) {
-      final SqlValidator validator = new ContextSqlValidator(context, false);
-      nameTypes((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(name.getParserPosition(),
-            RESOURCE.createTableRequiresColumnList());
-      }
-      builder.addAll(queryRowType.getFieldList());
-    }
-    final RelDataType rowType = builder.build();
-    schema.add(name.getSimple(),
-        new MutableArrayTable(name.getSimple(),
-            RelDataTypeImpl.proto(rowType)));
-    if (query != null) {
-      populate(name, query, context);
-    }
-  }
-
-  /** 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(
-            Objects.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().prepare(r.rel);
-      int rowCount = prepare.executeUpdate();
-      Util.discard(rowCount);
-      prepare.close();
-    } catch (SqlParseException | ValidationException
-        | RelConversionException | SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** 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);
-    }
-  }
 }
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java b/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
similarity index 68%
copy from core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
copy to core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
index 9717af0..7170c6e 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
+++ b/core/src/test/java/org/apache/calcite/test/ExtensionDdlExecutor.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.sql.parser.parserextensiontesting;
+package org.apache.calcite.test;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.jdbc.CalcitePrepare;
@@ -36,34 +36,29 @@ 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.sql.SqlCreate;
-import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.SqlExecutableStatement;
+import org.apache.calcite.server.DdlExecutor;
+import org.apache.calcite.server.DdlExecutorImpl;
 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.SqlUtil;
-import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
 import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+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.test.JdbcTest;
 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.ImmutableNullableList;
-import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.Reader;
 import java.lang.reflect.Type;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
@@ -71,110 +66,72 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
-import java.util.function.BiConsumer;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
-/**
- * Simple test example of a CREATE TABLE statement.
- */
-public class SqlCreateTable extends SqlCreate
-    implements SqlExecutableStatement {
-  private final SqlIdentifier name;
-  private final SqlNodeList columnList;
-  private final SqlNode query;
-
-  private static final SqlOperator OPERATOR =
-      new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
-
-  /** Creates a SqlCreateTable. */
-  public SqlCreateTable(SqlParserPos pos, SqlIdentifier name,
-      SqlNodeList columnList, SqlNode query) {
-    super(OPERATOR, pos, false, false);
-    this.name = Objects.requireNonNull(name);
-    this.columnList = columnList; // may be null
-    this.query = query; // for "CREATE TABLE ... AS query"; may be null
-  }
-
-  @Override public List<SqlNode> getOperandList() {
-    return ImmutableNullableList.of(name, columnList, query);
-  }
-
-  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-    writer.keyword("CREATE");
-    writer.keyword("TABLE");
-    name.unparse(writer, leftPrec, rightPrec);
-    if (columnList != null) {
-      SqlWriter.Frame frame = writer.startList("(", ")");
-      nameTypes((name, typeSpec) -> {
-        writer.sep(",");
-        name.unparse(writer, leftPrec, rightPrec);
-        typeSpec.unparse(writer, leftPrec, rightPrec);
-        if (Boolean.FALSE.equals(typeSpec.getNullable())) {
-          writer.keyword("NOT NULL");
+/** 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);
         }
-      });
-      writer.endList(frame);
-    }
-    if (query != null) {
-      writer.keyword("AS");
-      writer.newlineAndIndent();
-      query.unparse(writer, 0, 0);
-    }
-  }
 
-  /** Calls an action for each (name, type) pair from {@code columnList}, in which
-   * they alternate. */
-  @SuppressWarnings({"unchecked"})
-  private void nameTypes(BiConsumer<SqlIdentifier, SqlDataTypeSpec> consumer) {
-    final List list = columnList.getList();
-    Pair.forEach((List<SqlIdentifier>) Util.quotientList(list, 2, 0),
-        Util.quotientList((List<SqlDataTypeSpec>) list, 2, 1), consumer);
-  }
+        @Override public DdlExecutor getDdlExecutor() {
+          return ExtensionDdlExecutor.INSTANCE;
+        }
+      };
 
-  public void execute(CalcitePrepare.Context context) {
+  /** 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 (query != null) {
+    if (create.query != null) {
       // A bit of a hack: pretend it's a view, to get its row type
-      final String sql = query.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
+      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 (columnList != null
-          && queryRowType.getFieldCount() != columnList.size()) {
-        throw SqlUtil.newContextException(columnList.getParserPosition(),
+      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 (columnList != null) {
+    if (create.columnList != null) {
       final SqlValidator validator = new ContextSqlValidator(context, false);
-      nameTypes((name, typeSpec) ->
+      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(name.getParserPosition(),
+        throw SqlUtil.newContextException(create.name.getParserPosition(),
             RESOURCE.createTableRequiresColumnList());
       }
       builder.addAll(queryRowType.getFieldList());
     }
     final RelDataType rowType = builder.build();
-    schema.add(name.getSimple(),
-        new MutableArrayTable(name.getSimple(),
+    schema.add(create.name.getSimple(),
+        new MutableArrayTable(create.name.getSimple(),
             RelDataTypeImpl.proto(rowType)));
-    if (query != null) {
-      populate(name, query, context);
+    if (create.query != null) {
+      populate(create.name, create.query, context);
     }
   }
 
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 800173d..7284fd1 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -274,8 +274,7 @@ public abstract class QuidemTest {
       case "blank":
         return CalciteAssert.that()
             .with(CalciteConnectionProperty.PARSER_FACTORY,
-                "org.apache.calcite.sql.parser.parserextensiontesting"
-                    + ".ExtensionSqlParserImpl#FACTORY")
+                ExtensionDdlExecutor.class.getName() + "#PARSER_FACTORY")
             .with(CalciteAssert.SchemaSpec.BLANK)
             .connect();
       case "seq":
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/package-info.java b/file/src/main/java/org/apache/calcite/adapter/file/package-info.java
index 02c0246..2a726e9 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/package-info.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/package-info.java
@@ -16,10 +16,10 @@
  */
 
 /**
- * Calcite query provider that reads from web tables (HTML).
+ * Query provider that reads from files and web pages in various formats.
  *
- * <p>A Calcite schema that maps onto multiple URLs / HTML Tables.
- * Each HTML table appears as a table.
- * Full select SQL operations are available on those tables.
+ * <p>A Calcite schema that maps onto multiple URLs / HTML Tables.  Each HTML
+ * table appears as a table.  Full select SQL operations are available on those
+ * tables.
  */
 package org.apache.calcite.adapter.file;
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java b/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java
deleted file mode 100644
index 55a36d3..0000000
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Tests for functions.
- */
-package org.apache.calcite.linq4j.function;
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java b/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
deleted file mode 100644
index 1e0cd4f..0000000
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Tests for expressions.
- */
-package org.apache.calcite.linq4j.tree;
diff --git a/server/src/main/java/org/apache/calcite/server/AbstractModifiableTable.java b/server/src/main/java/org/apache/calcite/server/AbstractModifiableTable.java
new file mode 100644
index 0000000..a2ce131
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/server/AbstractModifiableTable.java
@@ -0,0 +1,50 @@
+/*
+ * 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.server;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+
+import java.util.List;
+
+/** Abstract base class for implementations of {@link ModifiableTable}. */
+abstract class AbstractModifiableTable
+    extends AbstractTable implements ModifiableTable {
+  AbstractModifiableTable(String tableName) {
+    super();
+  }
+
+  @Override public TableModify toModificationRel(
+      RelOptCluster cluster,
+      RelOptTable table,
+      Prepare.CatalogReader catalogReader,
+      RelNode child,
+      TableModify.Operation operation,
+      List<String> updateColumnList,
+      List<RexNode> sourceExpressionList,
+      boolean flattened) {
+    return LogicalTableModify.create(table, catalogReader, child, operation,
+        updateColumnList, sourceExpressionList, flattened);
+  }
+}
diff --git a/server/src/main/java/org/apache/calcite/server/MaterializedViewTable.java b/server/src/main/java/org/apache/calcite/server/MaterializedViewTable.java
new file mode 100644
index 0000000..46b1732
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/server/MaterializedViewTable.java
@@ -0,0 +1,47 @@
+/*
+ * 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.server;
+
+import org.apache.calcite.materialize.MaterializationKey;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
+
+/** A table that implements a materialized view. */
+class MaterializedViewTable
+    extends MutableArrayTable {
+  /** The key with which this was stored in the materialization service,
+   * or null if not (yet) materialized. */
+  MaterializationKey key;
+
+  MaterializedViewTable(String name, RelProtoDataType protoRowType) {
+    super(name, protoRowType, protoRowType,
+        NullInitializerExpressionFactory.INSTANCE);
+  }
+
+  @Override public Schema.TableType getJdbcTableType() {
+    return Schema.TableType.MATERIALIZED_VIEW;
+  }
+
+  @Override public <C> C unwrap(Class<C> aClass) {
+    if (MaterializationKey.class.isAssignableFrom(aClass)
+        && aClass.isInstance(key)) {
+      return aClass.cast(key);
+    }
+    return super.unwrap(aClass);
+  }
+}
diff --git a/server/src/main/java/org/apache/calcite/server/MutableArrayTable.java b/server/src/main/java/org/apache/calcite/server/MutableArrayTable.java
new file mode 100644
index 0000000..f450edc
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/server/MutableArrayTable.java
@@ -0,0 +1,100 @@
+/*
+ * 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.server;
+
+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.RelProtoDataType;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Wrapper;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql2rel.InitializerExpressionFactory;
+
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+
+/** Table backed by a Java list. */
+class MutableArrayTable extends AbstractModifiableTable
+    implements Wrapper {
+  final List rows = new ArrayList();
+  private final RelProtoDataType protoStoredRowType;
+  private final RelProtoDataType protoRowType;
+  private final InitializerExpressionFactory initializerExpressionFactory;
+
+  /** Creates a MutableArrayTable.
+   *
+   * @param name Name of table within its schema
+   * @param protoStoredRowType Prototype of row type of stored columns (all
+   *     columns except virtual columns)
+   * @param protoRowType Prototype of row type (all columns)
+   * @param initializerExpressionFactory How columns are populated
+   */
+  MutableArrayTable(String name, RelProtoDataType protoStoredRowType,
+      RelProtoDataType protoRowType,
+      InitializerExpressionFactory initializerExpressionFactory) {
+    super(name);
+    this.protoStoredRowType = Objects.requireNonNull(protoStoredRowType);
+    this.protoRowType = Objects.requireNonNull(protoRowType);
+    this.initializerExpressionFactory =
+        Objects.requireNonNull(initializerExpressionFactory);
+  }
+
+  public Collection getModifiableCollection() {
+    return rows;
+  }
+
+  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(rows);
+      }
+    };
+  }
+
+  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 public <C> C unwrap(Class<C> aClass) {
+    if (aClass.isInstance(initializerExpressionFactory)) {
+      return aClass.cast(initializerExpressionFactory);
+    }
+    return super.unwrap(aClass);
+  }
+}
diff --git a/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java b/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java
new file mode 100644
index 0000000..78f39b1
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java
@@ -0,0 +1,593 @@
+/*
+ * 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.server;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.avatica.AvaticaUtils;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.ContextSqlValidator;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.materialize.MaterializationKey;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.model.JsonSchema;
+import org.apache.calcite.plan.RelOptTable;
+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.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.ColumnStrategy;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.Wrapper;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.schema.impl.ViewTableMacro;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+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.SqlWriterConfig;
+import org.apache.calcite.sql.ddl.SqlAttributeDefinition;
+import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
+import org.apache.calcite.sql.ddl.SqlCreateForeignSchema;
+import org.apache.calcite.sql.ddl.SqlCreateFunction;
+import org.apache.calcite.sql.ddl.SqlCreateMaterializedView;
+import org.apache.calcite.sql.ddl.SqlCreateSchema;
+import org.apache.calcite.sql.ddl.SqlCreateTable;
+import org.apache.calcite.sql.ddl.SqlCreateType;
+import org.apache.calcite.sql.ddl.SqlCreateView;
+import org.apache.calcite.sql.ddl.SqlDropMaterializedView;
+import org.apache.calcite.sql.ddl.SqlDropObject;
+import org.apache.calcite.sql.ddl.SqlDropSchema;
+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.ddl.SqlDdlParserImpl;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql2rel.InitializerContext;
+import org.apache.calcite.sql2rel.InitializerExpressionFactory;
+import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
+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.NlsString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.io.Reader;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/** Executes DDL commands.
+ *
+ * <p>Given a DDL command that is a sub-class of {@link SqlNode}, dispatches
+ * the command to an appropriate {@code execute} method. For example,
+ * "CREATE TABLE" ({@link SqlCreateTable}) is dispatched to
+ * {@link #execute(SqlCreateTable, CalcitePrepare.Context)}. */
+public class ServerDdlExecutor extends DdlExecutorImpl {
+  /** Singleton instance. */
+  public static final ServerDdlExecutor INSTANCE = new ServerDdlExecutor();
+
+  /** Parser factory. */
+  @SuppressWarnings("unused") // used via reflection
+  public static final SqlParserImplFactory PARSER_FACTORY =
+      new SqlParserImplFactory() {
+        @Override public SqlAbstractParserImpl getParser(Reader stream) {
+          return SqlDdlParserImpl.FACTORY.getParser(stream);
+        }
+
+        @Override public DdlExecutor getDdlExecutor() {
+          return ServerDdlExecutor.INSTANCE;
+        }
+      };
+
+  /** Creates a ServerDdlExecutor.
+   * Protected only to allow sub-classing;
+   * use {@link #INSTANCE} where possible. */
+  protected ServerDdlExecutor() {
+  }
+
+  /** 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 = schema.getSubSchema(p, true);
+    }
+    return Pair.of(schema, name);
+  }
+
+  /**
+   * Returns the SqlValidator with the given {@code context} schema
+   * and type factory.
+   */
+  static SqlValidator validator(CalcitePrepare.Context context,
+      boolean mutable) {
+    return new ContextSqlValidator(context, mutable);
+  }
+
+  /** Wraps a query to rename its columns. Used by CREATE VIEW and CREATE
+   * MATERIALIZED VIEW. */
+  static SqlNode renameColumns(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);
+  }
+
+  /** Populates the table called {@code name} by executing {@code query}. */
+  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(context.getRootSchema().plus())
+        .build();
+    final Planner planner = Frameworks.getPlanner(config);
+    try {
+      final StringBuilder buf = new StringBuilder();
+      final SqlWriterConfig writerConfig =
+          SqlPrettyWriter.config().withAlwaysUseParentheses(false);
+      final SqlPrettyWriter w = new SqlPrettyWriter(writerConfig, 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().prepare(r.rel);
+      int rowCount = prepare.executeUpdate();
+      Util.discard(rowCount);
+      prepare.close();
+    } catch (SqlParseException | ValidationException
+        | RelConversionException | SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Returns the value of a literal, converting
+   * {@link NlsString} into String. */
+  static Comparable value(SqlNode node) {
+    final Comparable v = SqlLiteral.value(node);
+    return v instanceof NlsString ? ((NlsString) v).getValue() : v;
+  }
+
+  /** Executes a {@code CREATE FOREIGN SCHEMA} command. */
+  public void execute(SqlCreateForeignSchema create,
+      CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair =
+        schema(context, true, create.name);
+    final SchemaPlus subSchema0 = pair.left.plus().getSubSchema(pair.right);
+    if (subSchema0 != null) {
+      if (!create.getReplace() && !create.ifNotExists) {
+        throw SqlUtil.newContextException(create.name.getParserPosition(),
+            RESOURCE.schemaExists(pair.right));
+      }
+    }
+    final Schema subSchema;
+    final String libraryName;
+    if (create.type != null) {
+      Preconditions.checkArgument(create.library == null);
+      final String typeName = (String) value(create.type);
+      final JsonSchema.Type type =
+          Util.enumVal(JsonSchema.Type.class,
+              typeName.toUpperCase(Locale.ROOT));
+      if (type != null) {
+        switch (type) {
+        case JDBC:
+          libraryName = JdbcSchema.Factory.class.getName();
+          break;
+        default:
+          libraryName = null;
+        }
+      } else {
+        libraryName = null;
+      }
+      if (libraryName == null) {
+        throw SqlUtil.newContextException(create.type.getParserPosition(),
+            RESOURCE.schemaInvalidType(typeName,
+                Arrays.toString(JsonSchema.Type.values())));
+      }
+    } else {
+      Preconditions.checkArgument(create.library != null);
+      libraryName = (String) value(create.library);
+    }
+    final SchemaFactory schemaFactory =
+        AvaticaUtils.instantiatePlugin(SchemaFactory.class, libraryName);
+    final Map<String, Object> operandMap = new LinkedHashMap<>();
+    for (Pair<SqlIdentifier, SqlNode> option : create.options()) {
+      operandMap.put(option.left.getSimple(), value(option.right));
+    }
+    subSchema =
+        schemaFactory.create(pair.left.plus(), pair.right, operandMap);
+    pair.left.add(pair.right, subSchema);
+  }
+
+  /** Executes a {@code CREATE FUNCTION} command. */
+  public void execute(SqlCreateFunction create,
+      CalcitePrepare.Context context) {
+    throw new UnsupportedOperationException("CREATE FUNCTION is not supported");
+  }
+
+  /** Executes {@code DROP FUNCTION}, {@code DROP TABLE},
+   * {@code DROP MATERIALIZED VIEW}, {@code DROP TYPE},
+   * {@code DROP VIEW} commands. */
+  public void execute(SqlDropObject drop,
+      CalcitePrepare.Context context) {
+    final List<String> path = context.getDefaultSchemaPath();
+    CalciteSchema schema = context.getRootSchema();
+    for (String p : path) {
+      schema = schema.getSubSchema(p, true);
+    }
+    final boolean existed;
+    switch (drop.getKind()) {
+    case DROP_TABLE:
+    case DROP_MATERIALIZED_VIEW:
+      existed = schema.removeTable(drop.name.getSimple());
+      if (!existed && !drop.ifExists) {
+        throw SqlUtil.newContextException(drop.name.getParserPosition(),
+            RESOURCE.tableNotFound(drop.name.getSimple()));
+      }
+      break;
+    case DROP_VIEW:
+      // Not quite right: removes any other functions with the same name
+      existed = schema.removeFunction(drop.name.getSimple());
+      if (!existed && !drop.ifExists) {
+        throw SqlUtil.newContextException(drop.name.getParserPosition(),
+            RESOURCE.viewNotFound(drop.name.getSimple()));
+      }
+      break;
+    case DROP_TYPE:
+      existed = schema.removeType(drop.name.getSimple());
+      if (!existed && !drop.ifExists) {
+        throw SqlUtil.newContextException(drop.name.getParserPosition(),
+            RESOURCE.typeNotFound(drop.name.getSimple()));
+      }
+      break;
+    case DROP_FUNCTION:
+      existed = schema.removeFunction(drop.name.getSimple());
+      if (!existed && !drop.ifExists) {
+        throw SqlUtil.newContextException(drop.name.getParserPosition(),
+            RESOURCE.functionNotFound(drop.name.getSimple()));
+      }
+      break;
+    case OTHER_DDL:
+    default:
+      throw new AssertionError(drop.getKind());
+    }
+  }
+
+  /** Executes a {@code CREATE MATERIALIZED VIEW} command. */
+  public void execute(SqlCreateMaterializedView create,
+      CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair = schema(context, true, create.name);
+    if (pair.left.plus().getTable(pair.right) != null) {
+      // Materialized view exists.
+      if (!create.ifNotExists) {
+        // They did not specify IF NOT EXISTS, so give error.
+        throw SqlUtil.newContextException(create.name.getParserPosition(),
+            RESOURCE.tableExists(pair.right));
+      }
+      return;
+    }
+    final SqlNode q = renameColumns(create.columnList, create.query);
+    final String sql = q.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
+    final List<String> schemaPath = pair.left.path(null);
+    final ViewTableMacro viewTableMacro =
+        ViewTable.viewMacro(pair.left.plus(), sql, schemaPath,
+            context.getObjectPath(), false);
+    final TranslatableTable x = viewTableMacro.apply(ImmutableList.of());
+    final RelDataType rowType = x.getRowType(context.getTypeFactory());
+
+    // Table does not exist. Create it.
+    final MaterializedViewTable table =
+        new MaterializedViewTable(pair.right, RelDataTypeImpl.proto(rowType));
+    pair.left.add(pair.right, table);
+    populate(create.name, create.query, context);
+    table.key =
+        MaterializationService.instance().defineMaterialization(pair.left, null,
+            sql, schemaPath, pair.right, true, true);
+  }
+
+  /** Executes a {@code DROP MATERIALIZED VIEW} command. */
+  public void execute(SqlDropMaterializedView drop,
+      CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair = schema(context, true, drop.name);
+    final Table table = pair.left.plus().getTable(pair.right);
+    if (table != null) {
+      // Materialized view exists.
+      execute((SqlDropObject) drop, context);
+      if (table instanceof Wrapper) {
+        final MaterializationKey materializationKey =
+            ((Wrapper) table).unwrap(MaterializationKey.class);
+        if (materializationKey != null) {
+          MaterializationService.instance()
+              .removeMaterialization(materializationKey);
+        }
+      }
+    }
+  }
+
+  /** Executes a {@code CREATE SCHEMA} command. */
+  public void execute(SqlCreateSchema create,
+      CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair = schema(context, true, create.name);
+    final SchemaPlus subSchema0 = pair.left.plus().getSubSchema(pair.right);
+    if (subSchema0 != null) {
+      if (!create.getReplace() && !create.ifNotExists) {
+        throw SqlUtil.newContextException(create.name.getParserPosition(),
+            RESOURCE.schemaExists(pair.right));
+      }
+    }
+    final Schema subSchema = new AbstractSchema();
+    pair.left.add(pair.right, subSchema);
+  }
+
+  /** Executes a {@code DROP SCHEMA} command. */
+  public void execute(SqlDropSchema drop,
+      CalcitePrepare.Context context) {
+    final List<String> path = context.getDefaultSchemaPath();
+    CalciteSchema schema = context.getRootSchema();
+    for (String p : path) {
+      schema = schema.getSubSchema(p, true);
+    }
+    final boolean existed = schema.removeSubSchema(drop.name.getSimple());
+    if (!existed && !drop.ifExists) {
+      throw SqlUtil.newContextException(drop.name.getParserPosition(),
+          RESOURCE.schemaNotFound(drop.name.getSimple()));
+    }
+  }
+
+  /** Executes a {@code CREATE TABLE} command. */
+  public void execute(SqlCreateTable create,
+      CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair =
+        schema(context, true, create.name);
+    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(pair.left.plus(), sql, pair.left.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 List<SqlNode> columnList;
+    if (create.columnList != null) {
+      columnList = create.columnList.getList();
+    } 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());
+      }
+      columnList = new ArrayList<>();
+      for (String name : queryRowType.getFieldNames()) {
+        columnList.add(new SqlIdentifier(name, SqlParserPos.ZERO));
+      }
+    }
+    final ImmutableList.Builder<ColumnDef> b = ImmutableList.builder();
+    final RelDataTypeFactory.Builder builder = typeFactory.builder();
+    final RelDataTypeFactory.Builder storedBuilder = typeFactory.builder();
+    // REVIEW 2019-08-19 Danny Chan: Should we implement the
+    // #validate(SqlValidator) to get the SqlValidator instance?
+    final SqlValidator validator = validator(context, true);
+    for (Ord<SqlNode> c : Ord.zip(columnList)) {
+      if (c.e instanceof SqlColumnDeclaration) {
+        final SqlColumnDeclaration d = (SqlColumnDeclaration) c.e;
+        final RelDataType type = d.dataType.deriveType(validator, true);
+        builder.add(d.name.getSimple(), type);
+        if (d.strategy != ColumnStrategy.VIRTUAL) {
+          storedBuilder.add(d.name.getSimple(), type);
+        }
+        b.add(ColumnDef.of(d.expression, type, d.strategy));
+      } else if (c.e instanceof SqlIdentifier) {
+        final SqlIdentifier id = (SqlIdentifier) c.e;
+        if (queryRowType == null) {
+          throw SqlUtil.newContextException(id.getParserPosition(),
+              RESOURCE.createTableRequiresColumnTypes(id.getSimple()));
+        }
+        final RelDataTypeField f = queryRowType.getFieldList().get(c.i);
+        final ColumnStrategy strategy = f.getType().isNullable()
+            ? ColumnStrategy.NULLABLE
+            : ColumnStrategy.NOT_NULLABLE;
+        b.add(ColumnDef.of(c.e, f.getType(), strategy));
+        builder.add(id.getSimple(), f.getType());
+        storedBuilder.add(id.getSimple(), f.getType());
+      } else {
+        throw new AssertionError(c.e.getClass());
+      }
+    }
+    final RelDataType rowType = builder.build();
+    final RelDataType storedRowType = storedBuilder.build();
+    final List<ColumnDef> columns = b.build();
+    final InitializerExpressionFactory ief =
+        new NullInitializerExpressionFactory() {
+          @Override public ColumnStrategy generationStrategy(RelOptTable table,
+              int iColumn) {
+            return columns.get(iColumn).strategy;
+          }
+
+          @Override public RexNode newColumnDefaultValue(RelOptTable table,
+              int iColumn, InitializerContext context) {
+            final ColumnDef c = columns.get(iColumn);
+            if (c.expr != null) {
+              // REVIEW Danny 2019-10-09: Should we support validation for DDL nodes?
+              final SqlNode validated = context.validateExpression(storedRowType, c.expr);
+              // The explicit specified type should have the same nullability
+              // with the column expression inferred type,
+              // actually they should be exactly the same.
+              return context.convertExpression(validated);
+            }
+            return super.newColumnDefaultValue(table, iColumn, context);
+          }
+        };
+    if (pair.left.plus().getTable(pair.right) != null) {
+      // Table exists.
+      if (!create.ifNotExists) {
+        // They did not specify IF NOT EXISTS, so give error.
+        throw SqlUtil.newContextException(create.name.getParserPosition(),
+            RESOURCE.tableExists(pair.right));
+      }
+      return;
+    }
+    // Table does not exist. Create it.
+    pair.left.add(pair.right,
+        new MutableArrayTable(pair.right,
+            RelDataTypeImpl.proto(storedRowType),
+            RelDataTypeImpl.proto(rowType), ief));
+    if (create.query != null) {
+      populate(create.name, create.query, context);
+    }
+  }
+
+  /** Executes a {@code CREATE TYPE} command. */
+  public void execute(SqlCreateType create,
+      CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair = schema(context, true, create.name);
+    final SqlValidator validator = validator(context, false);
+    pair.left.add(pair.right, typeFactory -> {
+      if (create.dataType != null) {
+        return create.dataType.deriveType(validator);
+      } else {
+        final RelDataTypeFactory.Builder builder = typeFactory.builder();
+        for (SqlNode def : create.attributeDefs) {
+          final SqlAttributeDefinition attributeDef =
+              (SqlAttributeDefinition) def;
+          final SqlDataTypeSpec typeSpec = attributeDef.dataType;
+          final RelDataType type = typeSpec.deriveType(validator);
+          builder.add(attributeDef.name.getSimple(), type);
+        }
+        return builder.build();
+      }
+    });
+  }
+
+  /** 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);
+  }
+
+  /** Column definition. */
+  private static class ColumnDef {
+    final SqlNode expr;
+    final RelDataType type;
+    final ColumnStrategy strategy;
+
+    private ColumnDef(SqlNode expr, RelDataType type,
+        ColumnStrategy strategy) {
+      this.expr = expr;
+      this.type = type;
+      this.strategy = Objects.requireNonNull(strategy);
+      Preconditions.checkArgument(
+          strategy == ColumnStrategy.NULLABLE
+              || strategy == ColumnStrategy.NOT_NULLABLE
+              || expr != null);
+    }
+
+    static ColumnDef of(SqlNode expr, RelDataType type,
+        ColumnStrategy strategy) {
+      return new ColumnDef(expr, type, strategy);
+    }
+  }
+}
diff --git a/server/src/test/java/org/apache/calcite/test/ServerTest.java b/server/src/test/java/org/apache/calcite/test/ServerTest.java
index f91c688..45db9fb 100644
--- a/server/src/test/java/org/apache/calcite/test/ServerTest.java
+++ b/server/src/test/java/org/apache/calcite/test/ServerTest.java
@@ -18,9 +18,22 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.config.CalciteConnectionProperty;
 import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalcitePrepare;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.FunctionParameter;
-import org.apache.calcite.sql.parser.ddl.SqlDdlParserImpl;
+import org.apache.calcite.server.DdlExecutorImpl;
+import org.apache.calcite.server.ServerDdlExecutor;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.ddl.SqlCreateForeignSchema;
+import org.apache.calcite.sql.ddl.SqlCreateFunction;
+import org.apache.calcite.sql.ddl.SqlCreateMaterializedView;
+import org.apache.calcite.sql.ddl.SqlCreateSchema;
+import org.apache.calcite.sql.ddl.SqlCreateTable;
+import org.apache.calcite.sql.ddl.SqlCreateType;
+import org.apache.calcite.sql.ddl.SqlCreateView;
+import org.apache.calcite.sql.ddl.SqlDropFunction;
+import org.apache.calcite.sql.ddl.SqlDropMaterializedView;
+import org.apache.calcite.sql.ddl.SqlDropSchema;
 
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
@@ -57,13 +70,38 @@ class ServerTest {
     return DriverManager.getConnection(URL,
         CalciteAssert.propBuilder()
             .set(CalciteConnectionProperty.PARSER_FACTORY,
-                SqlDdlParserImpl.class.getName() + "#FACTORY")
+                ServerDdlExecutor.class.getName() + "#PARSER_FACTORY")
             .set(CalciteConnectionProperty.MATERIALIZATIONS_ENABLED,
                 "true")
             .set(CalciteConnectionProperty.FUN, "standard,oracle")
             .build());
   }
 
+  /** Contains calls to all overloaded {@code execute} methods in
+   * {@link DdlExecutorImpl} to silence warnings that these methods are not
+   * called. (They are, not from this test, but via reflection.) */
+  @Test void testAll() {
+    //noinspection ConstantConditions
+    if (true) {
+      return;
+    }
+    final ServerDdlExecutor executor = ServerDdlExecutor.INSTANCE;
+    final Object o = "x";
+    final CalcitePrepare.Context context = (CalcitePrepare.Context) o;
+    executor.execute((SqlNode) o, context);
+    executor.execute((SqlCreateFunction) o, context);
+    executor.execute((SqlCreateTable) o, context);
+    executor.execute((SqlCreateSchema) o, context);
+    executor.execute((SqlCreateMaterializedView) o, context);
+    executor.execute((SqlCreateView) o, context);
+    executor.execute((SqlCreateType) o, context);
+    executor.execute((SqlCreateSchema) o, context);
+    executor.execute((SqlCreateForeignSchema) o, context);
+    executor.execute((SqlDropMaterializedView) o, context);
+    executor.execute((SqlDropFunction) o, context);
+    executor.execute((SqlDropSchema) o, context);
+  }
+
   @Test void testStatement() throws Exception {
     try (Connection c = connect();
          Statement s = c.createStatement();
@@ -181,9 +219,10 @@ class ServerTest {
         boolean f = s.execute("create function if not exists s.t\n"
                 + "as 'org.apache.calcite.udf.TableFun.demoUdf'\n"
                 + "using jar 'file:/path/udf/udf-0.0.1-SNAPSHOT.jar'");
+        fail("expected error, got " + f);
       } catch (SQLException e) {
         assertThat(e.getMessage(),
-                containsString("CREATE FUNCTION is not supported yet"));
+            containsString("CREATE FUNCTION is not supported"));
       }
     }
   }
diff --git a/src/main/config/checkstyle/suppressions.xml b/src/main/config/checkstyle/suppressions.xml
index 44ce976..c6ee179 100644
--- a/src/main/config/checkstyle/suppressions.xml
+++ b/src/main/config/checkstyle/suppressions.xml
@@ -41,8 +41,11 @@ limitations under the License.
   <!-- Don't complain about method names in a class full of UDFs -->
   <suppress checks="MethodName" files="GeoFunctions.java"/>
 
-  <!-- Suppress JavadocPackage in the test packages -->
+  <!-- Suppress JavadocPackage in the test packages,
+       and in the 'org.apache.calcite.server' package, which is split between
+       'core' and 'server'. -->
   <suppress checks="JavadocPackage" files="src[/\\]test[/\\]java[/\\]"/>
+  <suppress checks="JavadocPackage" files="src[/\\]main[/\\]java[/\\]org[/\\]apache[/\\]calcite[/\\]server[/\\]"/>
 
   <!-- Method names in Resource can have underscores -->
   <suppress checks="MethodName" files="CalciteResource.java"/>