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:08 UTC

[3/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/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 74d0c0d..5a0f888 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -149,6 +149,7 @@ FunctionQuantifierNotAllowed=DISTINCT/ALL not allowed with {0} function
 SomeButNotAllArgumentsAreNamed=Some but not all arguments are named
 DuplicateArgumentName=Duplicate argument name ''{0}''
 DefaultForOptionalParameter=DEFAULT is only allowed for optional parameters
+DefaultNotAllowed=DEFAULT not allowed here
 AccessNotAllowed=Not allowed to perform {0} on {1}
 MinMaxBadType=The {0} function does not support the {1} data type.
 OnlyScalarSubQueryAllowed=Only scalar sub-queries allowed in select list.
@@ -172,7 +173,7 @@ InvalidBoolean=''{0}'' is not a valid boolean value
 ArgumentMustBeValidPrecision=Argument to function ''{0}'' must be a valid precision between ''{1,number,#}'' and ''{2,number,#}''
 IllegalArgumentForTableFunctionCall=Wrong arguments for table function ''{0}'' call. Expected ''{1}'', actual ''{2}''
 InvalidDatetimeFormat=''{0}'' is not a valid datetime format
-InsertIntoAlwaysGenerated=Cannot explicitly insert value into IDENTITY column ''{0}'' which is ALWAYS GENERATED
+InsertIntoAlwaysGenerated=Cannot INSERT into generated column ''{0}''
 ArgumentMustHaveScaleZero=Argument to function ''{0}'' must have a scale of 0
 PreparationAborted=Statement preparation aborted
 SQLFeature_E051_01=SELECT DISTINCT not supported
@@ -235,4 +236,12 @@ CannotUseWithinWithoutOrderBy=Must contain an ORDER BY clause when WITHIN is use
 FirstColumnOfOrderByMustBeTimestamp=First column of ORDER BY must be of type TIMESTAMP
 ExtendNotAllowed=Extended columns not allowed under the current SQL conformance level
 RolledUpNotAllowed=Rolled up column ''{0}'' is not allowed in {1}
+SchemaExists=Schema ''{0}'' already exists
+SchemaInvalidType=Invalid schema type ''{0}''; valid values: {1}
+TableExists=Table ''{0}'' already exists
+CreateTableRequiresColumnList=Missing column list
+CreateTableRequiresColumnTypes=Type required for column ''{0}'' in CREATE TABLE without AS
+ViewExists=View ''{0}'' already exists and REPLACE not specified
+SchemaNotFound=Schema ''{0}'' not found
+ViewNotFound=View ''{0}'' not found
 # End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 4f2d9cb..90603db 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -1442,18 +1442,18 @@ public class SqlParserTest {
   }
 
   @Test public void testFunctionDefaultArgument() {
-    checkExp("foo(1, DEFAULT, default, 'default', \"default\", 3)",
-        "`FOO`(1, DEFAULT, DEFAULT, 'default', `default`, 3)");
-    checkExp("foo(DEFAULT)",
-        "`FOO`(DEFAULT)");
-    checkExp("foo(x => 1, DEFAULT)",
-        "`FOO`(`X` => 1, DEFAULT)");
-    checkExp("foo(y => DEFAULT, x => 1)",
-        "`FOO`(`Y` => DEFAULT, `X` => 1)");
-    checkExp("foo(x => 1, y => DEFAULT)",
-        "`FOO`(`X` => 1, `Y` => DEFAULT)");
-    check("select sum(DISTINCT DEFAULT) from t group by x",
-        "SELECT SUM(DISTINCT DEFAULT)\n"
+    sql("foo(1, DEFAULT, default, 'default', \"default\", 3)").expression()
+        .ok("`FOO`(1, DEFAULT, DEFAULT, 'default', `default`, 3)");
+    sql("foo(DEFAULT)").expression()
+        .ok("`FOO`(DEFAULT)");
+    sql("foo(x => 1, DEFAULT)").expression()
+        .ok("`FOO`(`X` => 1, DEFAULT)");
+    sql("foo(y => DEFAULT, x => 1)").expression()
+        .ok("`FOO`(`Y` => DEFAULT, `X` => 1)");
+    sql("foo(x => 1, y => DEFAULT)").expression()
+        .ok("`FOO`(`X` => 1, `Y` => DEFAULT)");
+    sql("select sum(DISTINCT DEFAULT) from t group by x")
+        .ok("SELECT SUM(DISTINCT DEFAULT)\n"
             + "FROM `T`\n"
             + "GROUP BY `X`");
     checkExpFails("foo(x ^+^ DEFAULT)",
@@ -1464,6 +1464,31 @@ public class SqlParserTest {
         "(?s).*Encountered \"\\+\" at .*");
   }
 
+  @Test public void testDefault() {
+    sql("select ^DEFAULT^ from emp")
+        .fails("(?s)Encountered \"DEFAULT\" at .*");
+    sql("select cast(empno ^+^ DEFAULT as double) from emp")
+        .fails("(?s)Encountered \"\\+ DEFAULT\" at .*");
+    sql("select empno ^+^ DEFAULT + deptno from emp")
+        .fails("(?s)Encountered \"\\+ DEFAULT\" at .*");
+    sql("select power(0, DEFAULT ^+^ empno) from emp")
+        .fails("(?s)Encountered \"\\+\" at .*");
+    sql("select * from emp join dept ^on^ DEFAULT")
+        .fails("(?s)Encountered \"on DEFAULT\" at .*");
+    sql("select * from emp where empno ^>^ DEFAULT or deptno < 10")
+        .fails("(?s)Encountered \"> DEFAULT\" at .*");
+    sql("select * from emp order by ^DEFAULT^ desc")
+        .fails("(?s)Encountered \"DEFAULT\" at .*");
+    final String expected = "INSERT INTO `DEPT` (`NAME`, `DEPTNO`)\n"
+        + "VALUES (ROW('a', DEFAULT))";
+    sql("insert into dept (name, deptno) values ('a', DEFAULT)")
+        .ok(expected);
+    sql("insert into dept (name, deptno) values ('a', 1 ^+^ DEFAULT)")
+        .fails("(?s)Encountered \"\\+ DEFAULT\" at .*");
+    sql("insert into dept (name, deptno) select 'a'^,^ DEFAULT from (values 0)")
+        .fails("(?s)Encountered \", DEFAULT\" at .*");
+  }
+
   @Test public void testAggregateFilter() {
     sql("select sum(sal) filter (where gender = 'F') as femaleSal,\n"
         + " sum(sal) filter (where true) allSal,\n"
@@ -3340,6 +3365,24 @@ public class SqlParserTest {
         .node(not(isDdl()));
   }
 
+  @Test public void testInsertValuesDefault() {
+    final String expected = "INSERT INTO `EMPS`\n"
+        + "VALUES (ROW(1, DEFAULT, 'Fredkin'))";
+    sql("insert into emps values (1,DEFAULT,'Fredkin')")
+        .ok(expected)
+        .node(not(isDdl()));
+  }
+
+  @Test public void testInsertValuesRawDefault() {
+    final String expected = "INSERT INTO `EMPS`\n"
+        + "VALUES (ROW(DEFAULT))";
+    sql("insert into emps ^values^ default")
+        .fails("(?s).*Encountered \"values default\" at .*");
+    sql("insert into emps values (default)")
+        .ok(expected)
+        .node(not(isDdl()));
+  }
+
   @Test public void testInsertColumnList() {
     final String expected = "INSERT INTO `EMPS` (`X`, `Y`)\n"
         + "(SELECT *\n"
@@ -8268,18 +8311,32 @@ public class SqlParserTest {
    * {@code sql("values 1").ok();}. */
   protected class Sql {
     private final String sql;
+    private final boolean expression;
 
     Sql(String sql) {
+      this(sql, false);
+    }
+
+    Sql(String sql, boolean expression) {
       this.sql = sql;
+      this.expression = expression;
     }
 
     public Sql ok(String expected) {
-      getTester().check(sql, expected);
+      if (expression) {
+        getTester().checkExp(sql, expected);
+      } else {
+        getTester().check(sql, expected);
+      }
       return this;
     }
 
     public Sql fails(String expectedMsgPattern) {
-      getTester().checkFails(sql, expectedMsgPattern);
+      if (expression) {
+        getTester().checkExpFails(sql, expectedMsgPattern);
+      } else {
+        getTester().checkFails(sql, expectedMsgPattern);
+      }
       return this;
     }
 
@@ -8287,6 +8344,11 @@ public class SqlParserTest {
       getTester().checkNode(sql, matcher);
       return this;
     }
+
+    /** Flags that this is an expression, not a whole query. */
+    public Sql expression() {
+      return expression ? this : new Sql(sql, true);
+    }
   }
 
   /** Runs tests on period operators such as OVERLAPS, IMMEDIATELY PRECEDES. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
index 2f57a5f..fa57d30 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
@@ -63,20 +63,16 @@ public class SqlCreateTable extends SqlCreate
   private final SqlNodeList columnList;
 
   private static final SqlOperator OPERATOR =
-      new SqlSpecialOperator("CREATE TABLE", SqlKind.OTHER_DDL);
+      new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
 
   /** Creates a SqlCreateTable. */
   public SqlCreateTable(SqlParserPos pos, SqlIdentifier name,
       SqlNodeList columnList) {
-    super(pos, false);
+    super(OPERATOR, pos, false, false);
     this.name = name;
     this.columnList = columnList;
   }
 
-  @Override public SqlOperator getOperator() {
-    return OPERATOR;
-  }
-
   @Override public List<SqlNode> getOperandList() {
     return ImmutableList.of(name, columnList);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 3e6b414..06e351a 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -855,6 +855,11 @@ public class CalciteAssert {
     return (Function<F, T>) (Function) Functions.<T>constant(null);
   }
 
+  /** Returns a {@link PropBuilder}. */
+  static PropBuilder propBuilder() {
+    return new PropBuilder();
+  }
+
   /**
    * Result of calling {@link CalciteAssert#that}.
    */
@@ -1850,6 +1855,21 @@ public class CalciteAssert {
       return s;
     }
   }
+
+  /** Builds a {@link java.util.Properties} containing connection property
+   * settings. */
+  static class PropBuilder {
+    final Properties properties = new Properties();
+
+    PropBuilder set(CalciteConnectionProperty p, String v) {
+      properties.setProperty(p.camelName(), v);
+      return this;
+    }
+
+    Properties build() {
+      return properties;
+    }
+  }
 }
 
 // End CalciteAssert.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index b6f2b9f..d577057 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -159,7 +159,7 @@ import org.junit.runners.Suite;
     LatticeTest.class,
     ReflectiveSchemaTest.class,
     JdbcTest.class,
-    QuidemTest.class,
+    CoreQuidemTest.class,
     CalciteRemoteDriverTest.class,
     StreamTest.class,
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/CoreQuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CoreQuidemTest.java b/core/src/test/java/org/apache/calcite/test/CoreQuidemTest.java
new file mode 100644
index 0000000..a4e9446
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/CoreQuidemTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.prepare.Prepare;
+import org.apache.calcite.util.TryThreadLocal;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+
+/**
+ * Test that runs every Quidem file in the "core" module as a test.
+ */
+@RunWith(Parameterized.class)
+public class CoreQuidemTest extends QuidemTest {
+  public CoreQuidemTest(String path) {
+    super(path);
+  }
+
+  /** Runs a test from the command line.
+   *
+   * <p>For example:
+   *
+   * <blockquote>
+   *   <code>java CoreQuidemTest sql/dummy.iq</code>
+   * </blockquote> */
+  public static void main(String[] args) throws Exception {
+    for (String arg : args) {
+      new CoreQuidemTest(arg).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/agg.iq";
+    return data(first);
+  }
+
+  /** Override settings for "sql/misc.iq". */
+  public void testSqlMisc() throws Exception {
+    switch (CalciteAssert.DB) {
+    case ORACLE:
+      // There are formatting differences (e.g. "4.000" vs "4") when using
+      // Oracle as the JDBC data source.
+      return;
+    }
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun(path);
+    }
+  }
+
+  /** Override settings for "sql/scalar.iq". */
+  public void testSqlScalar() throws Exception {
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun(path);
+    }
+  }
+
+  /** Runs the dummy script "sql/dummy.iq", which is checked in empty but
+   * which you may use as scratch space during development. */
+
+  // Do not disable this test; just remember not to commit changes to dummy.iq
+  public void testSqlDummy() throws Exception {
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun(path);
+    }
+  }
+
+}
+
+// End CoreQuidemTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 8760d82..7d6a479 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -730,7 +730,7 @@ public class JdbcTest {
              driver.connect("jdbc:calcite:", new Properties());
         Statement statement = connection.createStatement()) {
       assertThat(driver.counter, is(0));
-      statement.executeQuery("COMMIT");
+      statement.executeUpdate("COMMIT");
       assertThat(driver.counter, is(1));
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index b25ff10..5f96848 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -54,6 +54,7 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.schema.ColumnStrategy;
 import org.apache.calcite.schema.CustomColumnResolvingTable;
 import org.apache.calcite.schema.ExtensibleTable;
 import org.apache.calcite.schema.Path;
@@ -173,7 +174,7 @@ public class MockCatalogReader extends CalciteCatalogReader {
     // Register "EMP" table with customer InitializerExpressionFactory
     // to check whether newDefaultValue method called or not.
     final InitializerExpressionFactory countingInitializerExpressionFactory =
-        new CountingFactory(typeFactory);
+        new CountingFactory(ImmutableList.of("DEPTNO"));
 
     // Register "EMP" table.
     final MockTable empTable =
@@ -205,28 +206,9 @@ public class MockCatalogReader extends CalciteCatalogReader {
     registerTable(empNullablesTable);
 
     // Register "EMPDEFAULTS" table with default values for some columns.
-    final InitializerExpressionFactory empInitializerExpressionFactory =
-        new NullInitializerExpressionFactory() {
-          @Override public RexNode newColumnDefaultValue(RelOptTable table,
-              int iColumn, InitializerContext context) {
-            final RexBuilder rexBuilder = context.getRexBuilder();
-            switch (iColumn) {
-            case 0:
-              return rexBuilder.makeExactLiteral(new BigDecimal(123),
-                  typeFactory.createSqlType(SqlTypeName.INTEGER));
-            case 1:
-              return rexBuilder.makeLiteral("Bob");
-            case 5:
-              return rexBuilder.makeExactLiteral(new BigDecimal(555),
-                  typeFactory.createSqlType(SqlTypeName.INTEGER));
-            default:
-              return rexBuilder.constantNull();
-            }
-          }
-        };
     final MockTable empDefaultsTable =
         MockTable.create(this, salesSchema, "EMPDEFAULTS", false, 14, null,
-            empInitializerExpressionFactory);
+            new EmpInitializerExpressionFactory());
     empDefaultsTable.addColumn("EMPNO", f.intType, true);
     empDefaultsTable.addColumn("ENAME", f.varchar20Type);
     empDefaultsTable.addColumn("JOB", f.varchar10TypeNull);
@@ -381,12 +363,11 @@ public class MockCatalogReader extends CalciteCatalogReader {
     //   SELECT EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, SLACKER
     //   FROM EMP
     //   WHERE DEPTNO = 20 AND SAL > 1000
-    final NullInitializerExpressionFactory nullInitializerFactory =
-        new NullInitializerExpressionFactory();
     final ImmutableIntList m0 = ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8);
     MockTable emp20View =
         new MockViewTable(this, salesSchema.getCatalogName(), salesSchema.name,
-            "EMP_20", false, 600, empTable, m0, null, nullInitializerFactory) {
+            "EMP_20", false, 600, empTable, m0, null,
+            NullInitializerExpressionFactory.INSTANCE) {
           public RexNode getConstraint(RexBuilder rexBuilder,
               RelDataType tableRowType) {
             final RelDataTypeField deptnoField =
@@ -421,7 +402,7 @@ public class MockCatalogReader extends CalciteCatalogReader {
     MockTable empNullables20View =
         new MockViewTable(this, salesSchema.getCatalogName(), salesSchema.name,
             "EMPNULLABLES_20", false, 600, empNullablesTable, m0, null,
-            nullInitializerFactory) {
+            NullInitializerExpressionFactory.INSTANCE) {
           public RexNode getConstraint(RexBuilder rexBuilder,
               RelDataType tableRowType) {
             final RelDataTypeField deptnoField =
@@ -506,7 +487,8 @@ public class MockCatalogReader extends CalciteCatalogReader {
     MockTable struct10View =
         new MockViewTable(this, structTypeSchema.getCatalogName(),
             structTypeSchema.name, "T_10", false, 20, structTypeTable,
-            m1, structTypeTableResolver, nullInitializerFactory) {
+            m1, structTypeTableResolver,
+            NullInitializerExpressionFactory.INSTANCE) {
           @Override public RexNode getConstraint(RexBuilder rexBuilder,
               RelDataType tableRowType) {
             final RelDataTypeField c0Field =
@@ -813,7 +795,7 @@ public class MockCatalogReader extends CalciteCatalogReader {
         } else if (aClass.isInstance(MockTable.this)) {
           return aClass.cast(MockTable.this);
         }
-        return null;
+        return super.unwrap(aClass);
       }
 
       @Override public Table extend(final List<RelDataTypeField> fields) {
@@ -841,23 +823,6 @@ public class MockCatalogReader extends CalciteCatalogReader {
       };
     }
 
-    /**
-     * Subclass of ModifiableTable that also implements
-     * CustomColumnResolvingTable.
-     */
-    private class ModifiableTableWithCustomColumnResolving
-        extends ModifiableTable implements CustomColumnResolvingTable, Wrapper {
-
-      protected ModifiableTableWithCustomColumnResolving(String tableName) {
-        super(tableName);
-      }
-
-      @Override public List<Pair<RelDataTypeField, List<String>>> resolveColumn(
-          RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names) {
-        return resolver.resolveColumn(rowType, typeFactory, names);
-      }
-    }
-
     public static MockTable create(MockCatalogReader catalogReader,
         MockSchema schema, String name, boolean stream, double rowCount) {
       return create(catalogReader, schema, name, stream, rowCount, null);
@@ -867,7 +832,7 @@ public class MockCatalogReader extends CalciteCatalogReader {
         MockSchema schema, String name, boolean stream, double rowCount,
         ColumnResolver resolver) {
       return create(catalogReader, schema, name, stream, rowCount, resolver,
-          new NullInitializerExpressionFactory());
+          NullInitializerExpressionFactory.INSTANCE);
     }
 
     public static MockTable create(MockCatalogReader catalogReader,
@@ -885,6 +850,9 @@ public class MockCatalogReader extends CalciteCatalogReader {
       if (clazz.isInstance(this)) {
         return clazz.cast(this);
       }
+      if (clazz.isInstance(initializerFactory)) {
+        return clazz.cast(initializerFactory);
+      }
       if (clazz.isAssignableFrom(Table.class)) {
         final Table table = resolver == null
             ? new ModifiableTable(Util.last(names))
@@ -978,6 +946,24 @@ public class MockCatalogReader extends CalciteCatalogReader {
     public StructKind getKind() {
       return kind;
     }
+
+    /**
+     * Subclass of {@link ModifiableTable} that also implements
+     * {@link CustomColumnResolvingTable}.
+     */
+    private class ModifiableTableWithCustomColumnResolving
+        extends ModifiableTable implements CustomColumnResolvingTable, Wrapper {
+
+      ModifiableTableWithCustomColumnResolving(String tableName) {
+        super(tableName);
+      }
+
+      @Override public List<Pair<RelDataTypeField, List<String>>> resolveColumn(
+          RelDataType rowType, RelDataTypeFactory typeFactory,
+          List<String> names) {
+        return resolver.resolveColumn(rowType, typeFactory, names);
+      }
+    }
   }
 
   /**
@@ -1014,16 +1000,14 @@ public class MockCatalogReader extends CalciteCatalogReader {
         MockCatalogReader catalogReader, String catalogName, String schemaName, String name,
         boolean stream, double rowCount, ColumnResolver resolver) {
       final Table underlying = modifiableViewTable.unwrap(Table.class);
-      final InitializerExpressionFactory maybeInitializerExpressionFactory =
+      final InitializerExpressionFactory initializerExpressionFactory =
           underlying != null && underlying instanceof Wrapper
               ? ((Wrapper) underlying).unwrap(InitializerExpressionFactory.class)
-              : new NullInitializerExpressionFactory();
-      final InitializerExpressionFactory initializerExpressionFactory =
-          maybeInitializerExpressionFactory == null
-              ? new NullInitializerExpressionFactory()
-              : maybeInitializerExpressionFactory;
-      return new MockModifiableViewRelOptTable(modifiableViewTable, catalogReader, catalogName,
-          schemaName, name, stream, rowCount, resolver, initializerExpressionFactory);
+              : NullInitializerExpressionFactory.INSTANCE;
+      return new MockModifiableViewRelOptTable(modifiableViewTable,
+          catalogReader, catalogName, schemaName, name, stream, rowCount,
+          resolver, Util.first(initializerExpressionFactory,
+          NullInitializerExpressionFactory.INSTANCE));
     }
 
     public static MockViewTableMacro viewMacro(CalciteSchema schema, String viewSql,
@@ -1105,7 +1089,7 @@ public class MockCatalogReader extends CalciteCatalogReader {
     MockViewTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, double rowCount,
         MockTable fromTable, ImmutableIntList mapping, ColumnResolver resolver,
-        NullInitializerExpressionFactory initializerFactory) {
+        InitializerExpressionFactory initializerFactory) {
       super(catalogReader, catalogName, schemaName, name, stream, rowCount,
           resolver, initializerFactory);
       this.fromTable = fromTable;
@@ -1156,9 +1140,12 @@ public class MockCatalogReader extends CalciteCatalogReader {
 
       @Override public <C> C unwrap(Class<C> aClass) {
         if (table instanceof Wrapper) {
-          return ((Wrapper) table).unwrap(aClass);
+          final C c = ((Wrapper) table).unwrap(aClass);
+          if (c != null) {
+            return c;
+          }
         }
-        return null;
+        return super.unwrap(aClass);
       }
     }
 
@@ -1176,9 +1163,12 @@ public class MockCatalogReader extends CalciteCatalogReader {
 
       @Override public <C> C unwrap(Class<C> aClass) {
         if (table instanceof Wrapper) {
-          return ((Wrapper) table).unwrap(aClass);
+          final C c = ((Wrapper) table).unwrap(aClass);
+          if (c != null) {
+            return c;
+          }
         }
-        return null;
+        return super.unwrap(aClass);
       }
     }
 
@@ -1232,7 +1222,7 @@ public class MockCatalogReader extends CalciteCatalogReader {
     MockDynamicTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, double rowCount) {
       super(catalogReader, catalogName, schemaName, name, stream, rowCount,
-          null, new NullInitializerExpressionFactory());
+          null, NullInitializerExpressionFactory.INSTANCE);
     }
 
     public void onRegister(RelDataTypeFactory typeFactory) {
@@ -1569,6 +1559,40 @@ public class MockCatalogReader extends CalciteCatalogReader {
     }
   }
 
+  /** Default values for the "EMPDEFAULTS" table. */
+  private static class EmpInitializerExpressionFactory
+      extends NullInitializerExpressionFactory {
+    @Override public ColumnStrategy generationStrategy(RelOptTable table,
+        int iColumn) {
+      switch (iColumn) {
+      case 0:
+      case 1:
+      case 5:
+        return ColumnStrategy.DEFAULT;
+      default:
+        return super.generationStrategy(table, iColumn);
+      }
+    }
+
+    @Override public RexNode newColumnDefaultValue(RelOptTable table,
+        int iColumn, InitializerContext context) {
+      final RexBuilder rexBuilder = context.getRexBuilder();
+      final RelDataTypeFactory typeFactory = rexBuilder.getTypeFactory();
+      switch (iColumn) {
+      case 0:
+        return rexBuilder.makeExactLiteral(new BigDecimal(123),
+            typeFactory.createSqlType(SqlTypeName.INTEGER));
+      case 1:
+        return rexBuilder.makeLiteral("Bob");
+      case 5:
+        return rexBuilder.makeExactLiteral(new BigDecimal(555),
+            typeFactory.createSqlType(SqlTypeName.INTEGER));
+      default:
+        return rexBuilder.constantNull();
+      }
+    }
+  }
+
   /** Types used during initialization. */
   private class Fixture {
     final RelDataType intType =
@@ -1644,7 +1668,10 @@ public class MockCatalogReader extends CalciteCatalogReader {
   }
 
   /** To check whether
-   * {@link InitializerExpressionFactory#newColumnDefaultValue} is called. */
+   * {@link InitializerExpressionFactory#newColumnDefaultValue} is called.
+   *
+   * <p>If a column is in {@code defaultColumns}, returns 1 as the default
+   * value. */
   public static class CountingFactory extends NullInitializerExpressionFactory {
     static final ThreadLocal<AtomicInteger> THREAD_CALL_COUNT =
         new ThreadLocal<AtomicInteger>() {
@@ -1653,13 +1680,31 @@ public class MockCatalogReader extends CalciteCatalogReader {
           }
         };
 
-    CountingFactory(RelDataTypeFactory typeFactory) {
-      super();
+    private final List<String> defaultColumns;
+
+    CountingFactory(List<String> defaultColumns) {
+      this.defaultColumns = ImmutableList.copyOf(defaultColumns);
+    }
+
+    @Override public ColumnStrategy generationStrategy(RelOptTable table,
+        int iColumn) {
+      final RelDataTypeField field =
+          table.getRowType().getFieldList().get(iColumn);
+      if (defaultColumns.contains(field.getName())) {
+        return ColumnStrategy.DEFAULT;
+      }
+      return super.generationStrategy(table, iColumn);
     }
 
     @Override public RexNode newColumnDefaultValue(RelOptTable table,
         int iColumn, InitializerContext context) {
       THREAD_CALL_COUNT.get().incrementAndGet();
+      final RelDataTypeField field =
+          table.getRowType().getFieldList().get(iColumn);
+      if (defaultColumns.contains(field.getName())) {
+        final RexBuilder rexBuilder = context.getRexBuilder();
+        return rexBuilder.makeExactLiteral(BigDecimal.ONE);
+      }
       return super.newColumnDefaultValue(table, iColumn, context);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
index f30ebc8..b97318e 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -29,7 +29,6 @@ import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Closer;
-import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -60,26 +59,16 @@ import static org.junit.Assert.fail;
  * Test that runs every Quidem file as a test.
  */
 @RunWith(Parameterized.class)
-public class QuidemTest {
-  private final String path;
-  private final Method method;
+public abstract class QuidemTest {
+  protected final String path;
+  protected final Method method;
 
-  public QuidemTest(String path) {
+  /** Creates a QuidemTest. */
+  protected QuidemTest(String path) {
     this.path = path;
     this.method = findMethod(path);
   }
 
-  /** Runs a test from the command line.
-   *
-   * <p>For example:
-   *
-   * <blockquote><code>java QuidemTest sql/dummy.iq</code></blockquote> */
-  public static void main(String[] args) throws Exception {
-    for (String arg : args) {
-      new QuidemTest(arg).test();
-    }
-  }
-
   private Method findMethod(String path) {
     // E.g. path "sql/agg.iq" gives method "testSqlAgg"
     String methodName =
@@ -93,12 +82,7 @@ public class QuidemTest {
     return m;
   }
 
-  /** For {@link org.junit.runners.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/agg.iq";
+  protected static Collection<Object[]> data(String first) {
     // inUrl = "file:/home/fred/calcite/core/target/test-classes/sql/agg.iq"
     final URL inUrl = JdbcTest.class.getResource("/" + first);
     String x = inUrl.getFile();
@@ -124,7 +108,7 @@ public class QuidemTest {
     });
   }
 
-  private void checkRun(String path) throws Exception {
+  protected void checkRun(String path) throws Exception {
     final File inFile;
     final File outFile;
     final File f = new File(path);
@@ -150,7 +134,7 @@ public class QuidemTest {
     try (final Reader reader = Util.reader(inFile);
          final Writer writer = Util.printWriter(outFile);
          final Closer closer = new Closer()) {
-      new Quidem(reader, writer, env(), new QuidemConnectionFactory())
+      new Quidem(reader, writer, env(), createConnectionFactory())
           .withPropertyHandler(new Quidem.PropertyHandler() {
             public void onSet(String propertyName, Object value) {
               if (propertyName.equals("bindable")) {
@@ -174,6 +158,11 @@ public class QuidemTest {
     }
   }
 
+  /** Creates a connection factory. */
+  protected Quidem.ConnectionFactory createConnectionFactory() {
+    return new QuidemConnectionFactory();
+  }
+
   /** Converts a path from Unix to native. On Windows, converts
    * forward-slashes to back-slashes; on Linux, does nothing. */
   private static String u2n(String s) {
@@ -221,37 +210,8 @@ public class QuidemTest {
     }
   }
 
-  /** Override settings for "sql/misc.iq". */
-  public void testSqlMisc() throws Exception {
-    switch (CalciteAssert.DB) {
-    case ORACLE:
-      // There are formatting differences (e.g. "4.000" vs "4") when using
-      // Oracle as the JDBC data source.
-      return;
-    }
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
-      checkRun(path);
-    }
-  }
-
-  /** Override settings for "sql/scalar.iq". */
-  public void testSqlScalar() throws Exception {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
-      checkRun(path);
-    }
-  }
-
-  /** Runs the dummy script "sql/dummy.iq", which is checked in empty but
-   * which you may use as scratch space during development. */
-  // Do not add disable this test; just remember not to commit changes to dummy.iq
-  public void testSqlDummy() throws Exception {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
-      checkRun(path);
-    }
-  }
-
   /** Quidem connection factory for Calcite's built-in test schemas. */
-  private static class QuidemConnectionFactory
+  protected static class QuidemConnectionFactory
       implements Quidem.ConnectionFactory {
     public Connection connect(String name) throws Exception {
       return connect(name, false);

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 67da90b..3f1158f 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -114,8 +114,6 @@ import java.util.List;
 
 import static org.junit.Assert.assertTrue;
 
-
-
 /**
  * Unit test for rules in {@code org.apache.calcite.rel} and subpackages.
  *

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index a6b8db4..c8ece76 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -39,6 +39,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.ColumnStrategy;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -403,6 +404,10 @@ public abstract class SqlToRelTestBase {
         return ImmutableList.of();
       }
 
+      public List<ColumnStrategy> getColumnStrategies() {
+        throw new UnsupportedOperationException();
+      }
+
       public Expression getExpression(Class clazz) {
         return null;
       }
@@ -476,6 +481,10 @@ public abstract class SqlToRelTestBase {
     public List<RelReferentialConstraint> getReferentialConstraints() {
       return parent.getReferentialConstraints();
     }
+
+    public List<ColumnStrategy> getColumnStrategies() {
+      return parent.getColumnStrategies();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 49de12d..c7378a3 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -8712,9 +8712,22 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "  timestamp '1970-01-01 00:00:00', 1, 1, 1)";
     pragmaticTester.checkQueryFails(sql3,
         "Column 'SLACKER' has no default value and does not allow NULLs");
-    assertThat("Missing non-NULL column generates a call to factory",
+    assertThat("Should not check for default value, even if if column is missing"
+            + "from INSERT and nullable",
         MockCatalogReader.CountingFactory.THREAD_CALL_COUNT.get().get(),
-        is(c + 1));
+        is(c));
+
+    // Now remove DEPTNO, which has a default value, from the target list.
+    // Will generate an extra call to newColumnDefaultValue at sql-to-rel time,
+    // just not yet.
+    final String sql4 = "insert into ^emp^ (empno, ename, job, mgr, hiredate,\n"
+        + "  sal, comm, slacker)\n"
+        + "values(1, 'nom', 'job', 0,\n"
+        + "  timestamp '1970-01-01 00:00:00', 1, 1, false)";
+    pragmaticTester.checkQuery(sql4);
+    assertThat("Missing DEFAULT column generates a call to factory",
+        MockCatalogReader.CountingFactory.THREAD_CALL_COUNT.get().get(),
+        is(c));
   }
 
   @Test public void testInsertView() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/core/src/test/resources/sql/blank.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/blank.iq b/core/src/test/resources/sql/blank.iq
index 6375030..bcaa9e8 100644
--- a/core/src/test/resources/sql/blank.iq
+++ b/core/src/test/resources/sql/blank.iq
@@ -19,7 +19,7 @@
 !set outputformat mysql
 
 create table foo (i int not null, j int);
-(-1 rows modified)
+(0 rows modified)
 
 !update
 insert into foo values (1, 0);
@@ -55,11 +55,11 @@ select * from foo as f where i in (
 
 # [CALCITE-1493] Wrong plan for NOT IN correlated queries
 create table table1(i int, j int);
-(-1 rows modified)
+(0 rows modified)
 
 !update
 create table table2(i int, j int);
-(-1 rows modified)
+(0 rows modified)
 
 !update
 insert into table1 values (1, 2), (1, 3);

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
index 9cd05c8..fde5927 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
@@ -28,7 +28,6 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.runtime.ConsList;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
@@ -51,6 +50,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.calcite.util.Static.cons;
+
 /**
  * Table based on a MongoDB collection.
  */
@@ -214,7 +215,7 @@ public class MongoTable extends AbstractQueryableTable
    */
   private AggregationOutput aggregateOldWay(DBCollection dbCollection,
        DBObject first, List<DBObject> rest) {
-    return dbCollection.aggregate(ConsList.of(first, rest));
+    return dbCollection.aggregate(cons(first, rest));
   }
 
   /** Implementation of {@link org.apache.calcite.linq4j.Queryable} based on

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 36ad85e..1feec71 100644
--- a/pom.xml
+++ b/pom.xml
@@ -156,6 +156,7 @@ limitations under the License.
     <module>pig</module>
     <module>piglet</module>
     <module>plus</module>
+    <module>server</module>
     <module>spark</module>
     <module>splunk</module>
     <module>ubenchmark</module>
@@ -655,9 +656,9 @@ limitations under the License.
         <artifactId>maven-javadoc-plugin</artifactId>
         <configuration>
           <links>
-            <link>https://docs.oracle.com/javase/8/docs/api/</link>
+            <link>https://docs.oracle.com/javase/9/docs/api/</link>
           </links>
-          <excludePackageNames>org.apache.calcite.benchmarks.generated,org.apache.calcite.sql.parser.impl,org.apache.calcite.sql.parser.parserextensiontesting,org.apache.calcite.piglet.parser,org.openjdk.jmh</excludePackageNames>
+          <excludePackageNames>org.apache.calcite.benchmarks.generated,org.apache.calcite.sql.parser.ddl,org.apache.calcite.sql.parser.impl,org.apache.calcite.sql.parser.parserextensiontesting,org.apache.calcite.piglet.parser,org.openjdk.jmh</excludePackageNames>
           <show>private</show>
         </configuration>
       </plugin>
@@ -869,7 +870,7 @@ limitations under the License.
           <links>
             <link>https://docs.oracle.com/javase/8/docs/api/</link>
           </links>
-          <excludePackageNames>org.apache.calcite.benchmarks.generated,org.apache.calcite.sql.parser.impl,org.apache.calcite.sql.parser.parserextensiontesting,org.apache.calcite.piglet.parser,org.openjdk.jmh</excludePackageNames>
+          <excludePackageNames>org.apache.calcite.benchmarks.generated,org.apache.calcite.sql.parser.ddl,org.apache.calcite.sql.parser.impl,org.apache.calcite.sql.parser.parserextensiontesting,org.apache.calcite.piglet.parser,org.openjdk.jmh</excludePackageNames>
           <notimestamp>true</notimestamp>
           <windowtitle>Apache Calcite API</windowtitle>
         </configuration>

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
new file mode 100644
index 0000000..562f8f4
--- /dev/null
+++ b/server/pom.xml
@@ -0,0 +1,245 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.calcite</groupId>
+    <artifactId>calcite</artifactId>
+    <version>1.15.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>calcite-server</artifactId>
+  <packaging>jar</packaging>
+  <version>1.15.0-SNAPSHOT</version>
+  <name>Calcite Server</name>
+  <description>Calcite Server</description>
+
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
+  <dependencies>
+    <!-- Sorted by groupId, artifactId; calcite dependencies first. Put versions
+         in dependencyManagement in the root POM, not here. -->
+    <dependency>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <type>jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-linq4j</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>net.hydromatic</groupId>
+      <artifactId>quidem</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>net.hydromatic</groupId>
+      <artifactId>scott-data-hsqldb</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.incava</groupId>
+      <artifactId>java-diff</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hsqldb</groupId>
+      <artifactId>hsqldb</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <!-- Sorted by groupId, artifactId. Put versions in
+           pluginManagement in the root POM, not here. -->
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>${maven-dependency-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>net.hydromatic:scott-data-hsqldb</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.hsqldb:hsqldb</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.incava:java-diff</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-api</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-log4j12</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-fmpp-resources</id>
+            <phase>initialize</phase>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/codegen</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>src/main/codegen</directory>
+                  <filtering>false</filtering>
+                </resource>
+              </resources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-release-plugin</artifactId>
+      </plugin>
+      <!-- Parent module has the same plugin and does the work of
+           generating -sources.jar for each project. But without the
+           plugin declared here, IDEs don't know the sources are
+           available. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>attach-sources</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>jar-no-fork</goal>
+              <goal>test-jar-no-fork</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>javacc-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>javacc</id>
+            <goals>
+              <goal>javacc</goal>
+            </goals>
+            <configuration>
+              <sourceDirectory>${project.build.directory}/generated-sources/fmpp</sourceDirectory>
+              <outputDirectory>${project.build.directory}/generated-sources/javacc</outputDirectory>
+              <includes>
+                <include>**/Parser.jj</include>
+              </includes>
+              <lookAhead>2</lookAhead>
+              <isStatic>false</isStatic>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <!-- CALCITE-538: workaround for https://github.com/freemarker/fmpp/issues/11
+        FMPP always overwrites destination file, however we do not want
+        recompile the whole module every time.
+      -->
+      <id>generate-parser</id>
+      <activation>
+        <property>
+          <name>!skipGenerate</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+            <artifactId>fmpp-maven-plugin</artifactId>
+            <executions>
+              <execution>
+                <configuration>
+                  <cfgFile>src/main/codegen/config.fmpp</cfgFile>
+                  <outputDirectory>${project.build.directory}/generated-sources/fmpp</outputDirectory>
+                  <templateDirectory>${top.dir}/core/src/main/codegen/templates</templateDirectory>
+                </configuration>
+                <id>generate-fmpp-sources</id>
+                <phase>validate</phase>
+                <goals>
+                  <goal>generate</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/server/src/main/codegen/config.fmpp b/server/src/main/codegen/config.fmpp
new file mode 100644
index 0000000..08b715a
--- /dev/null
+++ b/server/src/main/codegen/config.fmpp
@@ -0,0 +1,99 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to you under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+data: {
+    parser: {
+      # Generated parser implementation class package and name
+      package: "org.apache.calcite.sql.parser.ddl",
+      class: "SqlDdlParserImpl",
+
+      # List of import statements.
+      imports: [
+        "org.apache.calcite.schema.ColumnStrategy"
+        "org.apache.calcite.sql.SqlCreate"
+        "org.apache.calcite.sql.SqlDrop"
+        "org.apache.calcite.sql.ddl.SqlDdlNodes"
+      ]
+
+      # List of keywords.
+      keywords: [
+        "IF"
+        "MATERIALIZED"
+        "STORED"
+        "VIRTUAL"
+      ]
+
+      # List of keywords from "keywords" section that are not reserved.
+      nonReservedKeywords: [
+        "IF"
+        "MATERIALIZED"
+        "STORED"
+        "VIRTUAL"
+      ]
+
+      # List of methods for parsing custom SQL statements.
+      statementParserMethods: [
+      ]
+
+      # List of methods for parsing custom literals.
+      # Example: ParseJsonLiteral().
+      literalParserMethods: [
+      ]
+
+      # List of methods for parsing custom data types.
+      dataTypeParserMethods: [
+      ]
+
+      # List of methods for parsing extensions to "ALTER <scope>" calls.
+      # Each must accept arguments "(SqlParserPos pos, String scope)".
+      alterStatementParserMethods: [
+      ]
+
+      # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+      # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+      createStatementParserMethods: [
+        "SqlCreateForeignSchema"
+        "SqlCreateMaterializedView"
+        "SqlCreateSchema"
+        "SqlCreateTable"
+        "SqlCreateView"
+      ]
+
+      # List of methods for parsing extensions to "DROP" calls.
+      # Each must accept arguments "(SqlParserPos pos)".
+      dropStatementParserMethods: [
+        "SqlDropMaterializedView"
+        "SqlDropSchema"
+        "SqlDropTable"
+        "SqlDropView"
+      ]
+
+      # List of files in @includes directory that have parser method
+      # implementations for parsing custom SQL statements, literals or types
+      # given as part of "statementParserMethods", "literalParserMethods" or
+      # "dataTypeParserMethods".
+      implementationFiles: [
+        "parserImpls.ftl"
+      ]
+
+      includeCompoundIdentifier: true
+      includeBraces: true
+      includeAdditionalDeclarations: false
+
+    }
+}
+freemarkerLinks: {
+    includes: includes/
+}

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/server/src/main/codegen/includes/parserImpls.ftl b/server/src/main/codegen/includes/parserImpls.ftl
new file mode 100644
index 0000000..9c54540
--- /dev/null
+++ b/server/src/main/codegen/includes/parserImpls.ftl
@@ -0,0 +1,293 @@
+<#--
+// 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.
+-->
+
+boolean IfNotExistsOpt() :
+{
+}
+{
+    <IF> <NOT> <EXISTS> { return true; }
+|
+    { return false; }
+}
+
+boolean IfExistsOpt() :
+{
+}
+{
+    <IF> <EXISTS> { return true; }
+|
+    { return false; }
+}
+
+SqlCreate SqlCreateSchema(Span s, boolean replace) :
+{
+    final boolean ifNotExists;
+    final SqlIdentifier id;
+}
+{
+    <SCHEMA> ifNotExists = IfNotExistsOpt() id = CompoundIdentifier()
+    {
+        return SqlDdlNodes.createSchema(s.end(this), replace, ifNotExists, id);
+    }
+}
+
+SqlCreate SqlCreateForeignSchema(Span s, boolean replace) :
+{
+    final boolean ifNotExists;
+    final SqlIdentifier id;
+    SqlNode type = null;
+    SqlNode library = null;
+    SqlNodeList optionList = null;
+}
+{
+    <FOREIGN> <SCHEMA> ifNotExists = IfNotExistsOpt() id = CompoundIdentifier()
+    (
+         <TYPE> type = StringLiteral()
+    |
+         <LIBRARY> library = StringLiteral()
+    )
+    [ optionList = Options() ]
+    {
+        return SqlDdlNodes.createForeignSchema(s.end(this), replace,
+            ifNotExists, id, type, library, optionList);
+    }
+}
+
+SqlNodeList Options() :
+{
+    final Span s;
+    final List<SqlNode> list = Lists.newArrayList();
+}
+{
+    <OPTIONS> { s = span(); } <LPAREN>
+    [
+        Option(list)
+        (
+            <COMMA>
+            Option(list)
+        )*
+    ]
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+void Option(List<SqlNode> list) :
+{
+    final SqlIdentifier id;
+    final SqlNode value;
+}
+{
+    id = SimpleIdentifier()
+    value = Literal() {
+        list.add(id);
+        list.add(value);
+    }
+}
+
+SqlNodeList TableElementList() :
+{
+    final Span s;
+    final List<SqlNode> list = Lists.newArrayList();
+}
+{
+    <LPAREN> { s = span(); }
+    TableElement(list)
+    (
+        <COMMA> TableElement(list)
+    )*
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+void TableElement(List<SqlNode> list) :
+{
+    final SqlIdentifier id;
+    final SqlDataTypeSpec type;
+    final boolean nullable;
+    final SqlNode e;
+    final SqlNode constraint;
+    SqlIdentifier name = null;
+    final SqlNodeList columnList;
+    final Span s = Span.of();
+    final ColumnStrategy strategy;
+}
+{
+    id = SimpleIdentifier()
+    (
+        type = DataType()
+        (
+            <NULL> { nullable = true; }
+        |
+            <NOT> <NULL> { nullable = false; }
+        |
+            { nullable = true; }
+        )
+        (
+            [ <GENERATED> <ALWAYS> ] <AS> <LPAREN>
+            e = Expression(ExprContext.ACCEPT_SUB_QUERY) <RPAREN>
+            (
+                <VIRTUAL> { strategy = ColumnStrategy.VIRTUAL; }
+            |
+                <STORED> { strategy = ColumnStrategy.STORED; }
+            |
+                { strategy = ColumnStrategy.VIRTUAL; }
+            )
+        |
+            <DEFAULT_> e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+                strategy = ColumnStrategy.DEFAULT;
+            }
+        |
+            {
+                e = null;
+                strategy = nullable ? ColumnStrategy.NULLABLE
+                    : ColumnStrategy.NOT_NULLABLE;
+            }
+        )
+        {
+            list.add(
+                SqlDdlNodes.column(s.add(id).end(this), id,
+                    type.withNullable(nullable), e, strategy));
+        }
+    |
+        { list.add(id); }
+    )
+|
+    id = SimpleIdentifier() {
+        list.add(id);
+    }
+|
+    [ <CONSTRAINT> { s.add(this); } name = SimpleIdentifier() ]
+    (
+        <CHECK> { s.add(this); } <LPAREN>
+        e = Expression(ExprContext.ACCEPT_SUB_QUERY) <RPAREN> {
+            list.add(SqlDdlNodes.check(s.end(this), name, e));
+        }
+    |
+        <UNIQUE> { s.add(this); }
+        columnList = ParenthesizedSimpleIdentifierList() {
+            list.add(SqlDdlNodes.unique(s.end(columnList), name, columnList));
+        }
+    |
+        <PRIMARY>  { s.add(this); } <KEY>
+        columnList = ParenthesizedSimpleIdentifierList() {
+            list.add(SqlDdlNodes.primary(s.end(columnList), name, columnList));
+        }
+    )
+}
+
+SqlCreate SqlCreateTable(Span s, boolean replace) :
+{
+    final boolean ifNotExists;
+    final SqlIdentifier id;
+    SqlNodeList tableElementList = null;
+    SqlNode query = null;
+}
+{
+    <TABLE> ifNotExists = IfNotExistsOpt() id = CompoundIdentifier()
+    [ tableElementList = TableElementList() ]
+    [ <AS> query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) ]
+    {
+        return SqlDdlNodes.createTable(s.end(this), replace, ifNotExists, id,
+            tableElementList, query);
+    }
+}
+
+SqlCreate SqlCreateView(Span s, boolean replace) :
+{
+    final SqlIdentifier id;
+    SqlNodeList columnList = null;
+    final SqlNode query;
+}
+{
+    <VIEW> id = CompoundIdentifier()
+    [ columnList = ParenthesizedSimpleIdentifierList() ]
+    <AS> query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) {
+        return SqlDdlNodes.createView(s.end(this), replace, id, columnList,
+            query);
+    }
+}
+
+SqlCreate SqlCreateMaterializedView(Span s, boolean replace) :
+{
+    final boolean ifNotExists;
+    final SqlIdentifier id;
+    SqlNodeList columnList = null;
+    final SqlNode query;
+}
+{
+    <MATERIALIZED> <VIEW> ifNotExists = IfNotExistsOpt()
+    id = CompoundIdentifier()
+    [ columnList = ParenthesizedSimpleIdentifierList() ]
+    <AS> query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) {
+        return SqlDdlNodes.createMaterializedView(s.end(this), replace,
+            ifNotExists, id, columnList, query);
+    }
+}
+
+SqlDrop SqlDropSchema(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier id;
+    final boolean foreign;
+}
+{
+    (
+        <FOREIGN> { foreign = true; }
+    |
+        { foreign = false; }
+    )
+    <SCHEMA> ifExists = IfExistsOpt() id = CompoundIdentifier() {
+        return SqlDdlNodes.dropSchema(s.end(this), foreign, ifExists, id);
+    }
+}
+
+SqlDrop SqlDropTable(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier id;
+}
+{
+    <TABLE> ifExists = IfExistsOpt() id = CompoundIdentifier() {
+        return SqlDdlNodes.dropTable(s.end(this), ifExists, id);
+    }
+}
+
+SqlDrop SqlDropView(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier id;
+}
+{
+    <VIEW> ifExists = IfExistsOpt() id = CompoundIdentifier() {
+        return SqlDdlNodes.dropView(s.end(this), ifExists, id);
+    }
+}
+
+SqlDrop SqlDropMaterializedView(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier id;
+}
+{
+    <MATERIALIZED> <VIEW> ifExists = IfExistsOpt() id = CompoundIdentifier() {
+        return SqlDdlNodes.dropMaterializedView(s.end(this), ifExists, id);
+    }
+}
+
+// End parserImpls.ftl

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlCheckConstraint.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCheckConstraint.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCheckConstraint.java
new file mode 100644
index 0000000..b80e092
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCheckConstraint.java
@@ -0,0 +1,75 @@
+/*
+ * 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.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 SqlCheckConstraint extends SqlCall {
+  private static final SqlSpecialOperator OPERATOR =
+      new SqlSpecialOperator("CHECK", SqlKind.CHECK);
+
+  private final SqlIdentifier name;
+  private final SqlNode expression;
+
+  /** Creates a SqlCheckConstraint; use {@link SqlDdlNodes#check}. */
+  SqlCheckConstraint(SqlParserPos pos, SqlIdentifier name,
+      SqlNode expression) {
+    super(pos);
+    this.name = name; // may be null
+    this.expression = expression;
+  }
+
+  @Override public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.of(name, expression);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    if (name != null) {
+      writer.keyword("CONSTRAINT");
+      name.unparse(writer, 0, 0);
+    }
+    writer.keyword("CHECK");
+    if (writer.isAlwaysUseParentheses()) {
+      expression.unparse(writer, 0, 0);
+    } else {
+      writer.sep("(");
+      expression.unparse(writer, 0, 0);
+      writer.sep(")");
+    }
+  }
+}
+
+// End SqlCheckConstraint.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java
new file mode 100644
index 0000000..95ff82c
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlColumnDeclaration.java
@@ -0,0 +1,102 @@
+/*
+ * 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.schema.ColumnStrategy;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Parse tree for {@code UNIQUE}, {@code PRIMARY KEY} constraints.
+ *
+ * <p>And {@code FOREIGN KEY}, when we support it.
+ */
+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;
+
+  /** Creates a SqlColumnDeclaration; use {@link SqlDdlNodes#column}. */
+  SqlColumnDeclaration(SqlParserPos pos, SqlIdentifier name,
+      SqlDataTypeSpec dataType, SqlNode expression,
+      ColumnStrategy strategy) {
+    super(pos);
+    this.name = name;
+    this.dataType = dataType;
+    this.expression = expression;
+    this.strategy = strategy;
+  }
+
+  @Override public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return ImmutableList.of(name, dataType);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    name.unparse(writer, 0, 0);
+    dataType.unparse(writer, 0, 0);
+    if (dataType.getNullable() != null && !dataType.getNullable()) {
+      writer.keyword("NOT NULL");
+    }
+    if (expression != null) {
+      switch (strategy) {
+      case VIRTUAL:
+      case STORED:
+        writer.keyword("AS");
+        exp(writer);
+        writer.keyword(strategy.name());
+        break;
+      case DEFAULT:
+        writer.keyword("DEFAULT");
+        exp(writer);
+        break;
+      default:
+        throw new AssertionError("unexpected: " + strategy);
+      }
+    }
+  }
+
+  private void exp(SqlWriter writer) {
+    if (writer.isAlwaysUseParentheses()) {
+      expression.unparse(writer, 0, 0);
+    } else {
+      writer.sep("(");
+      expression.unparse(writer, 0, 0);
+      writer.sep(")");
+    }
+  }
+}
+
+// End SqlColumnDeclaration.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java
new file mode 100644
index 0000000..96ce3f9
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateForeignSchema.java
@@ -0,0 +1,191 @@
+/*
+ * 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.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 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;
+  private final SqlNodeList optionList;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("CREATE FOREIGN SCHEMA",
+          SqlKind.CREATE_FOREIGN_SCHEMA);
+
+  /** Creates a SqlCreateForeignSchema. */
+  SqlCreateForeignSchema(SqlParserPos pos, boolean replace, boolean ifNotExists,
+      SqlIdentifier name, SqlNode type, SqlNode library,
+      SqlNodeList optionList) {
+    super(OPERATOR, pos, replace, ifNotExists);
+    this.name = Preconditions.checkNotNull(name);
+    this.type = type;
+    this.library = library;
+    Preconditions.checkArgument((type == null) != (library == null),
+        "of type and library, exactly one must be specified");
+    this.optionList = optionList; // may be null
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.of(name, type, library, optionList);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    if (getReplace()) {
+      writer.keyword("CREATE OR REPLACE");
+    } else {
+      writer.keyword("CREATE");
+    }
+    writer.keyword("FOREIGN SCHEMA");
+    if (ifNotExists) {
+      writer.keyword("IF NOT EXISTS");
+    }
+    name.unparse(writer, leftPrec, rightPrec);
+    if (library != null) {
+      writer.keyword("LIBRARY");
+      library.unparse(writer, 0, 0);
+    }
+    if (type != null) {
+      writer.keyword("TYPE");
+      type.unparse(writer, 0, 0);
+    }
+    if (optionList != null) {
+      writer.keyword("OPTIONS");
+      SqlWriter.Frame frame = writer.startList("(", ")");
+      int i = 0;
+      for (Pair<SqlIdentifier, SqlNode> c : options(optionList)) {
+        if (i++ > 0) {
+          writer.sep(",");
+        }
+        c.left.unparse(writer, 0, 0);
+        c.right.unparse(writer, 0, 0);
+      }
+      writer.endList(frame);
+    }
+  }
+
+  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;
+  }
+
+  private static List<Pair<SqlIdentifier, SqlNode>> options(
+      final SqlNodeList optionList) {
+    return new AbstractList<Pair<SqlIdentifier, SqlNode>>() {
+      public Pair<SqlIdentifier, SqlNode> get(int index) {
+        return Pair.of((SqlIdentifier) optionList.get(index * 2),
+            optionList.get(index * 2 + 1));
+      }
+
+      public int size() {
+        return optionList.size() / 2;
+      }
+    };
+  }
+}
+
+// End SqlCreateForeignSchema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java
new file mode 100644
index 0000000..9e32429
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateMaterializedView.java
@@ -0,0 +1,157 @@
+/*
+ * 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.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.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 MATERIALIZED VIEW} statement.
+ */
+public class SqlCreateMaterializedView extends SqlCreate
+    implements SqlExecutableStatement {
+  private final SqlIdentifier name;
+  private final SqlNodeList columnList;
+  private final SqlNode query;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("CREATE MATERIALIZED VIEW",
+          SqlKind.CREATE_MATERIALIZED_VIEW);
+
+  /** Creates a SqlCreateView. */
+  SqlCreateMaterializedView(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 = Preconditions.checkNotNull(query);
+  }
+
+  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("MATERIALIZED VIEW");
+    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);
+    }
+    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);
+    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);
+    }
+  }
+}
+
+// End SqlCreateMaterializedView.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/238b3225/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java
new file mode 100644
index 0000000..84eab77
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateSchema.java
@@ -0,0 +1,92 @@
+/*
+ * 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.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 com.google.common.base.Preconditions;
+
+import java.util.List;
+
+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;
+
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("CREATE SCHEMA", SqlKind.CREATE_SCHEMA);
+
+  /** Creates a SqlCreateSchema. */
+  SqlCreateSchema(SqlParserPos pos, boolean replace, boolean ifNotExists,
+      SqlIdentifier name) {
+    super(OPERATOR, pos, replace, ifNotExists);
+    this.name = Preconditions.checkNotNull(name);
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.<SqlNode>of(name);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    if (getReplace()) {
+      writer.keyword("CREATE OR REPLACE");
+    } else {
+      writer.keyword("CREATE");
+    }
+    writer.keyword("SCHEMA");
+    if (ifNotExists) {
+      writer.keyword("IF NOT EXISTS");
+    }
+    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);
+  }
+}
+
+// End SqlCreateSchema.java