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 2017/11/27 19:04:07 UTC

[2/5] calcite git commit: [CALCITE-707] Add "server" module, with built-in support for simple DDL statements

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java
new file mode 100644
index 0000000..22cc2d3
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateTable.java
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.ddl;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.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.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 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;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
+
+  /** Creates a SqlCreateTable. */
+  SqlCreateTable(SqlParserPos pos, boolean replace, boolean ifNotExists,
+      SqlIdentifier name, SqlNodeList columnList, SqlNode query) {
+    super(OPERATOR, pos, replace, ifNotExists);
+    this.name = Preconditions.checkNotNull(name);
+    this.columnList = columnList; // may be null
+    this.query = query; // for "CREATE TABLE ... AS query"; may be null
+  }
+
+  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");
+    if (ifNotExists) {
+      writer.keyword("IF NOT EXISTS");
+    }
+    name.unparse(writer, leftPrec, rightPrec);
+    if (columnList != null) {
+      SqlWriter.Frame frame = writer.startList("(", ")");
+      for (SqlNode c : columnList) {
+        writer.sep(",");
+        c.unparse(writer, 0, 0);
+      }
+      writer.endList(frame);
+    }
+    if (query != null) {
+      writer.keyword("AS");
+      writer.newlineAndIndent();
+      query.unparse(writer, 0, 0);
+    }
+  }
+
+  public void execute(CalcitePrepare.Context context) {
+    final Pair<CalciteSchema, String> pair =
+        SqlDdlNodes.schema(context, true, name);
+    final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl();
+    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();
+    for (Ord<SqlNode> c : Ord.zip(columnList)) {
+      if (c.e instanceof SqlColumnDeclaration) {
+        final SqlColumnDeclaration d = (SqlColumnDeclaration) c.e;
+        final RelDataType type = d.dataType.deriveType(typeFactory, 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) {
+              return context.convertExpression(c.expr);
+            }
+            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 = Preconditions.checkNotNull(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 = Preconditions.checkNotNull(protoStoredRowType);
+      this.protoRowType = Preconditions.checkNotNull(protoRowType);
+      this.initializerExpressionFactory =
+          Preconditions.checkNotNull(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);
+    }
+  }
+}
+
+// End SqlCreateTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java
new file mode 100644
index 0000000..e9e3076
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateView.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+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;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("CREATE VIEW", SqlKind.CREATE_VIEW);
+
+  /** Creates a SqlCreateView. */
+  SqlCreateView(SqlParserPos pos, boolean replace, SqlIdentifier name,
+      SqlNodeList columnList, SqlNode query) {
+    super(OPERATOR, pos, replace, false);
+    this.name = Preconditions.checkNotNull(name);
+    this.columnList = columnList; // may be null
+    this.query = Preconditions.checkNotNull(query);
+  }
+
+  public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.of(name, columnList, query);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    if (getReplace()) {
+      writer.keyword("CREATE OR REPLACE");
+    } else {
+      writer.keyword("CREATE");
+    }
+    writer.keyword("VIEW");
+    name.unparse(writer, leftPrec, rightPrec);
+    if (columnList != null) {
+      SqlWriter.Frame frame = writer.startList("(", ")");
+      for (SqlNode c : columnList) {
+        writer.sep(",");
+        c.unparse(writer, 0, 0);
+      }
+      writer.endList(frame);
+    }
+    writer.keyword("AS");
+    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);
+  }
+
+}
+
+// End SqlCreateView.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
new file mode 100644
index 0000000..057c2ee
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.ddl;
+
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.schema.ColumnStrategy;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDrop;
+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.dialect.CalciteSqlDialect;
+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.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.io.PrintWriter;
+import java.io.StringWriter;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+
+/**
+ * Utilities concerning {@link SqlNode} for DDL.
+ */
+public class SqlDdlNodes {
+  private SqlDdlNodes() {}
+
+  /** Creates a CREATE SCHEMA. */
+  public static SqlCreateSchema createSchema(SqlParserPos pos, boolean replace,
+      boolean ifNotExists, SqlIdentifier name) {
+    return new SqlCreateSchema(pos, replace, ifNotExists, name);
+  }
+
+  /** Creates a CREATE FOREIGN SCHEMA. */
+  public static SqlCreateForeignSchema createForeignSchema(SqlParserPos pos,
+      boolean replace, boolean ifNotExists, SqlIdentifier name, SqlNode type,
+      SqlNode library, SqlNodeList optionList) {
+    return new SqlCreateForeignSchema(pos, replace, ifNotExists, name, type,
+        library, optionList);
+  }
+
+  /** Creates a CREATE TABLE. */
+  public static SqlCreateTable createTable(SqlParserPos pos, boolean replace,
+      boolean ifNotExists, SqlIdentifier name, SqlNodeList columnList,
+      SqlNode query) {
+    return new SqlCreateTable(pos, replace, ifNotExists, name, columnList,
+        query);
+  }
+
+  /** Creates a CREATE VIEW. */
+  public static SqlCreateView createView(SqlParserPos pos, boolean replace,
+      SqlIdentifier name, SqlNodeList columnList, SqlNode query) {
+    return new SqlCreateView(pos, replace, name, columnList, query);
+  }
+
+  /** Creates a CREATE MATERIALIZED VIEW. */
+  public static SqlCreateMaterializedView createMaterializedView(
+      SqlParserPos pos, boolean replace, boolean ifNotExists,
+      SqlIdentifier name, SqlNodeList columnList, SqlNode query) {
+    return new SqlCreateMaterializedView(pos, replace, ifNotExists, name,
+        columnList, query);
+  }
+
+  /** Creates a DROP [ FOREIGN ] SCHEMA. */
+  public static SqlDropSchema dropSchema(SqlParserPos pos, boolean foreign,
+      boolean ifExists, SqlIdentifier name) {
+    return new SqlDropSchema(pos, foreign, ifExists, name);
+  }
+
+  /** Creates a DROP TABLE. */
+  public static SqlDropTable dropTable(SqlParserPos pos, boolean ifExists,
+      SqlIdentifier name) {
+    return new SqlDropTable(pos, ifExists, name);
+  }
+
+  /** Creates a DROP VIEW. */
+  public static SqlDrop dropView(SqlParserPos pos, boolean ifExists,
+      SqlIdentifier name) {
+    return new SqlDropView(pos, ifExists, name);
+  }
+
+  /** Creates a DROP MATERIALIZED VIEW. */
+  public static SqlDrop dropMaterializedView(SqlParserPos pos,
+      boolean ifExists, SqlIdentifier name) {
+    return new SqlDropMaterializedView(pos, ifExists, name);
+  }
+
+  /** Creates a column declaration. */
+  public static SqlNode column(SqlParserPos pos, SqlIdentifier name,
+      SqlDataTypeSpec dataType, SqlNode expression, ColumnStrategy strategy) {
+    return new SqlColumnDeclaration(pos, name, dataType, expression, strategy);
+  }
+
+  /** Creates a CHECK constraint. */
+  public static SqlNode check(SqlParserPos pos, SqlIdentifier name,
+      SqlNode expression) {
+    return new SqlCheckConstraint(pos, name, expression);
+  }
+
+  /** Creates a UNIQUE constraint. */
+  public static SqlKeyConstraint unique(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList) {
+    return new SqlKeyConstraint(pos, name, columnList);
+  }
+
+  /** Creates a PRIMARY KEY constraint. */
+  public static SqlKeyConstraint primary(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList) {
+    return new SqlKeyConstraint(pos, name, columnList) {
+      @Override public SqlOperator getOperator() {
+        return PRIMARY;
+      }
+    };
+  }
+
+  /** 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);
+  }
+
+  /** 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 =
+        new SqlNodeList(ImmutableList.<SqlNode>of(SqlIdentifier.star(p)), p);
+    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);
+  }
+
+  /** 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 StringWriter sw = new StringWriter();
+      final PrintWriter pw = new PrintWriter(sw);
+      final SqlPrettyWriter w =
+          new SqlPrettyWriter(CalciteSqlDialect.DEFAULT, false, pw);
+      pw.print("INSERT INTO ");
+      name.unparse(w, 0, 0);
+      pw.print(" ");
+      query.unparse(w, 0, 0);
+      pw.flush();
+      final String sql = sw.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);
+    }
+  }
+}
+
+// End SqlDdlNodes.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java
new file mode 100644
index 0000000..14f68fd
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropMaterializedView.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.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.
+ */
+public class SqlDropMaterializedView extends SqlDropObject {
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("DROP MATERIALIZED VIEW",
+          SqlKind.DROP_MATERIALIZED_VIEW);
+
+  /** Creates a SqlDropMaterializedView. */
+  SqlDropMaterializedView(SqlParserPos pos, boolean ifExists,
+      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);
+        }
+      }
+    }
+  }
+}
+
+// End SqlDropMaterializedView.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
new file mode 100644
index 0000000..81672f2
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.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;
+
+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} and
+ * {@code DROP MATERIALIZED VIEW} statements.
+ */
+abstract class SqlDropObject extends SqlDrop
+    implements SqlExecutableStatement {
+  protected final SqlIdentifier name;
+
+  /** Creates a SqlDropObject. */
+  SqlDropObject(SqlOperator operator, SqlParserPos pos, boolean ifExists,
+      SqlIdentifier name) {
+    super(operator, pos, ifExists);
+    this.name = name;
+  }
+
+  public List<SqlNode> getOperandList() {
+    return ImmutableList.<SqlNode>of(name);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword(getOperator().getName()); // "DROP TABLE" etc.
+    if (ifExists) {
+      writer.keyword("IF EXISTS");
+    }
+    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;
+    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;
+    default:
+      throw new AssertionError(getKind());
+    }
+  }
+}
+
+// End SqlDropObject.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java
new file mode 100644
index 0000000..074fb8b
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropSchema.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.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;
+
+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.
+ */
+public class SqlDropSchema extends SqlDrop
+    implements SqlExecutableStatement {
+  private final boolean foreign;
+  private final SqlIdentifier name;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("DROP SCHEMA", SqlKind.DROP_TABLE);
+
+  /** Creates a SqlDropSchema. */
+  SqlDropSchema(SqlParserPos pos, boolean foreign, boolean ifExists,
+      SqlIdentifier name) {
+    super(OPERATOR, pos, ifExists);
+    this.foreign = foreign;
+    this.name = name;
+  }
+
+  public List<SqlNode> getOperandList() {
+    return ImmutableList.of(
+        SqlLiteral.createBoolean(foreign, SqlParserPos.ZERO), name);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword("DROP");
+    if (foreign) {
+      writer.keyword("FOREIGN");
+    }
+    writer.keyword("SCHEMA");
+    if (ifExists) {
+      writer.keyword("IF EXISTS");
+    }
+    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()));
+    }
+  }
+}
+
+// End SqlDropSchema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropTable.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropTable.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropTable.java
new file mode 100644
index 0000000..dda24e0
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropTable.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.ddl;
+
+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;
+
+/**
+ * Parse tree for {@code DROP TABLE} statement.
+ */
+public class SqlDropTable extends SqlDropObject {
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("DROP TABLE", SqlKind.DROP_TABLE);
+
+  /** Creates a SqlDropTable. */
+  SqlDropTable(SqlParserPos pos, boolean ifExists, SqlIdentifier name) {
+    super(OPERATOR, pos, ifExists, name);
+  }
+}
+
+// End SqlDropTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropView.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropView.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropView.java
new file mode 100644
index 0000000..091de18
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropView.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.ddl;
+
+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;
+
+/**
+ * Parse tree for {@code DROP VIEW} statement.
+ */
+public class SqlDropView extends SqlDropObject {
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("DROP VIEW", SqlKind.DROP_VIEW);
+
+  /** Creates a SqlDropView. */
+  SqlDropView(SqlParserPos pos, boolean ifExists, SqlIdentifier name) {
+    super(OPERATOR, pos, ifExists, name);
+  }
+}
+
+// End SqlDropView.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlKeyConstraint.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlKeyConstraint.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlKeyConstraint.java
new file mode 100644
index 0000000..6c75975
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlKeyConstraint.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.ddl;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import java.util.List;
+
+/**
+ * Parse tree for {@code UNIQUE}, {@code PRIMARY KEY} constraints.
+ *
+ * <p>And {@code FOREIGN KEY}, when we support it.
+ */
+public class SqlKeyConstraint extends SqlCall {
+  private static final SqlSpecialOperator UNIQUE =
+      new SqlSpecialOperator("UNIQUE", SqlKind.UNIQUE);
+
+  protected static final SqlSpecialOperator PRIMARY =
+      new SqlSpecialOperator("PRIMARY KEY", SqlKind.PRIMARY_KEY);
+
+  private final SqlIdentifier name;
+  private final SqlNodeList columnList;
+
+  /** Creates a SqlKeyConstraint. */
+  SqlKeyConstraint(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList) {
+    super(pos);
+    this.name = name;
+    this.columnList = columnList;
+  }
+
+  /** Creates a UNIQUE constraint. */
+  public static SqlKeyConstraint unique(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList) {
+    return new SqlKeyConstraint(pos, name, columnList);
+  }
+
+  /** Creates a PRIMARY KEY constraint. */
+  public static SqlKeyConstraint primary(SqlParserPos pos, SqlIdentifier name,
+      SqlNodeList columnList) {
+    return new SqlKeyConstraint(pos, name, columnList) {
+      @Override public SqlOperator getOperator() {
+        return PRIMARY;
+      }
+    };
+  }
+
+  @Override public SqlOperator getOperator() {
+    return UNIQUE;
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.of(name, columnList);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    if (name != null) {
+      writer.keyword("CONSTRAINT");
+      name.unparse(writer, 0, 0);
+    }
+    writer.keyword(getOperator().getName()); // "UNIQUE" or "PRIMARY KEY"
+    columnList.unparse(writer, 1, 1);
+  }
+}
+
+// End SqlKeyConstraint.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/package-info.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/package-info.java b/server/src/main/java/org/apache/calcite/sql/ddl/package-info.java
new file mode 100644
index 0000000..89ff42f
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/package-info.java
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/**
+ * Parse tree for SQL DDL statements.
+ *
+ * <p>These are available in the extended SQL parser that is part of Calcite's
+ * "server" module; the core parser in the "core" module only supports SELECT
+ * and DML.
+ *
+ * <p>If you are writing a project that requires DDL it is likely that your
+ * DDL syntax is different than ours. We recommend that you copy-paste this
+ * the parser and its supporting classes into your own module, rather than try
+ * to extend this one.
+ */
+@PackageMarker
+package org.apache.calcite.sql.ddl;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/test/ServerParserTest.java b/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
new file mode 100644
index 0000000..8cdf336
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.SqlParserTest;
+import org.apache.calcite.sql.parser.ddl.SqlDdlParserImpl;
+
+import org.junit.Test;
+
+/**
+ * Tests SQL parser extensions for DDL.
+ *
+ * <p>Remaining tasks:
+ * <ul>
+ *
+ * <li>"create table x (a int) as values 1, 2" should fail validation;
+ * data type not allowed in "create table ... as".
+ *
+ * <li>"create table x (a int, b int as (a + 1)) stored"
+ * should not allow b to be specified in insert;
+ * should generate check constraint on b;
+ * should populate b in insert as if it had a default
+ *
+ * <li>"create table as select" should store constraints
+ * deduced by planner
+ *
+ * <li>during CREATE VIEW, check for a table and a materialized view
+ * with the same name (they have the same namespace)
+ *
+ * </ul>
+ */
+public class ServerParserTest extends SqlParserTest {
+
+  @Override protected SqlParserImplFactory parserImplFactory() {
+    return SqlDdlParserImpl.FACTORY;
+  }
+
+  @Override public void testGenerateKeyWords() {
+    // by design, method only works in base class; no-ops in this sub-class
+  }
+
+  @Test public void testCreateSchema() {
+    sql("create schema x")
+        .ok("CREATE SCHEMA `X`");
+  }
+
+  @Test public void testCreateOrReplaceSchema() {
+    sql("create or replace schema x")
+        .ok("CREATE OR REPLACE SCHEMA `X`");
+  }
+
+  @Test public void testCreateForeignSchema() {
+    final String sql = "create or replace foreign schema x\n"
+        + "type 'jdbc'\n"
+        + "options (\n"
+        + "  aBoolean true,\n"
+        + "  anInteger -45,\n"
+        + "  aDate DATE '1970-03-21',\n"
+        + "  \"quoted.id\" TIMESTAMP '1970-03-21 12:4:56.78',\n"
+        + "  aString 'foo''bar')";
+    final String expected = "CREATE OR REPLACE FOREIGN SCHEMA `X` TYPE 'jdbc' "
+        + "OPTIONS (`ABOOLEAN` TRUE,"
+        + " `ANINTEGER` -45,"
+        + " `ADATE` DATE '1970-03-21',"
+        + " `quoted.id` TIMESTAMP '1970-03-21 12:04:56.78',"
+        + " `ASTRING` 'foo''bar')";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testCreateForeignSchema2() {
+    final String sql = "create or replace foreign schema x\n"
+        + "library 'com.example.ExampleSchemaFactory'\n"
+        + "options ()";
+    final String expected = "CREATE OR REPLACE FOREIGN SCHEMA `X` "
+        + "LIBRARY 'com.example.ExampleSchemaFactory' "
+        + "OPTIONS ()";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testCreateTable() {
+    sql("create table x (i int not null, j varchar(5) null)")
+        .ok("CREATE TABLE `X` (`I` INTEGER NOT NULL, `J` VARCHAR(5))");
+  }
+
+  @Test public void testCreateTableAsSelect() {
+    final String expected = "CREATE TABLE `X` AS\n"
+        + "SELECT *\n"
+        + "FROM `EMP`";
+    sql("create table x as select * from emp")
+        .ok(expected);
+  }
+
+  @Test public void testCreateTableIfNotExistsAsSelect() {
+    final String expected = "CREATE TABLE IF NOT EXISTS `X`.`Y` AS\n"
+        + "SELECT *\n"
+        + "FROM `EMP`";
+    sql("create table if not exists x.y as select * from emp")
+        .ok(expected);
+  }
+
+  @Test public void testCreateTableAsValues() {
+    final String expected = "CREATE TABLE `X` AS\n"
+        + "VALUES (ROW(1)),\n"
+        + "(ROW(2))";
+    sql("create table x as values 1, 2")
+        .ok(expected);
+  }
+
+  @Test public void testCreateTableAsSelectColumnList() {
+    final String expected = "CREATE TABLE `X` (`A`, `B`) AS\n"
+        + "SELECT *\n"
+        + "FROM `EMP`";
+    sql("create table x (a, b) as select * from emp")
+        .ok(expected);
+  }
+
+  @Test public void testCreateTableCheck() {
+    final String expected = "CREATE TABLE `X` (`I` INTEGER NOT NULL,"
+        + " CONSTRAINT `C1` CHECK (`I` < 10), `J` INTEGER)";
+    sql("create table x (i int not null, constraint c1 check (i < 10), j int)")
+        .ok(expected);
+  }
+
+  @Test public void testCreateTableVirtualColumn() {
+    final String sql = "create table if not exists x (\n"
+        + " i int not null,\n"
+        + " j int generated always as (i + 1) stored,\n"
+        + " k int as (j + 1) virtual,\n"
+        + " m int as (k + 1))";
+    final String expected = "CREATE TABLE IF NOT EXISTS `X` "
+        + "(`I` INTEGER NOT NULL,"
+        + " `J` INTEGER AS (`I` + 1) STORED,"
+        + " `K` INTEGER AS (`J` + 1) VIRTUAL,"
+        + " `M` INTEGER AS (`K` + 1) VIRTUAL)";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testCreateView() {
+    final String sql = "create or replace view v as\n"
+        + "select * from (values (1, '2'), (3, '45')) as t (x, y)";
+    final String expected = "CREATE OR REPLACE VIEW `V` AS\n"
+        + "SELECT *\n"
+        + "FROM (VALUES (ROW(1, '2')),\n"
+        + "(ROW(3, '45'))) AS `T` (`X`, `Y`)";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testCreateMaterializedView() {
+    final String sql = "create materialized view mv (d, v) as\n"
+        + "select deptno, count(*) from emp\n"
+        + "group by deptno order by deptno desc";
+    final String expected = "CREATE MATERIALIZED VIEW `MV` (`D`, `V`) AS\n"
+        + "SELECT `DEPTNO`, COUNT(*)\n"
+        + "FROM `EMP`\n"
+        + "GROUP BY `DEPTNO`\n"
+        + "ORDER BY `DEPTNO` DESC";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testCreateMaterializedView2() {
+    final String sql = "create materialized view if not exists mv as\n"
+        + "select deptno, count(*) from emp\n"
+        + "group by deptno order by deptno desc";
+    final String expected = "CREATE MATERIALIZED VIEW IF NOT EXISTS `MV` AS\n"
+        + "SELECT `DEPTNO`, COUNT(*)\n"
+        + "FROM `EMP`\n"
+        + "GROUP BY `DEPTNO`\n"
+        + "ORDER BY `DEPTNO` DESC";
+    sql(sql).ok(expected);
+  }
+
+  // "OR REPLACE" is allowed by the parser, but the validator will give an
+  // error later
+  @Test public void testCreateOrReplaceMaterializedView() {
+    final String sql = "create or replace materialized view mv as\n"
+        + "select * from emp";
+    final String expected = "CREATE MATERIALIZED VIEW `MV` AS\n"
+        + "SELECT *\n"
+        + "FROM `EMP`";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testDropSchema() {
+    sql("drop schema x")
+        .ok("DROP SCHEMA `X`");
+  }
+
+  @Test public void testDropSchemaIfExists() {
+    sql("drop schema if exists x")
+        .ok("DROP SCHEMA IF EXISTS `X`");
+  }
+
+  @Test public void testDropForeignSchema() {
+    sql("drop foreign schema x")
+        .ok("DROP FOREIGN SCHEMA `X`");
+  }
+
+  @Test public void testDropTable() {
+    sql("drop table x")
+        .ok("DROP TABLE `X`");
+  }
+
+  @Test public void testDropTableComposite() {
+    sql("drop table x.y")
+        .ok("DROP TABLE `X`.`Y`");
+  }
+
+  @Test public void testDropTableIfExists() {
+    sql("drop table if exists x")
+        .ok("DROP TABLE IF EXISTS `X`");
+  }
+
+  @Test public void testDropView() {
+    sql("drop view x")
+        .ok("DROP VIEW `X`");
+  }
+
+  @Test public void testDropMaterializedView() {
+    sql("drop materialized view x")
+        .ok("DROP MATERIALIZED VIEW `X`");
+  }
+
+  @Test public void testDropMaterializedViewIfExists() {
+    sql("drop materialized view if exists x")
+        .ok("DROP MATERIALIZED VIEW IF EXISTS `X`");
+  }
+
+}
+
+// End ServerParserTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/test/java/org/apache/calcite/test/ServerQuidemTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/test/ServerQuidemTest.java b/server/src/test/java/org/apache/calcite/test/ServerQuidemTest.java
new file mode 100644
index 0000000..9e2efc3
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/test/ServerQuidemTest.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.materialize.MaterializationService;
+
+import net.hydromatic.quidem.Quidem;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.util.Collection;
+
+/**
+ * Unit tests for server and DDL.
+ */
+@RunWith(Parameterized.class)
+public class ServerQuidemTest extends QuidemTest {
+  /** Creates a ServerQuidemTest. Public per {@link Parameterized}. */
+  @SuppressWarnings("WeakerAccess")
+  public ServerQuidemTest(String path) {
+    super(path);
+  }
+
+  /** Runs a test from the command line.
+   *
+   * <p>For example:
+   *
+   * <blockquote>
+   *   <code>java ServerQuidemTest sql/table.iq</code>
+   * </blockquote> */
+  public static void main(String[] args) throws Exception {
+    for (String arg : args) {
+      new ServerQuidemTest(arg).test();
+    }
+  }
+
+  @Override @Test public void test() throws Exception {
+    MaterializationService.setThreadLocal();
+    super.test();
+  }
+
+  /** For {@link Parameterized} runner. */
+  @Parameterized.Parameters(name = "{index}: quidem({0})")
+  public static Collection<Object[]> data() {
+    // Start with a test file we know exists, then find the directory and list
+    // its files.
+    final String first = "sql/table.iq";
+    return data(first);
+  }
+
+  @Override protected Quidem.ConnectionFactory createConnectionFactory() {
+    return new QuidemConnectionFactory() {
+      @Override public Connection connect(String name, boolean reference)
+          throws Exception {
+        switch (name) {
+        case "server":
+          return ServerTest.connect();
+        }
+        return super.connect(name, reference);
+      }
+    };
+  }
+}
+
+// End ServerQuidemTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/test/java/org/apache/calcite/test/ServerTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/test/ServerTest.java b/server/src/test/java/org/apache/calcite/test/ServerTest.java
new file mode 100644
index 0000000..3a97412
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/test/ServerTest.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.sql.parser.ddl.SqlDdlParserImpl;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit tests for server and DDL.
+ */
+public class ServerTest {
+
+  static final String URL = "jdbc:calcite:";
+
+  static Connection connect() throws SQLException {
+    return DriverManager.getConnection(URL,
+        CalciteAssert.propBuilder()
+            .set(CalciteConnectionProperty.PARSER_FACTORY,
+                SqlDdlParserImpl.class.getName() + "#FACTORY")
+            .set(CalciteConnectionProperty.MATERIALIZATIONS_ENABLED,
+                "true")
+            .build());
+  }
+
+  @Test public void testStatement() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement();
+         ResultSet r = s.executeQuery("values 1, 2")) {
+      assertThat(r.next(), is(true));
+      assertThat(r.getString(1), notNullValue());
+      assertThat(r.next(), is(true));
+      assertThat(r.next(), is(false));
+    }
+  }
+
+  @Test public void testCreateSchema() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      boolean b = s.execute("create schema s");
+      assertThat(b, is(false));
+      b = s.execute("create table s.t (i int not null)");
+      assertThat(b, is(false));
+      int x = s.executeUpdate("insert into s.t values 1");
+      assertThat(x, is(1));
+      try (ResultSet r = s.executeQuery("select count(*) from s.t")) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getInt(1), is(1));
+        assertThat(r.next(), is(false));
+      }
+    }
+  }
+
+  @Test public void testCreateTable() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      boolean b = s.execute("create table t (i int not null)");
+      assertThat(b, is(false));
+      int x = s.executeUpdate("insert into t values 1");
+      assertThat(x, is(1));
+      x = s.executeUpdate("insert into t values 3");
+      assertThat(x, is(1));
+      try (ResultSet r = s.executeQuery("select sum(i) from t")) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getInt(1), is(4));
+        assertThat(r.next(), is(false));
+      }
+    }
+  }
+
+  @Test public void testStoredGeneratedColumn() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      final String sql0 = "create table t (\n"
+          + " h int not null,\n"
+          + " i int,\n"
+          + " j int as (i + 1) stored)";
+      boolean b = s.execute(sql0);
+      assertThat(b, is(false));
+
+      int x;
+
+      // A successful row.
+      x = s.executeUpdate("insert into t (h, i) values (3, 4)");
+      assertThat(x, is(1));
+
+      final String sql1 = "explain plan for\n"
+          + "insert into t (h, i) values (3, 4)";
+      try (ResultSet r = s.executeQuery(sql1)) {
+        assertThat(r.next(), is(true));
+        final String plan = ""
+            + "EnumerableTableModify(table=[[T]], operation=[INSERT], flattened=[false])\n"
+            + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], proj#0..1=[{exprs}], J=[$t3])\n"
+            + "    EnumerableValues(tuples=[[{ 3, 4 }]])\n";
+        assertThat(r.getString(1), is(plan));
+        assertThat(r.next(), is(false));
+      }
+
+      try (ResultSet r = s.executeQuery("select * from t")) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getInt("H"), is(3));
+        assertThat(r.wasNull(), is(false));
+        assertThat(r.getInt("I"), is(4));
+        assertThat(r.getInt("J"), is(5)); // j = i + 1
+        assertThat(r.next(), is(false));
+      }
+
+      // No target column list; too few values provided
+      try {
+        x = s.executeUpdate("insert into t values (2, 3)");
+        fail("expected error, got " + x);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("Number of INSERT target columns (3) does not equal "
+                + "number of source items (2)"));
+      }
+
+      // No target column list; too many values provided
+      try {
+        x = s.executeUpdate("insert into t values (3, 4, 5, 6)");
+        fail("expected error, got " + x);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("Number of INSERT target columns (3) does not equal "
+                + "number of source items (4)"));
+      }
+
+      // No target column list;
+      // source count = target count;
+      // but one of the target columns is virtual.
+      try {
+        x = s.executeUpdate("insert into t values (3, 4, 5)");
+        fail("expected error, got " + x);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("Cannot INSERT into generated column 'J'"));
+      }
+
+      // Explicit target column list, omits virtual column
+      x = s.executeUpdate("insert into t (h, i) values (1, 2)");
+      assertThat(x, is(1));
+
+      // Explicit target column list, includes virtual column but assigns
+      // DEFAULT.
+      x = s.executeUpdate("insert into t (h, i, j) values (1, 2, DEFAULT)");
+      assertThat(x, is(1));
+
+      // As previous, re-order columns.
+      x = s.executeUpdate("insert into t (h, j, i) values (1, DEFAULT, 3)");
+      assertThat(x, is(1));
+
+      // Target column list exists,
+      // target column count equals the number of non-virtual columns;
+      // but one of the target columns is virtual.
+      try {
+        x = s.executeUpdate("insert into t (h, j) values (1, 3)");
+        fail("expected error, got " + x);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("Cannot INSERT into generated column 'J'"));
+      }
+
+      // Target column list exists and contains all columns,
+      // expression for virtual column is not DEFAULT.
+      try {
+        x = s.executeUpdate("insert into t (h, i, j) values (2, 3, 3 + 1)");
+        fail("expected error, got " + x);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("Cannot INSERT into generated column 'J'"));
+      }
+      x = s.executeUpdate("insert into t (h, i) values (0, 1)");
+      assertThat(x, is(1));
+      x = s.executeUpdate("insert into t (h, i, j) values (0, 1, DEFAULT)");
+      assertThat(x, is(1));
+      x = s.executeUpdate("insert into t (j, i, h) values (DEFAULT, NULL, 7)");
+      assertThat(x, is(1));
+      x = s.executeUpdate("insert into t (h, i) values (6, 5), (7, 4)");
+      assertThat(x, is(2));
+      try (ResultSet r = s.executeQuery("select sum(i), count(*) from t")) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getInt(1), is(19));
+        assertThat(r.getInt(2), is(9));
+        assertThat(r.next(), is(false));
+      }
+    }
+  }
+
+  @Ignore("not working yet")
+  @Test public void testStoredGeneratedColumn2() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      final String sql = "create table t (\n"
+          + " h int not null,\n"
+          + " i int,\n"
+          + " j int as (i + 1) stored)";
+      boolean b = s.execute(sql);
+      assertThat(b, is(false));
+
+      // Planner uses constraint to optimize away condition.
+      final String sql2 = "explain plan for\n"
+          + "select * from t where j = i + 1";
+      final String plan = "EnumerableTableScan(table=[[T]])\n";
+      try (ResultSet r = s.executeQuery(sql2)) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getString(1), is(plan));
+        assertThat(r.next(), is(false));
+      }
+    }
+  }
+
+  @Test public void testVirtualColumn() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      final String sql0 = "create table t (\n"
+          + " h int not null,\n"
+          + " i int,\n"
+          + " j int as (i + 1) virtual)";
+      boolean b = s.execute(sql0);
+      assertThat(b, is(false));
+
+      int x = s.executeUpdate("insert into t (h, i) values (1, 2)");
+      assertThat(x, is(1));
+
+      // In plan, "j" is replaced by "i + 1".
+      final String sql = "select * from t";
+      try (ResultSet r = s.executeQuery(sql)) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getInt(1), is(1));
+        assertThat(r.getInt(2), is(2));
+        assertThat(r.getInt(3), is(3));
+        assertThat(r.next(), is(false));
+      }
+
+      final String plan = ""
+          + "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], proj#0..1=[{exprs}], $f2=[$t3])\n"
+          + "  EnumerableTableScan(table=[[T]])\n";
+      try (ResultSet r = s.executeQuery("explain plan for " + sql)) {
+        assertThat(r.next(), is(true));
+        assertThat(r.getString(1), is(plan));
+      }
+    }
+  }
+}
+
+// End ServerTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/test/resources/sql/materialized_view.iq
----------------------------------------------------------------------
diff --git a/server/src/test/resources/sql/materialized_view.iq b/server/src/test/resources/sql/materialized_view.iq
new file mode 100644
index 0000000..e587313
--- /dev/null
+++ b/server/src/test/resources/sql/materialized_view.iq
@@ -0,0 +1,272 @@
+# materialized_view.iq - Materialized view DDL
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to you under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+!use server
+!set outputformat mysql
+
+# Create a source table
+create table dept (deptno int not null, name varchar(10));
+(0 rows modified)
+
+!update
+
+insert into dept
+values (10, 'Sales'), (20, 'Marketing'), (30, 'Engineering');
+(3 rows modified)
+
+!update
+
+# Create as select
+create materialized view v as
+select * from dept where deptno > 10;
+(0 rows modified)
+
+!update
+
+# Check contents
+select * from v;
++--------+-------------+
+| DEPTNO | NAME        |
++--------+-------------+
+|     20 | Marketing   |
+|     30 | Engineering |
++--------+-------------+
+(2 rows)
+
+!ok
+
+# Try to create again - fails
+create materialized view v as
+select * from dept where deptno < 30;
+Table 'V' already exists
+!error
+
+# Try to create again - fails silently
+create materialized view if not exists v as
+select * from dept where deptno < 30;
+(0 rows modified)
+
+!update
+
+# Check contents are unchanged
+select * from v;
++--------+-------------+
+| DEPTNO | NAME        |
++--------+-------------+
+|     20 | Marketing   |
+|     30 | Engineering |
++--------+-------------+
+(2 rows)
+
+!ok
+
+# Drop
+drop materialized view if exists v;
+(0 rows modified)
+
+!update
+
+# It's gone
+select * from v;
+Object 'V' not found
+!error
+
+# Drop does nothing because materialized view does not exist
+drop materialized view if exists v;
+(0 rows modified)
+
+!update
+
+# Create materialized view without AS - fails
+create materialized view d;
+Encountered "<EOF>" at line 1, column 27.
+!error
+
+# Create materialized view without AS - fails
+create materialized view d (x, y);
+Encountered "<EOF>" at line 1, column 34.
+!error
+
+# Create materialized view without AS - fails
+create materialized view d (x int, y);
+Encountered "int" at line 1, column 31.
+!error
+
+# Create based on itself - fails
+create materialized view d2 as select * from d2;
+Object 'D2' not found
+!error
+
+# Create materialized view based on UNION
+create materialized view d3 as
+select deptno as dd from dept where deptno < 15
+union all
+select deptno as ee from dept where deptno > 25;
+(0 rows modified)
+
+!update
+
+# Check contents
+select * from d3;
++----+
+| DD |
++----+
+| 10 |
+| 30 |
++----+
+(2 rows)
+
+!ok
+
+# Drop
+drop materialized view d3;
+(0 rows modified)
+
+!update
+
+# Create materialized view based on UNION and ORDER BY
+create materialized view d4 as
+select deptno as dd from dept where deptno < 15
+union all
+select deptno as dd from dept where deptno > 25
+order by 1 desc;
+(0 rows modified)
+
+!update
+
+# Check contents
+select * from d4;
++----+
+| DD |
++----+
+| 10 |
+| 30 |
++----+
+(2 rows)
+
+!ok
+
+# Drop
+drop materialized view d4;
+
+# Create materialized view based on VALUES
+create materialized view d5 as
+values (1, 'a'), (2, 'b');
+(0 rows modified)
+
+!update
+
+# Check contents
+select * from d5;
++--------+--------+
+| EXPR$0 | EXPR$1 |
++--------+--------+
+|      1 | a      |
+|      2 | b      |
++--------+--------+
+(2 rows)
+
+!ok
+
+# Use just aliases
+create materialized view d6 (x, y) as
+select * from dept where deptno < 15;
+(0 rows modified)
+
+!update
+
+# Check contents
+select * from d6;
++----+-------+
+| X  | Y     |
++----+-------+
+| 10 | Sales |
++----+-------+
+(1 row)
+
+!ok
+
+# Use a mixture of aliases and column declarations - fails
+create materialized view d7 (x int, y) as
+select * from dept where deptno < 15;
+Encountered "int" at line 1, column 32.
+!error
+
+# Too many columns
+create materialized view d8 (x, y, z) as
+select * from dept where deptno < 15;
+List of column aliases must have same degree as table; table has 2 columns ('DEPTNO', 'NAME'), whereas alias list has 3 columns
+!error
+
+# Too few columns
+create materialized view d9 (x) as
+select * from dept where deptno < 15;
+List of column aliases must have same degree as table; table has 2 columns ('DEPTNO', 'NAME'), whereas alias list has 1 columns
+!error
+
+create schema s;
+(0 rows modified)
+
+!update
+
+# Materialized view in explicit schema
+create materialized view s.d10 (x, y) as
+select * from dept where deptno < 25;
+(0 rows modified)
+
+!update
+
+# Check contents
+select * from s.d10;
++----+-----------+
+| X  | Y         |
++----+-----------+
+| 10 | Sales     |
+| 20 | Marketing |
++----+-----------+
+(2 rows)
+
+!ok
+
+# Appears in catalog, with table type 'MATERIALIZED VIEW'
+# (Materialized views in root schema should also, but currently do not.)
+select * from "metadata".TABLES;
++----------+------------+-----------+-------------------+---------+---------+-----------+----------+------------------------+---------------+
+| tableCat | tableSchem | tableName | tableType         | remarks | typeCat | typeSchem | typeName | selfReferencingColName | refGeneration |
++----------+------------+-----------+-------------------+---------+---------+-----------+----------+------------------------+---------------+
+|          | S          | D10       | MATERIALIZED VIEW |         |         |           |          |                        |               |
+|          | metadata   | COLUMNS   | SYSTEM TABLE      |         |         |           |          |                        |               |
+|          | metadata   | TABLES    | SYSTEM TABLE      |         |         |           |          |                        |               |
++----------+------------+-----------+-------------------+---------+---------+-----------+----------+------------------------+---------------+
+(3 rows)
+
+!ok
+
+# Check that exact match materialized view is used
+select * from dept where deptno < 15;
+EnumerableTableScan(table=[[D6]])
+!plan
++--------+-------+
+| DEPTNO | NAME  |
++--------+-------+
+|     10 | Sales |
++--------+-------+
+(1 row)
+
+!ok
+
+# End materialized_view.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/test/resources/sql/schema.iq
----------------------------------------------------------------------
diff --git a/server/src/test/resources/sql/schema.iq b/server/src/test/resources/sql/schema.iq
new file mode 100755
index 0000000..65c6396
--- /dev/null
+++ b/server/src/test/resources/sql/schema.iq
@@ -0,0 +1,168 @@
+# schema.iq - DDL on schemas
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to you under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+!use server
+!set outputformat mysql
+
+# Create a schema
+create schema s;
+(0 rows modified)
+
+!update
+
+# Create a table and a view in the schema
+create table s.t (i int);
+(0 rows modified)
+
+!update
+
+create view s.v as select * from s.t;
+(0 rows modified)
+
+!update
+
+select count(*) as c from s.v;
++---+
+| C |
++---+
+| 0 |
++---+
+(1 row)
+
+!ok
+
+# Try to create a schema that already exists
+create schema s;
+Schema 'S' already exists
+!error
+
+create or replace schema s;
+(0 rows modified)
+
+!update
+
+create schema if exists s;
+Encountered "exists" at line 1, column 18.
+!error
+
+create schema if not exists s;
+(0 rows modified)
+
+!update
+
+# Bad library
+create foreign schema fs library 'com.example.BadSchemaFactory';
+Property 'com.example.BadSchemaFactory' not valid for plugin type org.apache.calcite.schema.SchemaFactory
+!error
+
+# Bad type
+create foreign schema fs type 'bad';
+Invalid schema type 'bad'; valid values: [MAP, JDBC, CUSTOM]
+!error
+
+# Can not specify both type and library
+create foreign schema fs
+  type 'jdbc'
+  library 'org.apache.calcite.test.JdbcTest.MySchemaFactory';
+Encountered "library" at line 3, column 3.
+!error
+
+# Cannot specify type or library with non-foreign schema
+create schema fs type 'jdbc';
+Encountered "type" at line 1, column 18.
+!error
+
+create schema fs library 'org.apache.calcite.test.JdbcTest.MySchemaFactory';
+Encountered "library" at line 1, column 18.
+!error
+
+create foreign schema fs;
+Encountered "<EOF>" at line 1, column 25.
+Was expecting one of:
+    "TYPE" ...
+    "LIBRARY" ...
+    "." ...
+!error
+
+# JDBC schema
+create foreign schema scott type 'jdbc' options (
+  "jdbcUrl" 'jdbc:hsqldb:res:scott',
+  "jdbcSchema" 'SCOTT',
+  "jdbcUser" 'SCOTT',
+  "jdbcPassword" 'TIGER');
+(0 rows modified)
+
+!update
+
+select count(*) as c from scott.dept;
++---+
+| C |
++---+
+| 4 |
++---+
+(1 row)
+
+!ok
+
+# Drop schema, then make sure that a query can't find it
+drop schema if exists s;
+(0 rows modified)
+
+!update
+
+select * from s.t;
+Object 'T' not found
+!error
+
+# Create again and objects are still gone
+create schema s;
+
+select * from s.t;
+Object 'T' not found
+!error
+
+select * from s.v;
+Object 'V' not found
+!error
+
+# Try to drop schema that does not exist
+drop schema sss;
+Schema 'SSS' not found
+!error
+
+drop schema if exists sss;
+(0 rows modified)
+
+!update
+
+drop foreign schema if exists sss;
+(0 rows modified)
+
+!update
+
+# Use 'if exists' to drop a foreign schema that does exist
+drop foreign schema if exists scott;
+(0 rows modified)
+
+!update
+
+drop foreign schema if exists scott;
+(0 rows modified)
+
+!update
+
+# End schema.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/test/resources/sql/table.iq
----------------------------------------------------------------------
diff --git a/server/src/test/resources/sql/table.iq b/server/src/test/resources/sql/table.iq
new file mode 100755
index 0000000..ed89e0f
--- /dev/null
+++ b/server/src/test/resources/sql/table.iq
@@ -0,0 +1,154 @@
+# table.iq - Table DDL
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to you under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+!use server
+!set outputformat mysql
+
+# Create a basic table
+create table t (i int, j int not null);
+(0 rows modified)
+
+!update
+
+create table if not exists t (i int, j int not null, k date);
+(0 rows modified)
+
+!update
+
+# There is no "K" column, because table was not re-created
+select * from t;
+I INTEGER(10)
+J INTEGER(10) NOT NULL
+!type
+
+insert into t values (1, 2);
+(1 row modified)
+
+!update
+
+select * from t;
++---+---+
+| I | J |
++---+---+
+| 1 | 2 |
++---+---+
+(1 row)
+
+!ok
+
+drop table t;
+(0 rows modified)
+
+!update
+
+# Create a table with a DEFAULT column
+create table t (i int, j int default i + 2);
+(0 rows modified)
+
+!update
+
+insert into t values (1, 2);
+(1 row modified)
+
+!update
+
+insert into t (i) values (3);
+(1 row modified)
+
+!update
+
+select * from t;
++---+---+
+| I | J |
++---+---+
+| 1 | 2 |
+| 3 | 5 |
++---+---+
+(2 rows)
+
+!ok
+
+drop table t;
+(0 rows modified)
+
+!update
+
+# Create a table with a VIRTUAL column
+
+create table t (i int, j int as (i + k + 2) virtual, k int);
+(0 rows modified)
+
+!update
+
+insert into t values (1, 2, 3);
+Cannot INSERT into generated column 'J'
+!error
+
+insert into t (i, j) values (1, 2);
+Cannot INSERT into generated column 'J'
+!error
+
+insert into t (i, k) values (1, 3);
+(1 row modified)
+
+!update
+EnumerableTableModify(table=[[T]], operation=[INSERT], flattened=[false])
+  EnumerableValues(tuples=[[{ 1, 3 }]])
+!plan
+
+insert into t (k, i) values (5, 2);
+(1 row modified)
+
+!update
+EnumerableTableModify(table=[[T]], operation=[INSERT], flattened=[false])
+  EnumerableCalc(expr#0..1=[{inputs}], I=[$t1], K=[$t0])
+    EnumerableValues(tuples=[[{ 5, 2 }]])
+!plan
+
+select * from t;
++---+---+---+
+| I | J | K |
++---+---+---+
+| 1 | 6 | 3 |
+| 2 | 9 | 5 |
++---+---+---+
+(2 rows)
+
+!ok
+EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[2], expr#4=[+($t2, $t3)], I=[$t0], $f1=[$t4], K=[$t1])
+  EnumerableTableScan(table=[[T]])
+!plan
+
+drop table if exists t;
+(0 rows modified)
+
+!update
+
+select * from t;
+Object 'T' not found
+!error
+
+drop table t;
+Table 'T' not found
+!error
+
+drop table if exists t;
+(0 rows modified)
+
+!update
+
+# End table.iq