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 2015/05/13 21:06:32 UTC

[01/13] incubator-calcite git commit: [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING clause gives AssertionError

Repository: incubator-calcite
Updated Branches:
  refs/heads/master 61ba314f1 -> bc45a2c04


[CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING clause gives AssertionError

Based on work by Sean Hsuan-Yi Chu.


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

Branch: refs/heads/master
Commit: f98d567fa6843762a830a272e06f2d74908ce440
Parents: 61ba314
Author: Julian Hyde <jh...@apache.org>
Authored: Fri May 8 12:36:25 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri May 8 12:36:25 2015 -0700

----------------------------------------------------------------------
 .../calcite/sql/validate/SqlValidator.java      |  1 +
 .../calcite/sql2rel/SqlToRelConverter.java      | 41 ++++++++++++------
 .../calcite/test/SqlToRelConverterTest.java     | 28 ++++++++++++
 .../calcite/test/SqlToRelConverterTest.xml      | 43 +++++++++++++++++++
 core/src/test/resources/sql/subquery.oq         | 45 ++++++++++++++++++++
 5 files changed, 145 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f98d567f/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index bd6d0b6..7e44b15 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -346,6 +346,7 @@ public interface SqlValidator {
    * @param selectNode Expression in SELECT clause
    * @return whether expression is an aggregate function
    */
+  @Deprecated // to be removed before 2.0
   boolean isAggregate(SqlNode selectNode);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f98d567f/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 07ec53e..5dad7f4 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2545,23 +2545,16 @@ public class SqlToRelConverter {
       SqlNodeList groupList,
       SqlNode having,
       List<SqlNode> orderExprList) {
-    SqlNodeList aggList = new SqlNodeList(SqlParserPos.ZERO);
-
-    for (SqlNode selectNode : selectList) {
-      if (validator.isAggregate(selectNode)) {
-        aggList.add(selectNode);
-      }
-    }
-
-    // The aggregate functions in having clause are also needed
-    // to be added to aggList to replace subqueries
-    if (having != null && validator.isAggregate(having)) {
-      aggList.add(having);
+    // Find aggregate functions in SELECT and HAVING clause
+    final AggregateFinder aggregateFinder = new AggregateFinder();
+    selectList.accept(aggregateFinder);
+    if (having != null) {
+      having.accept(aggregateFinder);
     }
 
     // first replace the subqueries inside the aggregates
     // because they will provide input rows to the aggregates.
-    replaceSubqueries(bb, aggList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+    replaceSubqueries(bb, aggregateFinder.list, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
 
     // If group-by clause is missing, pretend that it has zero elements.
     if (groupList == null) {
@@ -4931,6 +4924,28 @@ public class SqlToRelConverter {
       this.logic = logic;
     }
   }
+
+  /**
+   * Visitor that collects all aggregate functions in a {@link SqlNode} tree.
+   */
+  private static class AggregateFinder extends SqlBasicVisitor<Void> {
+    final SqlNodeList list = new SqlNodeList(SqlParserPos.ZERO);
+
+    @Override public Void visit(SqlCall call) {
+      if (call.getOperator().isAggregator()) {
+        list.add(call);
+        return null;
+      }
+
+      // Don't traverse into sub-queries, even if they contain aggregate
+      // functions.
+      if (call instanceof SqlSelect) {
+        return null;
+      }
+
+      return call.getOperator().acceptCall(this, call);
+    }
+  }
 }
 
 // End SqlToRelConverter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f98d567f/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 711895d..1853244 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1225,6 +1225,34 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   /**
+   * Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-716">[CALCITE-716]
+   * Scalar sub-query and aggregate function in SELECT or HAVING clause gives
+   * AssertionError</a>; variant involving HAVING clause.
+   */
+  @Test public void testAggregateAndScalarSubQueryInHaving() {
+    sql("select deptno\n"
+            + "from emp\n"
+            + "group by deptno\n"
+            + "having max(emp.empno) > (SELECT min(emp.empno) FROM emp)\n")
+        .convertsTo("${plan}");
+  }
+
+  /**
+   * Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-716">[CALCITE-716]
+   * Scalar sub-query and aggregate function in SELECT or HAVING clause gives
+   * AssertionError</a>; variant involving SELECT clause.
+   */
+  @Test public void testAggregateAndScalarSubQueryInSelect() {
+    sql("select deptno,\n"
+            + "  max(emp.empno) > (SELECT min(emp.empno) FROM emp) as b\n"
+            + "from emp\n"
+            + "group by deptno\n")
+        .convertsTo("${plan}");
+  }
+
+  /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *
    * @see RelNode#isValid(boolean)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f98d567f/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 9897fa9..ef17e98 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2535,4 +2535,47 @@ LogicalProject(SAL=[$0])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testAggregateAndScalarSubQueryInHaving">
+        <Resource name="sql">
+            <![CDATA[select deptno
+from emp
+group by deptno
+having max(emp.empno) > (SELECT min(emp.empno) FROM emp)
+]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0])
+  LogicalFilter(condition=[>($1, $2)])
+    LogicalJoin(condition=[true], joinType=[left])
+      LogicalAggregate(group=[{0}], agg#0=[MAX($1)])
+        LogicalProject(DEPTNO=[$7], EMPNO=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], EXPR$0=[MIN($0)])
+        LogicalProject(EMPNO=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testAggregateAndScalarSubQueryInSelect">
+        <Resource name="sql">
+            <![CDATA[select deptno,
+  max(emp.empno) > (SELECT min(emp.empno) FROM emp) as b
+from emp
+group by deptno
+]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], B=[>($1, $2)])
+  LogicalJoin(condition=[true], joinType=[left])
+    LogicalAggregate(group=[{0}], agg#0=[MAX($1)])
+      LogicalProject(DEPTNO=[$7], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{}], EXPR$0=[MIN($0)])
+      LogicalProject(EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f98d567f/core/src/test/resources/sql/subquery.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.oq b/core/src/test/resources/sql/subquery.oq
index db345a5..72349a6 100644
--- a/core/src/test/resources/sql/subquery.oq
+++ b/core/src/test/resources/sql/subquery.oq
@@ -230,4 +230,49 @@ where (gender, deptno) in (select gender, 10 from emp where gender = 'F');
 
 !ok
 
+!use scott
+
+# [CALCITE-694] Scan HAVING clause for sub-queries and IN-lists
+SELECT count(*) AS c
+FROM "scott".emp
+GROUP BY emp.deptno
+HAVING sum(case when emp.empno in (7369, 7839, 7902) then emp.sal else 0 end)
+     BETWEEN 5000.0 AND 10000.0;
+ C
+---
+ 3
+(1 row)
+
+!ok
+
+# [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING
+# clause gives AssertionError
+SELECT emp.deptno
+FROM "scott".emp
+GROUP BY emp.deptno
+HAVING max(emp.empno) > (SELECT min(emp.empno) FROM "scott".emp);
+ DEPTNO
+--------
+     10
+     20
+     30
+(3 rows)
+
+!ok
+
+# [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING
+# clause gives AssertionError
+SELECT emp.deptno,
+  max(emp.empno) > (SELECT min(emp.empno) FROM "scott".emp) as bbbb
+FROM "scott".emp
+GROUP BY emp.deptno;
+ DEPTNO | BBBB
+--------+------
+     10 | true
+     20 | true
+     30 | true
+(3 rows)
+
+!ok
+
 # End subquery.oq


[13/13] incubator-calcite git commit: [CALCITE-505] Support modifiable view

Posted by jh...@apache.org.
[CALCITE-505] Support modifiable view


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

Branch: refs/heads/master
Commit: bc45a2c047349826e22641e82468a2d4dad948e3
Parents: aee32bc
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Mar 19 09:56:09 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue May 12 14:18:10 2015 -0700

----------------------------------------------------------------------
 .../adapter/enumerable/RexToLixTranslator.java  |   3 +
 .../org/apache/calcite/jdbc/CalcitePrepare.java |  61 +++--
 .../java/org/apache/calcite/model/JsonView.java |  14 ++
 .../org/apache/calcite/model/ModelHandler.java  |   3 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  48 +++-
 .../calcite/prepare/CalcitePrepareImpl.java     | 149 ++++++++++++-
 .../apache/calcite/prepare/RelOptTableImpl.java | 146 +++++++++++-
 .../java/org/apache/calcite/rex/RexCopier.java  |  18 +-
 .../apache/calcite/runtime/CalciteResource.java |   9 +
 .../apache/calcite/schema/ModifiableTable.java  |   2 +
 .../apache/calcite/schema/ModifiableView.java   |  69 ++++++
 .../java/org/apache/calcite/schema/Path.java    |  38 ++++
 .../java/org/apache/calcite/schema/Schemas.java |  94 +++++++-
 .../schema/impl/MaterializedViewTable.java      |   2 +-
 .../apache/calcite/schema/impl/ViewTable.java   | 100 +++++++--
 .../calcite/sql2rel/SqlToRelConverter.java      |  98 ++++++--
 .../calcite/runtime/CalciteResource.properties  |   3 +
 .../apache/calcite/sql/test/SqlAdvisorTest.java |   1 +
 .../org/apache/calcite/test/CalciteAssert.java  |   6 +-
 .../calcite/test/JdbcFrontLinqBackTest.java     |  93 ++++----
 .../java/org/apache/calcite/test/JdbcTest.java  | 223 ++++++++++++++++++-
 .../apache/calcite/test/MockCatalogReader.java  | 223 +++++++++++++++----
 .../calcite/test/ReflectiveSchemaTest.java      |  10 +-
 .../calcite/test/SqlToRelConverterTest.java     |  16 ++
 .../apache/calcite/test/SqlValidatorTest.java   |   6 +
 .../calcite/test/SqlToRelConverterTest.xml      |  40 ++++
 doc/model.md                                    |  26 ++-
 .../calcite/linq4j/tree/ConstantExpression.java |  27 ++-
 .../apache/calcite/linq4j/tree/Expressions.java |   6 +-
 .../apache/calcite/linq4j/test/Linq4jTest.java  |  33 +++
 30 files changed, 1371 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index 9e7502d..eed3d9d 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -586,6 +586,9 @@ public class RexToLixTranslator {
     final Object value2;
     switch (literal.getType().getSqlTypeName()) {
     case DECIMAL:
+      if (javaClass == float.class) {
+        return Expressions.constant(value, javaClass);
+      }
       assert javaClass == BigDecimal.class;
       return Expressions.new_(BigDecimal.class,
           Expressions.constant(value.toString()));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
index acbdfe9..c792ed2 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
@@ -33,14 +33,19 @@ import org.apache.calcite.prepare.CalcitePrepareImpl;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.ArrayBindable;
 import org.apache.calcite.runtime.Bindable;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Stacks;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 
+import com.google.common.collect.ImmutableList;
+
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Type;
@@ -61,7 +66,7 @@ public interface CalcitePrepare {
   ThreadLocal<ArrayList<Context>> THREAD_CONTEXT_STACK =
       new ThreadLocal<ArrayList<Context>>() {
         @Override protected ArrayList<Context> initialValue() {
-          return new ArrayList<Context>();
+          return new ArrayList<>();
         }
       };
 
@@ -69,6 +74,16 @@ public interface CalcitePrepare {
 
   ConvertResult convert(Context context, String sql);
 
+  /** Analyzes a view.
+   *
+   * @param context Context
+   * @param sql View SQL
+   * @param fail Whether to fail (and throw a descriptive error message) if the
+   *             view is not modifiable
+   * @return Result of analyzing the view
+   */
+  AnalyzeViewResult analyzeView(Context context, String sql, boolean fail);
+
   <T> CalciteSignature<T> prepareSql(
       Context context,
       String sql,
@@ -97,7 +112,7 @@ public interface CalcitePrepare {
   }
 
   /** Callback to register Spark as the main engine. */
-  public interface SparkHandler {
+  interface SparkHandler {
     RelNode flattenTypes(RelOptPlanner planner, RelNode rootRel,
         boolean restructure);
 
@@ -118,9 +133,11 @@ public interface CalcitePrepare {
 
   /** Namespace that allows us to define non-abstract methods inside an
    * interface. */
-  public static class Dummy {
+  class Dummy {
     private static SparkHandler sparkHandler;
 
+    private Dummy() {}
+
     /** Returns a spark handler. Returns a trivial handler, for which
      * {@link SparkHandler#enabled()} returns {@code false}, if {@code enable}
      * is {@code false} or if Spark is not on the class path. Never returns
@@ -140,13 +157,10 @@ public interface CalcitePrepare {
         return (CalcitePrepare.SparkHandler) method.invoke(null);
       } catch (ClassNotFoundException e) {
         return new TrivialSparkHandler();
-      } catch (IllegalAccessException e) {
-        throw new RuntimeException(e);
-      } catch (ClassCastException e) {
-        throw new RuntimeException(e);
-      } catch (NoSuchMethodException e) {
-        throw new RuntimeException(e);
-      } catch (InvocationTargetException e) {
+      } catch (IllegalAccessException
+          | ClassCastException
+          | InvocationTargetException
+          | NoSuchMethodException e) {
         throw new RuntimeException(e);
       }
     }
@@ -189,7 +203,7 @@ public interface CalcitePrepare {
   }
 
   /** The result of parsing and validating a SQL query. */
-  public static class ParseResult {
+  class ParseResult {
     public final CalcitePrepareImpl prepare;
     public final String sql; // for debug
     public final SqlNode sqlNode;
@@ -210,7 +224,7 @@ public interface CalcitePrepare {
 
   /** The result of parsing and validating a SQL query and converting it to
    * relational algebra. */
-  public static class ConvertResult extends ParseResult {
+  class ConvertResult extends ParseResult {
     public final RelNode relNode;
 
     public ConvertResult(CalcitePrepareImpl prepare, SqlValidator validator,
@@ -220,10 +234,31 @@ public interface CalcitePrepare {
     }
   }
 
+  /** The result of analyzing a view. */
+  class AnalyzeViewResult extends ConvertResult {
+    /** Not null if and only if the view is modifiable. */
+    public final Table table;
+    public final ImmutableList<String> tablePath;
+    public final RexNode constraint;
+    public final ImmutableIntList columnMapping;
+
+    public AnalyzeViewResult(CalcitePrepareImpl prepare,
+        SqlValidator validator, String sql, SqlNode sqlNode,
+        RelDataType rowType, RelNode relNode, Table table,
+        ImmutableList<String> tablePath, RexNode constraint,
+        ImmutableIntList columnMapping) {
+      super(prepare, validator, sql, sqlNode, rowType, relNode);
+      this.table = table;
+      this.tablePath = tablePath;
+      this.constraint = constraint;
+      this.columnMapping = columnMapping;
+    }
+  }
+
   /** The result of preparing a query. It gives the Avatica driver framework
    * the information it needs to create a prepared statement, or to execute a
    * statement directly, without an explicit prepare step. */
-  public static class CalciteSignature<T> extends Meta.Signature {
+  class CalciteSignature<T> extends Meta.Signature {
     @JsonIgnore public final RelDataType rowType;
     private final int maxRowCount;
     private final Bindable<T> bindable;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/model/JsonView.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonView.java b/core/src/main/java/org/apache/calcite/model/JsonView.java
index f482765..48e1cf3 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonView.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonView.java
@@ -31,6 +31,20 @@ public class JsonView extends JsonTable {
    * to current schema. */
   public List<String> path;
 
+  /** Whether this view should allow INSERT requests.
+   *
+   * <p>The values have the following meanings:
+   * <ul>
+   * <li>If true, Calcite throws an error when validating the schema if the
+   *     view is not modifiable.
+   * <li>If null, Calcite deduces whether the view is modifiable.
+   * <li>If false, Calcite will not allow inserts.
+   * </ul>
+   *
+   * <p>The default value is {@code null}.
+   */
+  public Boolean modifiable;
+
   public void accept(ModelHandler handler) {
     handler.visit(this);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/model/ModelHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
index 01ed89e..dfdb901 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -331,7 +331,8 @@ public class ModelHandler {
       final SchemaPlus schema = currentMutableSchema("view");
       final List<String> path = Util.first(jsonView.path, currentSchemaPath());
       schema.add(jsonView.name,
-          ViewTable.viewMacro(schema, jsonView.getSql(), path));
+          ViewTable.viewMacro(schema, jsonView.getSql(), path,
+              jsonView.modifiable));
     } catch (Exception e) {
       throw new RuntimeException("Error instantiating " + jsonView, e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index 3deb4e4..6090dd2 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -92,6 +92,7 @@ import java.util.BitSet;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -636,8 +637,8 @@ public abstract class RelOptUtil {
       final RelNode rel,
       RelDataType castRowType,
       boolean rename) {
-    return createCastRel(rel, castRowType, rename,
-        RelFactories.DEFAULT_PROJECT_FACTORY);
+    return createCastRel(
+        rel, castRowType, rename, RelFactories.DEFAULT_PROJECT_FACTORY);
   }
 
   /**
@@ -2033,6 +2034,45 @@ public abstract class RelOptUtil {
     return left;
   }
 
+  /** Decomposes the WHERE clause of a view into predicates that constraint
+   * a column to a particular value.
+   *
+   * <p>This method is key to the validation of a modifiable view. Columns that
+   * are constrained to a single value can be omitted from the
+   * SELECT clause of a modifiable view.
+   *
+   * @param projectMap Mapping from column ordinal to the expression that
+   * populate that column, to be populated by this method
+   * @param filters List of remaining filters, to be populated by this method
+   * @param constraint Constraint to be analyzed
+   */
+  public static void inferViewPredicates(Map<Integer, RexNode> projectMap,
+      List<RexNode> filters, RexNode constraint) {
+    for (RexNode node : conjunctions(constraint)) {
+      switch (node.getKind()) {
+      case EQUALS:
+        final List<RexNode> operands = ((RexCall) node).getOperands();
+        RexNode o0 = operands.get(0);
+        RexNode o1 = operands.get(1);
+        if (o0 instanceof RexLiteral) {
+          o0 = operands.get(1);
+          o1 = operands.get(0);
+        }
+        if (o0.getKind() == SqlKind.CAST) {
+          o0 = ((RexCall) o0).getOperands().get(0);
+        }
+        if (o0 instanceof RexInputRef && o1 instanceof RexLiteral) {
+          final int index = ((RexInputRef) o0).getIndex();
+          if (projectMap.get(index) == null) {
+            projectMap.put(index, o1);
+            continue;
+          }
+        }
+      }
+      filters.add(node);
+    }
+  }
+
   /**
    * Adjusts key values in a list by some fixed amount.
    *
@@ -2623,8 +2663,8 @@ public abstract class RelOptUtil {
    */
   public static RelNode createProject(final RelNode child,
       final List<Integer> posList) {
-    return createProject(RelFactories.DEFAULT_PROJECT_FACTORY,
-        child, posList);
+    return createProject(
+        RelFactories.DEFAULT_PROJECT_FACTORY, child, posList);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 9d00a87..9ebfe8a 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -64,6 +64,9 @@ import org.apache.calcite.plan.hep.HepProgramBuilder;
 import org.apache.calcite.plan.volcano.VolcanoPlanner;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
 import org.apache.calcite.rel.rules.AggregateStarTableRule;
@@ -87,11 +90,13 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.Bindable;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.runtime.Typed;
 import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlExplainLevel;
@@ -109,6 +114,7 @@ import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
 import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -121,10 +127,13 @@ import java.math.BigDecimal;
 import java.sql.DatabaseMetaData;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.calcite.util.Static.RESOURCE;
+
 /**
  * Shit just got real.
  *
@@ -223,15 +232,21 @@ public class CalcitePrepareImpl implements CalcitePrepare {
 
   public ParseResult parse(
       Context context, String sql) {
-    return parse_(context, sql, false);
+    return parse_(context, sql, false, false, false);
   }
 
   public ConvertResult convert(Context context, String sql) {
-    return (ConvertResult) parse_(context, sql, true);
+    return (ConvertResult) parse_(context, sql, true, false, false);
+  }
+
+  public AnalyzeViewResult analyzeView(Context context, String sql, boolean fail) {
+    return (AnalyzeViewResult) parse_(context, sql, true, true, fail);
   }
 
-  /** Shared implementation for {@link #parse} and {@link #convert}. */
-  private ParseResult parse_(Context context, String sql, boolean convert) {
+  /** Shared implementation for {@link #parse}, {@link #convert} and
+   * {@link #analyzeView}. */
+  private ParseResult parse_(Context context, String sql, boolean convert,
+      boolean analyze, boolean fail) {
     final JavaTypeFactory typeFactory = context.getTypeFactory();
     CalciteCatalogReader catalogReader =
         new CalciteCatalogReader(
@@ -250,24 +265,144 @@ public class CalcitePrepareImpl implements CalcitePrepare {
         new CalciteSqlValidator(
             SqlStdOperatorTable.instance(), catalogReader, typeFactory);
     SqlNode sqlNode1 = validator.validate(sqlNode);
-    if (!convert) {
-      return new ParseResult(this, validator, sql, sqlNode1,
-          validator.getValidatedNodeType(sqlNode1));
+    if (convert) {
+      return convert_(
+          context, sql, analyze, fail, catalogReader, validator, sqlNode1);
     }
+    return new ParseResult(this, validator, sql, sqlNode1,
+        validator.getValidatedNodeType(sqlNode1));
+  }
+
+  private ParseResult convert_(Context context, String sql, boolean analyze,
+      boolean fail, CalciteCatalogReader catalogReader, SqlValidator validator,
+      SqlNode sqlNode1) {
+    final JavaTypeFactory typeFactory = context.getTypeFactory();
     final Convention resultConvention =
         ENABLE_BINDABLE ? BindableConvention.INSTANCE
             : EnumerableConvention.INSTANCE;
     final HepPlanner planner = new HepPlanner(new HepProgramBuilder().build());
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
     final CalcitePreparingStmt preparingStmt =
         new CalcitePreparingStmt(this, context, catalogReader, typeFactory,
             context.getRootSchema(), null, planner, resultConvention);
     final SqlToRelConverter converter =
         preparingStmt.getSqlToRelConverter(validator, catalogReader);
+    if (analyze) {
+      converter.enableTableAccessConversion(false);
+    }
     final RelNode relNode = converter.convertQuery(sqlNode1, false, true);
+    if (analyze) {
+      return analyze_(validator, sql, sqlNode1, relNode, fail);
+    }
     return new ConvertResult(this, validator, sql, sqlNode1,
         validator.getValidatedNodeType(sqlNode1), relNode);
   }
 
+  private AnalyzeViewResult analyze_(SqlValidator validator, String sql,
+      SqlNode sqlNode, RelNode rel, boolean fail) {
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+    final RelNode viewRel = rel;
+    Project project;
+    if (rel instanceof Project) {
+      project = (Project) rel;
+      rel = project.getInput();
+    } else {
+      project = null;
+    }
+    Filter filter;
+    if (rel instanceof Filter) {
+      filter = (Filter) rel;
+      rel = filter.getInput();
+    } else {
+      filter = null;
+    }
+    TableScan scan;
+    if (rel instanceof TableScan) {
+      scan = (TableScan) rel;
+    } else {
+      scan = null;
+    }
+    if (scan == null) {
+      if (fail) {
+        throw validator.newValidationError(sqlNode,
+            RESOURCE.modifiableViewMustBeBasedOnSingleTable());
+      }
+      return new AnalyzeViewResult(this, validator, sql, sqlNode,
+          validator.getValidatedNodeType(sqlNode), rel, null, null, null,
+          null);
+    }
+    final RelOptTable targetRelTable = scan.getTable();
+    final RelDataType targetRowType = targetRelTable.getRowType();
+    final Table table = targetRelTable.unwrap(Table.class);
+    final List<String> tablePath = targetRelTable.getQualifiedName();
+    assert table != null;
+    List<Integer> columnMapping;
+    final Map<Integer, RexNode> projectMap = new HashMap<>();
+    if (project == null) {
+      columnMapping = ImmutableIntList.range(0, targetRowType.getFieldCount());
+    } else {
+      columnMapping = new ArrayList<>();
+      for (Ord<RexNode> node : Ord.zip(project.getProjects())) {
+        if (node.e instanceof RexInputRef) {
+          RexInputRef rexInputRef = (RexInputRef) node.e;
+          int index = rexInputRef.getIndex();
+          if (projectMap.get(index) != null) {
+            if (fail) {
+              throw validator.newValidationError(sqlNode,
+                  RESOURCE.moreThanOneMappedColumn(
+                      targetRowType.getFieldList().get(index).getName(),
+                      Util.last(tablePath)));
+            }
+            return new AnalyzeViewResult(this, validator, sql, sqlNode,
+                validator.getValidatedNodeType(sqlNode), rel, null, null, null,
+                null);
+          }
+          projectMap.put(index, rexBuilder.makeInputRef(viewRel, node.i));
+          columnMapping.add(index);
+        } else {
+          columnMapping.add(-1);
+        }
+      }
+    }
+    final RexNode constraint;
+    if (filter != null) {
+      constraint = filter.getCondition();
+    } else {
+      constraint = rexBuilder.makeLiteral(true);
+    }
+    final List<RexNode> filters = new ArrayList<>();
+    RelOptUtil.inferViewPredicates(projectMap, filters, constraint);
+
+    // Check that all columns that are not projected have a constant value
+    for (RelDataTypeField field : targetRowType.getFieldList()) {
+      final int x = columnMapping.indexOf(field.getIndex());
+      if (x >= 0) {
+        assert Util.skip(columnMapping, x + 1).indexOf(field.getIndex()) < 0
+            : "column projected more than once; should have checked above";
+        continue; // target column is projected
+      }
+      if (projectMap.get(field.getIndex()) != null) {
+        continue; // constant expression
+      }
+      if (field.getType().isNullable()) {
+        continue; // don't need expression for nullable columns; NULL suffices
+      }
+      if (fail) {
+        throw validator.newValidationError(sqlNode,
+            RESOURCE.noValueSuppliedForViewColumn(field.getName(),
+                Util.last(tablePath)));
+      }
+      return new AnalyzeViewResult(this, validator, sql, sqlNode,
+          validator.getValidatedNodeType(sqlNode), rel, null, null, null,
+          null);
+    }
+
+    return new AnalyzeViewResult(this, validator, sql, sqlNode,
+        validator.getValidatedNodeType(sqlNode), rel, table,
+        ImmutableList.copyOf(tablePath),
+        constraint, ImmutableIntList.copyOf(columnMapping));
+  }
+
   /** Factory method for default SQL parser. */
   protected SqlParser createParser(String sql) {
     return createParser(sql, createParserConfig());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index 8925ede..9919cec 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -19,6 +19,7 @@ package org.apache.calcite.prepare;
 import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.materialize.Lattice;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RelOptTable;
@@ -32,9 +33,13 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.schema.ExtensibleTable;
 import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.Path;
 import org.apache.calcite.schema.ProjectableFilterableTable;
 import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.StreamableTable;
 import org.apache.calcite.schema.Table;
@@ -43,6 +48,7 @@ import org.apache.calcite.sql.SqlAccessType;
 import org.apache.calcite.sql.validate.SqlModality;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -50,7 +56,9 @@ import com.google.common.base.Functions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
+import java.util.Collection;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Implementation of {@link org.apache.calcite.plan.RelOptTable}.
@@ -99,6 +107,15 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
   }
 
   public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType,
+      Table table, Path path) {
+    final SchemaPlus schemaPlus = MySchemaPlus.create(path);
+    Function<Class, Expression> expressionFunction =
+        getClassExpressionFunction(schemaPlus, Util.last(path).left, table);
+    return new RelOptTableImpl(schema, rowType, Pair.left(path), table,
+        expressionFunction, table.getStatistic().getRowCount());
+  }
+
+  public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType,
       final CalciteSchema.TableEntry tableEntry, Double rowCount) {
     final Table table = tableEntry.getTable();
     Function<Class, Expression> expressionFunction =
@@ -108,13 +125,18 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
   }
 
   private static Function<Class, Expression> getClassExpressionFunction(
-      final CalciteSchema.TableEntry tableEntry, final Table table) {
+      CalciteSchema.TableEntry tableEntry, Table table) {
+    return getClassExpressionFunction(tableEntry.schema.plus(), tableEntry.name,
+        table);
+  }
+
+  private static Function<Class, Expression> getClassExpressionFunction(
+      final SchemaPlus schema, final String tableName, final Table table) {
     if (table instanceof QueryableTable) {
       final QueryableTable queryableTable = (QueryableTable) table;
       return new Function<Class, Expression>() {
         public Expression apply(Class clazz) {
-          return queryableTable.getExpression(tableEntry.schema.plus(),
-              tableEntry.name, clazz);
+          return queryableTable.getExpression(schema, tableName, clazz);
         }
       };
     } else if (table instanceof ScannableTable
@@ -122,14 +144,12 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
         || table instanceof ProjectableFilterableTable) {
       return new Function<Class, Expression>() {
         public Expression apply(Class clazz) {
-          return Schemas.tableExpression(tableEntry.schema.plus(),
-              Object[].class,
-              tableEntry.name,
+          return Schemas.tableExpression(schema, Object[].class, tableName,
               table.getClass());
         }
       };
     } else if (table instanceof StreamableTable) {
-      return getClassExpressionFunction(tableEntry,
+      return getClassExpressionFunction(schema, tableName,
           ((StreamableTable) table).stream());
     } else {
       return new Function<Class, Expression>() {
@@ -145,7 +165,8 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
       RelDataType rowType,
       Table table) {
     assert table instanceof TranslatableTable
-        || table instanceof ScannableTable;
+        || table instanceof ScannableTable
+        || table instanceof ModifiableTable;
     return new RelOptTableImpl(schema, rowType, ImmutableList.<String>of(),
         table, null, null);
   }
@@ -299,6 +320,115 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
   public SqlAccessType getAllowedAccess() {
     return SqlAccessType.ALL;
   }
+
+  /** Im0plementation of {@link SchemaPlus} that wraps a regular schema and knows
+   * its name and parent.
+   *
+   * <p>It is read-only, and functionality is limited in other ways, it but
+   * allows table expressions to be genenerated. */
+  private static class MySchemaPlus implements SchemaPlus {
+    private final SchemaPlus parent;
+    private final String name;
+    private final Schema schema;
+
+    public MySchemaPlus(SchemaPlus parent, String name, Schema schema) {
+      this.parent = parent;
+      this.name = name;
+      this.schema = schema;
+    }
+
+    public static MySchemaPlus create(Path path) {
+      final Pair<String, Schema> pair = Util.last(path);
+      final SchemaPlus parent;
+      if (path.size() == 1) {
+        parent = null;
+      } else {
+        parent = create(path.parent());
+      }
+      return new MySchemaPlus(parent, pair.left, pair.right);
+    }
+
+    @Override public SchemaPlus getParentSchema() {
+      return parent;
+    }
+
+    @Override public String getName() {
+      return name;
+    }
+
+    @Override public SchemaPlus getSubSchema(String name) {
+      final Schema subSchema = schema.getSubSchema(name);
+      return subSchema == null ? null : new MySchemaPlus(this, name, subSchema);
+    }
+
+    @Override public SchemaPlus add(String name, Schema schema) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override public void add(String name, Table table) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override public void add(String name,
+        org.apache.calcite.schema.Function function) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override public void add(String name, Lattice lattice) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override public boolean isMutable() {
+      return schema.isMutable();
+    }
+
+    @Override public <T> T unwrap(Class<T> clazz) {
+      return null;
+    }
+
+    @Override public void setPath(ImmutableList<ImmutableList<String>> path) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override public void setCacheEnabled(boolean cache) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override public boolean isCacheEnabled() {
+      return false;
+    }
+
+    @Override public Table getTable(String name) {
+      return schema.getTable(name);
+    }
+
+    @Override public Set<String> getTableNames() {
+      return schema.getTableNames();
+    }
+
+    @Override public Collection<org.apache.calcite.schema.Function>
+    getFunctions(String name) {
+      return schema.getFunctions(name);
+    }
+
+    @Override public Set<String> getFunctionNames() {
+      return schema.getFunctionNames();
+    }
+
+    @Override public Set<String> getSubSchemaNames() {
+      return schema.getSubSchemaNames();
+    }
+
+    @Override public Expression getExpression(SchemaPlus parentSchema,
+        String name) {
+      return schema.getExpression(parentSchema, name);
+    }
+
+    @Override public boolean contentsHaveChangedSince(long lastCheck,
+        long now) {
+      return schema.contentsHaveChangedSince(lastCheck, now);
+    }
+  }
 }
 
 // End RelOptTableImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/rex/RexCopier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCopier.java b/core/src/main/java/org/apache/calcite/rex/RexCopier.java
index 708a59a..3563cdd 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCopier.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCopier.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.rex;
 
+import org.apache.calcite.rel.type.RelDataType;
+
 /**
  * Shuttle which creates a deep copy of a Rex expression.
  *
@@ -45,6 +47,10 @@ class RexCopier extends RexShuttle {
 
   //~ Methods ----------------------------------------------------------------
 
+  private RelDataType copy(RelDataType type) {
+    return builder.getTypeFactory().copyType(type);
+  }
+
   public RexNode visitOver(RexOver over) {
     throw new UnsupportedOperationException();
   }
@@ -55,8 +61,7 @@ class RexCopier extends RexShuttle {
 
   public RexNode visitCall(final RexCall call) {
     final boolean[] update = null;
-    return builder.makeCall(
-        builder.getTypeFactory().copyType(call.getType()),
+    return builder.makeCall(copy(call.getType()),
         call.getOperator(),
         visitList(call.getOperands(), update));
   }
@@ -66,13 +71,12 @@ class RexCopier extends RexShuttle {
   }
 
   public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
-    return builder.makeFieldAccess(
-        fieldAccess.getReferenceExpr().accept(this),
+    return builder.makeFieldAccess(fieldAccess.getReferenceExpr().accept(this),
         fieldAccess.getField().getIndex());
   }
 
   public RexNode visitInputRef(RexInputRef inputRef) {
-    throw new UnsupportedOperationException();
+    return builder.makeInputRef(copy(inputRef.getType()), inputRef.getIndex());
   }
 
   public RexNode visitLocalRef(RexLocalRef localRef) {
@@ -80,9 +84,7 @@ class RexCopier extends RexShuttle {
   }
 
   public RexNode visitLiteral(RexLiteral literal) {
-    return new RexLiteral(
-        literal.getValue(),
-        builder.getTypeFactory().copyType(literal.getType()),
+    return new RexLiteral(literal.getValue(), copy(literal.getType()),
         literal.getTypeName());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 1e8badf..ee3cb5a 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -573,6 +573,15 @@ public interface CalciteResource {
 
   @BaseMessage("Cannot stream VALUES")
   ExInst<SqlValidatorException> cannotStreamValues();
+
+  @BaseMessage("Modifiable view must be based on a single table")
+  ExInst<SqlValidatorException> modifiableViewMustBeBasedOnSingleTable();
+
+  @BaseMessage("View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''")
+  ExInst<SqlValidatorException> moreThanOneMappedColumn(String columnName, String tableName);
+
+  @BaseMessage("View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''")
+  ExInst<SqlValidatorException> noValueSuppliedForViewColumn(String columnName, String tableName);
 }
 
 // End CalciteResource.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java b/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
index 85b2a87..ce0f363 100644
--- a/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
@@ -30,6 +30,8 @@ import java.util.List;
  *
  * <p>NOTE: The current API is inefficient and experimental. It will change
  * without notice.</p>
+ *
+ * @see ModifiableView
  */
 public interface ModifiableTable extends QueryableTable {
   /** Returns the modifiable collection.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/ModifiableView.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ModifiableView.java b/core/src/main/java/org/apache/calcite/schema/ModifiableView.java
new file mode 100644
index 0000000..c1a2d81
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/ModifiableView.java
@@ -0,0 +1,69 @@
+/*
+ * 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.schema;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableIntList;
+
+/**
+ * A modifiable view onto {@link ModifiableTable}.
+ *
+ * <p>It describes how its columns map onto the underlying table's columns,
+ * and any constraints that incoming rows must satisfy.
+ *
+ * <p>For example, given
+ *
+ * <blockquote><pre>
+ *   CREATE TABLE emps (empno INTEGER, gender VARCHAR(1), deptno INTEGER);
+ *   CREATE VIEW female_emps AS
+ *     SELECT empno, deptno FROM emps WHERE gender = 'F';
+ * </pre></blockquote>
+ *
+ * constraint is {@code $1 = 'F'}
+ * and column mapping is {@code [0, 2]}.
+ *
+ * <p>NOTE: The current API is inefficient and experimental. It will change
+ * without notice.</p>
+ */
+public interface ModifiableView extends Table {
+  /** Returns a constraint that each candidate row must satisfy.
+   *
+   * <p>Never null; if there is no constraint, returns "true".
+   *
+   * @param rexBuilder Rex builder
+   * @param tableRowType Row type of the table that this view maps onto
+   */
+  RexNode getConstraint(RexBuilder rexBuilder, RelDataType tableRowType);
+
+  /** Returns the column mapping onto another table.
+   *
+   * {@code mapping[i]} contains the column of the underlying table that the
+   * {@code i}th column of the view comes from, or -1 if it is based on an
+   * expression.
+   */
+  ImmutableIntList getColumnMapping();
+
+  /** Returns the underlying table. */
+  Table getTable();
+
+  /** Returns the full path of the underlying table. */
+  Path getTablePath();
+}
+
+// End ModifiableView.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/Path.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Path.java b/core/src/main/java/org/apache/calcite/schema/Path.java
new file mode 100644
index 0000000..7c363a0
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/Path.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.schema;
+
+import org.apache.calcite.util.Pair;
+
+import java.util.List;
+import java.util.RandomAccess;
+
+/**
+ * Path from a root schema to a particular object (schema, table, function).
+ *
+ * <p>Examples:
+ * <ul>
+ *   <li>The root schema has a single element [(root, "")].
+ *   <li>A direct child "foo" of the root schema has a two elements
+ *   [(root, ""), (child, "foo")].
+ * </ul>
+ */
+public interface Path extends List<Pair<String, Schema>>, RandomAccess {
+  Path parent();
+}
+
+// End Path.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/Schemas.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Schemas.java b/core/src/main/java/org/apache/calcite/schema/Schemas.java
index 63d0e4e..b634233 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schemas.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schemas.java
@@ -39,6 +39,7 @@ import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -47,6 +48,7 @@ import com.google.common.collect.Lists;
 
 import java.lang.reflect.Type;
 import java.sql.Connection;
+import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -92,8 +94,7 @@ public final class Schemas {
       String name,
       Collection<CalciteSchema.FunctionEntry> functionEntries,
       List<RelDataType> argumentTypes) {
-    final List<CalciteSchema.FunctionEntry> matches =
-        new ArrayList<CalciteSchema.FunctionEntry>();
+    final List<CalciteSchema.FunctionEntry> matches = new ArrayList<>();
     for (CalciteSchema.FunctionEntry entry : functionEntries) {
       if (matches(typeFactory, entry.getFunction(), argumentTypes)) {
         matches.add(entry);
@@ -285,9 +286,10 @@ public final class Schemas {
       final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath, final String sql) {
     final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply();
+    final ImmutableMap<CalciteConnectionProperty, String> propValues =
+        ImmutableMap.of();
     final CalcitePrepare.Context context =
-        makeContext(connection, schema, schemaPath,
-            ImmutableMap.<CalciteConnectionProperty, String>of());
+        makeContext(connection, schema, schemaPath, propValues);
     CalcitePrepare.Dummy.push(context);
     try {
       return prepare.parse(context, sql);
@@ -302,9 +304,10 @@ public final class Schemas {
       final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath, final String sql) {
     final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply();
+    final ImmutableMap<CalciteConnectionProperty, String> propValues =
+        ImmutableMap.of();
     final CalcitePrepare.Context context =
-        makeContext(connection, schema, schemaPath,
-            ImmutableMap.<CalciteConnectionProperty, String>of());
+        makeContext(connection, schema, schemaPath, propValues);
     CalcitePrepare.Dummy.push(context);
     try {
       return prepare.convert(context, sql);
@@ -313,6 +316,23 @@ public final class Schemas {
     }
   }
 
+  /** Analyzes a view. For use within Calcite only. */
+  public static CalcitePrepare.AnalyzeViewResult analyzeView(
+      final CalciteConnection connection, final CalciteSchema schema,
+      final List<String> schemaPath, final String sql, boolean fail) {
+    final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply();
+    final ImmutableMap<CalciteConnectionProperty, String> propValues =
+        ImmutableMap.of();
+    final CalcitePrepare.Context context =
+        makeContext(connection, schema, schemaPath, propValues);
+    CalcitePrepare.Dummy.push(context);
+    try {
+      return prepare.analyzeView(context, sql, fail);
+    } finally {
+      CalcitePrepare.Dummy.pop(context);
+    }
+  }
+
   /** Prepares a SQL query for execution. For use within Calcite only. */
   public static CalcitePrepare.CalciteSignature<Object> prepare(
       final CalciteConnection connection, final CalciteSchema schema,
@@ -470,6 +490,30 @@ public final class Schemas {
     return t;
   }
 
+  /** Creates a path with a given list of names starting from a given root
+   * schema. */
+  public static Path path(CalciteSchema rootSchema, Iterable<String> names) {
+    final ImmutableList.Builder<Pair<String, Schema>> builder =
+        ImmutableList.builder();
+    Schema schema = rootSchema.schema;
+    final Iterator<String> iterator = names.iterator();
+    if (!iterator.hasNext()) {
+      return PathImpl.EMPTY;
+    }
+    for (;;) {
+      final String name = iterator.next();
+      builder.add(Pair.of(name, schema));
+      if (!iterator.hasNext()) {
+        return path(builder.build());
+      }
+      schema = schema.getSubSchema(name);
+    }
+  }
+
+  public static PathImpl path(ImmutableList<Pair<String, Schema>> build) {
+    return new PathImpl(build);
+  }
+
   /** Dummy data context that has no variables. */
   private static class DummyDataContext implements DataContext {
     private final CalciteConnection connection;
@@ -497,6 +541,44 @@ public final class Schemas {
       return map.get(name);
     }
   }
+
+  /** Implementation of {@link Path}. */
+  private static class PathImpl
+      extends AbstractList<Pair<String, Schema>> implements Path {
+    private final ImmutableList<Pair<String, Schema>> pairs;
+
+    private static final PathImpl EMPTY =
+        new PathImpl(ImmutableList.<Pair<String, Schema>>of());
+
+    PathImpl(ImmutableList<Pair<String, Schema>> pairs) {
+      this.pairs = pairs;
+    }
+
+    @Override public boolean equals(Object o) {
+      return this == o
+          || o instanceof PathImpl
+          && pairs.equals(((PathImpl) o).pairs);
+    }
+
+    @Override public int hashCode() {
+      return pairs.hashCode();
+    }
+
+    public Pair<String, Schema> get(int index) {
+      return pairs.get(index);
+    }
+
+    public int size() {
+      return pairs.size();
+    }
+
+    @Override public Path parent() {
+      if (pairs.isEmpty()) {
+        throw new IllegalArgumentException("at root");
+      }
+      return new PathImpl(pairs.subList(0, pairs.size() - 1));
+    }
+  }
 }
 
 // End Schemas.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java b/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
index 5810f76..27e873a 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
@@ -103,7 +103,7 @@ public class MaterializedViewTable extends ViewTable {
 
     private MaterializedViewTableMacro(CalciteSchema schema, String viewSql,
         List<String> viewSchemaPath, String suggestedTableName) {
-      super(schema, viewSql, viewSchemaPath);
+      super(schema, viewSql, viewSchemaPath, Boolean.TRUE);
       this.key = Preconditions.checkNotNull(
           MaterializationService.instance().defineMaterialization(
               schema, null, viewSql, schemaPath, suggestedTableName, true));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java b/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
index c978a3a..b3a8ff7 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
@@ -29,12 +29,18 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.ModifiableView;
+import org.apache.calcite.schema.Path;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -64,9 +70,33 @@ public class ViewTable
   }
 
   /** Table macro that returns a view. */
+  @Deprecated // to be removed before 2.0
   public static ViewTableMacro viewMacro(SchemaPlus schema,
       final String viewSql, final List<String> schemaPath) {
-    return new ViewTableMacro(CalciteSchema.from(schema), viewSql, schemaPath);
+    return viewMacro(schema, viewSql, schemaPath, Boolean.TRUE);
+  }
+
+  /** Table macro that returns a view.
+   *
+   * @param schema Schema the view will belong to
+   * @param viewSql SQL query
+   * @param schemaPath Path of schema
+   * @param modifiable Whether view is modifiable, or null to deduce it
+   */
+  public static ViewTableMacro viewMacro(SchemaPlus schema, String viewSql,
+      List<String> schemaPath, Boolean modifiable) {
+    return new ViewTableMacro(CalciteSchema.from(schema), viewSql, schemaPath,
+        modifiable);
+  }
+
+  /** Returns the view's SQL definition. */
+  public String getViewSql() {
+    return viewSql;
+  }
+
+  /** Returns the the schema path of the view. */
+  public List<String> getSchemaPath() {
+    return schemaPath;
   }
 
   @Override public Schema.TableType getJdbcTableType() {
@@ -80,8 +110,7 @@ public class ViewTable
   public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
       SchemaPlus schema, String tableName) {
     return queryProvider.createQuery(
-        getExpression(schema, tableName, Queryable.class),
-        elementType);
+        getExpression(schema, tableName, Queryable.class), elementType);
   }
 
   public RelNode toRel(
@@ -111,14 +140,16 @@ public class ViewTable
   static class ViewTableMacro implements TableMacro {
     protected final String viewSql;
     protected final CalciteSchema schema;
+    private final Boolean modifiable;
     /** Typically null. If specified, overrides the path of the schema as the
      * context for validating {@code viewSql}. */
     protected final List<String> schemaPath;
 
-    ViewTableMacro(CalciteSchema schema, String viewSql,
-        List<String> schemaPath) {
+    ViewTableMacro(CalciteSchema schema, String viewSql, List<String> schemaPath,
+        Boolean modifiable) {
       this.viewSql = viewSql;
       this.schema = schema;
+      this.modifiable = modifiable;
       this.schemaPath =
           schemaPath == null ? null : ImmutableList.copyOf(schemaPath);
     }
@@ -128,25 +159,60 @@ public class ViewTable
     }
 
     public TranslatableTable apply(List<Object> arguments) {
-      CalcitePrepare.ParseResult parsed =
-          Schemas.parse(MaterializedViewTable.MATERIALIZATION_CONNECTION,
-              schema, schemaPath, viewSql);
+      CalcitePrepare.AnalyzeViewResult parsed =
+          Schemas.analyzeView(MaterializedViewTable.MATERIALIZATION_CONNECTION,
+              schema, schemaPath, viewSql, modifiable != null && modifiable);
       final List<String> schemaPath1 =
           schemaPath != null ? schemaPath : schema.path(null);
       final JavaTypeFactory typeFactory = (JavaTypeFactory) parsed.typeFactory;
-      return new ViewTable(typeFactory.getJavaClass(parsed.rowType),
-          RelDataTypeImpl.proto(parsed.rowType), viewSql, schemaPath1);
+      final Type elementType = typeFactory.getJavaClass(parsed.rowType);
+      if ((modifiable == null || modifiable) && parsed.table != null) {
+        return new ModifiableViewTable(elementType,
+            RelDataTypeImpl.proto(parsed.rowType), viewSql, schemaPath1,
+            parsed.table, Schemas.path(schema.root(), parsed.tablePath),
+            parsed.constraint, parsed.columnMapping);
+      } else {
+        return new ViewTable(elementType,
+            RelDataTypeImpl.proto(parsed.rowType), viewSql, schemaPath1);
+      }
     }
   }
 
-  /** Returns the view's SQL definition. */
-  public String getViewSql() {
-    return viewSql;
-  }
+  /** Extension to {@link ViewTable} that is modifiable. */
+  static class ModifiableViewTable extends ViewTable
+      implements ModifiableView {
+    private final Table table;
+    private final Path tablePath;
+    private final RexNode constraint;
+    private final ImmutableIntList columnMapping;
+
+    public ModifiableViewTable(Type elementType, RelProtoDataType rowType,
+        String viewSql, List<String> schemaPath, Table table,
+        Path tablePath, RexNode constraint,
+        ImmutableIntList columnMapping) {
+      super(elementType, rowType, viewSql, schemaPath);
+      this.table = table;
+      this.tablePath = tablePath;
+      this.constraint = constraint;
+      this.columnMapping = columnMapping;
+    }
 
-  /** Returns the the schema path of the view. */
-  public List<String> getSchemaPath() {
-    return schemaPath;
+    public RexNode getConstraint(RexBuilder rexBuilder,
+        RelDataType tableRowType) {
+      return rexBuilder.copy(constraint);
+    }
+
+    public ImmutableIntList getColumnMapping() {
+      return columnMapping;
+    }
+
+    public Table getTable() {
+      return table;
+    }
+
+    public Path getTablePath() {
+      return tablePath;
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 5dad7f4..a190159 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -73,6 +73,8 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.ModifiableView;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.sql.JoinConditionType;
 import org.apache.calcite.sql.JoinType;
@@ -561,11 +563,7 @@ public class SqlToRelConverter {
     validatedRowType = uniquifyFields(validatedRowType);
 
     return RelOptUtil.equal(
-        "validated row type",
-        validatedRowType,
-        "converted row type",
-        convertedRowType,
-        false);
+        "validated row type", validatedRowType, "converted row type", convertedRowType, false);
   }
 
   protected RelDataType uniquifyFields(RelDataType rowType) {
@@ -2024,7 +2022,8 @@ public class SqlToRelConverter {
     Set<RelColumnMapping> columnMappings =
         getColumnMappings(operator);
     LogicalTableFunctionScan callRel =
-        LogicalTableFunctionScan.create(cluster,
+        LogicalTableFunctionScan.create(
+            cluster,
             inputs,
             rexCall,
             elementType,
@@ -2767,8 +2766,8 @@ public class SqlToRelConverter {
   protected RelNode createAggregate(Blackboard bb, boolean indicator,
       ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls) {
-    return LogicalAggregate.create(bb.root, indicator, groupSet, groupSets,
-        aggCalls);
+    return LogicalAggregate.create(
+        bb.root, indicator, groupSet, groupSets, aggCalls);
   }
 
   public RexDynamicParam convertDynamicParam(
@@ -2999,27 +2998,88 @@ public class SqlToRelConverter {
     assert targetRowType != null;
     RelNode sourceRel =
         convertQueryRecursive(
-            call.getSource(),
-            false,
-            targetRowType);
+            call.getSource(), false, targetRowType);
     RelNode massagedRel = convertColumnList(call, sourceRel);
 
+    return createModify(targetTable, massagedRel);
+  }
+
+  /** Creates a relational expression to modify a table or modifiable view. */
+  private RelNode createModify(RelOptTable targetTable, RelNode source) {
     final ModifiableTable modifiableTable =
         targetTable.unwrap(ModifiableTable.class);
     if (modifiableTable != null) {
-      return modifiableTable.toModificationRel(
-          cluster,
-          targetTable,
-          catalogReader,
-          massagedRel,
-          LogicalTableModify.Operation.INSERT,
-          null,
+      return modifiableTable.toModificationRel(cluster, targetTable,
+          catalogReader, source, LogicalTableModify.Operation.INSERT, null,
           false);
     }
-    return LogicalTableModify.create(targetTable, catalogReader, massagedRel,
+    final ModifiableView modifiableView =
+        targetTable.unwrap(ModifiableView.class);
+    if (modifiableView != null) {
+      final Table delegateTable = modifiableView.getTable();
+      final RelDataType delegateRowType = delegateTable.getRowType(typeFactory);
+      final RelOptTable delegateRelOptTable =
+          RelOptTableImpl.create(null, delegateRowType, delegateTable,
+              modifiableView.getTablePath());
+      final RelNode newSource =
+          createSource(targetTable, source, modifiableView, delegateRowType);
+      return createModify(delegateRelOptTable, newSource);
+    }
+    return LogicalTableModify.create(targetTable, catalogReader, source,
         LogicalTableModify.Operation.INSERT, null, false);
   }
 
+  /** Wraps a relational expression in the projects and filters implied by
+   * a {@link ModifiableView}.
+   *
+   * <p>The input relational expression is suitable for inserting into the view,
+   * and the returned relational expression is suitable for inserting into its
+   * delegate table.
+   *
+   * <p>In principle, the delegate table of a view might be another modifiable
+   * view, and if so, the process can be repeated. */
+  private RelNode createSource(RelOptTable targetTable, RelNode source,
+      ModifiableView modifiableView, RelDataType delegateRowType) {
+    final ImmutableIntList mapping = modifiableView.getColumnMapping();
+    assert mapping.size() == targetTable.getRowType().getFieldCount();
+
+    // For columns represented in the mapping, the expression is just a field
+    // reference.
+    final Map<Integer, RexNode> projectMap = new HashMap<>();
+    final List<RexNode> filters = new ArrayList<>();
+    for (int i = 0; i < mapping.size(); i++) {
+      int target = mapping.get(i);
+      if (target >= 0) {
+        projectMap.put(target, RexInputRef.of(i, source.getRowType()));
+      }
+    }
+
+    // For columns that are not in the mapping, and have a constraint of the
+    // form "column = value", the expression is the literal "value".
+    //
+    // If a column has multiple constraints, the extra ones will become a
+    // filter.
+    final RexNode constraint =
+        modifiableView.getConstraint(rexBuilder, delegateRowType);
+    RelOptUtil.inferViewPredicates(projectMap, filters, constraint);
+    final List<Pair<RexNode, String>> projects = new ArrayList<>();
+    for (RelDataTypeField field : delegateRowType.getFieldList()) {
+      RexNode node = projectMap.get(field.getIndex());
+      if (node == null) {
+        node = rexBuilder.makeNullLiteral(field.getType().getSqlTypeName());
+      }
+      projects.add(
+          Pair.of(rexBuilder.ensureType(field.getType(), node, false),
+              field.getName()));
+    }
+
+    source = RelOptUtil.createProject(source, projects, true);
+    if (filters.size() > 0) {
+      source = RelOptUtil.createFilter(source, filters);
+    }
+    return source;
+  }
+
   private RelOptTable.ToRelContext createToRelContext() {
     return new RelOptTable.ToRelContext() {
       public RelOptCluster getCluster() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/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 e2321bb..e170b67 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -187,4 +187,7 @@ StreamMustGroupByMonotonic=Streaming aggregation requires at least one monotonic
 StreamMustOrderByMonotonic=Streaming ORDER BY must start with monotonic expression
 StreamSetOpInconsistentInputs=Set operator cannot combine streaming and non-streaming inputs
 CannotStreamValues=Cannot stream VALUES
+ModifiableViewMustBeBasedOnSingleTable=Modifiable view must be based on a single table
+MoreThanOneMappedColumn=View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''
+NoValueSuppliedForViewColumn=View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''
 # End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 3694094..1a71ec4 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -66,6 +66,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
   protected static final List<String> SALES_TABLES =
       Arrays.asList(
           "TABLE(CATALOG.SALES.EMP)",
+          "TABLE(CATALOG.SALES.EMP_20)",
           "TABLE(CATALOG.SALES.EMP_ADDRESS)",
           "TABLE(CATALOG.SALES.DEPT)",
           "TABLE(CATALOG.SALES.BONUS)",

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/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 cc4e1f2..8160c71 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -698,7 +698,7 @@ public class CalciteAssert {
                   + "    ('Grace', 60, 'F'),\n"
                   + "    ('Wilma', cast(null as integer), 'F'))\n"
                   + "  as t(ename, deptno, gender)",
-              ImmutableList.<String>of()));
+              ImmutableList.<String>of(), null));
       post.add("DEPT",
           ViewTable.viewMacro(post,
               "select * from (values\n"
@@ -706,7 +706,7 @@ public class CalciteAssert {
                   + "    (20, 'Marketing'),\n"
                   + "    (30, 'Engineering'),\n"
                   + "    (40, 'Empty')) as t(deptno, dname)",
-              ImmutableList.<String>of()));
+              ImmutableList.<String>of(), null));
       post.add("EMPS",
           ViewTable.viewMacro(post,
               "select * from (values\n"
@@ -716,7 +716,7 @@ public class CalciteAssert {
                   + "    (120, 'Wilma', 20, 'F',                   CAST(NULL AS VARCHAR(20)), 1,                 5, UNKNOWN, TRUE,  DATE '2005-09-07'),\n"
                   + "    (130, 'Alice', 40, 'F',                   'Vancouver',               2, CAST(NULL AS INT), FALSE,   TRUE,  DATE '2007-01-01'))\n"
                   + " as t(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)",
-              ImmutableList.<String>of()));
+              ImmutableList.<String>of(), null));
       return post;
     default:
       throw new AssertionError("unknown schema " + schema);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
index a6d23e0..e4131c3 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
@@ -237,54 +237,59 @@ public class JdbcFrontLinqBackTest {
     employees.add(new JdbcTest.Employee(0, 0, "first", 0f, null));
     return that()
         .with(CalciteAssert.Config.REGULAR)
-        .with(new CalciteAssert.ConnectionPostProcessor() {
-          public Connection apply(final Connection connection)
-              throws SQLException {
-            CalciteConnection calciteConnection =
-                connection.unwrap(CalciteConnection.class);
-            SchemaPlus rootSchema =
-                calciteConnection.getRootSchema();
-            SchemaPlus mapSchema =
-                rootSchema.add("foo", new AbstractSchema());
-            final String tableName = "bar";
-            final JdbcTest.AbstractModifiableTable table =
-                new JdbcTest.AbstractModifiableTable(tableName) {
-                  public RelDataType getRowType(
-                      RelDataTypeFactory typeFactory) {
-                    return ((JavaTypeFactory) typeFactory)
-                        .createType(JdbcTest.Employee.class);
-                  }
+        .with(
+            new CalciteAssert.ConnectionPostProcessor() {
+              public Connection apply(final Connection connection)
+                  throws SQLException {
+                CalciteConnection calciteConnection =
+                    connection.unwrap(CalciteConnection.class);
+                SchemaPlus rootSchema =
+                    calciteConnection.getRootSchema();
+                SchemaPlus mapSchema =
+                    rootSchema.add("foo", new AbstractSchema());
+                final String tableName = "bar";
+                final JdbcTest.AbstractModifiableTable table =
+                    mutable(tableName, employees);
+                mapSchema.add(tableName, table);
+                return calciteConnection;
+              }
+            });
+  }
 
-                  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(employees);
-                      }
-                    };
-                  }
+  static JdbcTest.AbstractModifiableTable mutable(String tableName,
+      final List<JdbcTest.Employee> employees) {
+    return new JdbcTest.AbstractModifiableTable(tableName) {
+      public RelDataType getRowType(
+          RelDataTypeFactory typeFactory) {
+        return ((JavaTypeFactory) typeFactory)
+            .createType(JdbcTest.Employee.class);
+      }
 
-                  public Type getElementType() {
-                    return JdbcTest.Employee.class;
-                  }
+      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(employees);
+          }
+        };
+      }
 
-                  public Expression getExpression(SchemaPlus schema,
-                      String tableName, Class clazz) {
-                    return Schemas.tableExpression(schema, getElementType(),
-                        tableName, clazz);
-                  }
+      public Type getElementType() {
+        return JdbcTest.Employee.class;
+      }
 
-                  public Collection getModifiableCollection() {
-                    return employees;
-                  }
-                };
-            mapSchema.add(tableName, table);
-            return calciteConnection;
-          }
-        });
+      public Expression getExpression(SchemaPlus schema, String tableName,
+          Class clazz) {
+        return Schemas.tableExpression(schema, getElementType(), tableName,
+            clazz);
+      }
+
+      public Collection getModifiableCollection() {
+        return employees;
+      }
+    };
   }
 
   @Test public void testInsert2() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/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 27eb0e5..8f73fbd 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -58,6 +58,7 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.ModifiableView;
 import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaFactory;
@@ -79,11 +80,13 @@ import org.apache.calcite.sql.advise.SqlAdvisorGetHintsFunction;
 import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
@@ -130,6 +133,8 @@ import java.util.TimeZone;
 import java.util.regex.Pattern;
 import javax.sql.DataSource;
 
+import static org.apache.calcite.util.Static.RESOURCE;
+
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.instanceOf;
@@ -137,6 +142,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
 import static org.hamcrest.CoreMatchers.notNullValue;
 import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -250,6 +256,165 @@ public class JdbcTest {
     return FOODMART_QUERIES;
   }
 
+  /** Tests a modifiable view. */
+  @Test public void testModelWithModifiableView() throws Exception {
+    final List<Employee> employees = new ArrayList<>();
+    employees.add(new Employee(135, 10, "Simon", 56.7f, null));
+    try {
+      EmpDeptTableFactory.THREAD_COLLECTION.set(employees);
+      final CalciteAssert.AssertThat with = modelWithView(
+          "select \"name\", \"empid\" as e, \"salary\" "
+              + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" = 10",
+          null);
+      with.query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .returns("name=Simon\n");
+      with.doWithConnection(
+          new Function<CalciteConnection, Object>() {
+            @Override public Object apply(CalciteConnection input) {
+              try {
+                final Statement statement = input.createStatement();
+                ResultSet resultSet =
+                    statement.executeQuery("explain plan for\n"
+                        + "insert into \"adhoc\".V\n"
+                        + "values ('Fred', 56, 123.4)");
+                assertThat(resultSet.next(), is(true));
+                assertThat(resultSet.getString(1),
+                    is(
+                        "EnumerableTableModify(table=[[adhoc, MUTABLE_EMPLOYEES]], operation=[INSERT], updateColumnList=[[]], flattened=[false])\n"
+                        + "  EnumerableCalc(expr#0..2=[{inputs}], expr#3=[CAST($t1):JavaType(int) NOT NULL], expr#4=[10], expr#5=[CAST($t0):JavaType(class java.lang.String)], expr#6=[CAST($t2):JavaType(float) NOT NULL], expr#7=[null], empid=[$t3], deptno=[$t4], name=[$t5], salary=[$t6], commission=[$t7])\n"
+                        + "    EnumerableValues(tuples=[[{ 'Fred', 56, 123.4 }]])\n"));
+
+                // With named columns
+                resultSet =
+                    statement.executeQuery("explain plan for\n"
+                        + "insert into \"adhoc\".V (\"name\", e, \"salary\")\n"
+                        + "values ('Fred', 56, 123.4)");
+                assertThat(resultSet.next(), is(true));
+
+                // With named columns, in different order
+                resultSet =
+                    statement.executeQuery("explain plan for\n"
+                        + "insert into \"adhoc\".V (e, \"salary\", \"name\")\n"
+                        + "values (56, 123.4, 'Fred')");
+                assertThat(resultSet.next(), is(true));
+
+                // Mis-named column
+                try {
+                  final PreparedStatement s =
+                      input.prepareStatement("explain plan for\n"
+                          + "insert into \"adhoc\".V (empno, \"salary\", \"name\")\n"
+                          + "values (56, 123.4, 'Fred')");
+                  fail("expected error, got " + s);
+                } catch (SQLException e) {
+                  assertThat(e.getMessage(),
+                      startsWith("Error while preparing statement"));
+                }
+
+                // Fail to provide mandatory column
+                try {
+                  final PreparedStatement s =
+                      input.prepareStatement("explain plan for\n"
+                          + "insert into \"adhoc\".V (e, name)\n"
+                          + "values (56, 'Fred')");
+                  fail("expected error, got " + s);
+                } catch (SQLException e) {
+                  assertThat(e.getMessage(),
+                      startsWith("Error while preparing statement"));
+                }
+
+                statement.close();
+                return null;
+              } catch (SQLException e) {
+                throw Throwables.propagate(e);
+              }
+            }
+          });
+    } finally {
+      EmpDeptTableFactory.THREAD_COLLECTION.remove();
+    }
+  }
+
+  /** Tests a few cases where modifiable views are invalid. */
+  @Test public void testModelWithInvalidModifiableView() throws Exception {
+    final List<Employee> employees = new ArrayList<>();
+    employees.add(new Employee(135, 10, "Simon", 56.7f, null));
+    try {
+      EmpDeptTableFactory.THREAD_COLLECTION.set(employees);
+
+      Util.discard(RESOURCE.noValueSuppliedForViewColumn(null, null));
+      modelWithView("select \"name\", \"empid\" as e, \"salary\" "
+              + "from \"MUTABLE_EMPLOYEES\" where \"commission\" = 10",
+          true)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .throws_(
+              "View is not modifiable. No value is supplied for NOT NULL "
+                  + "column 'deptno' of base table 'MUTABLE_EMPLOYEES'");
+
+      // no error if we do not claim that the view is modifiable
+      modelWithView(
+          "select \"name\", \"empid\" as e, \"salary\" "
+              + "from \"MUTABLE_EMPLOYEES\" where \"commission\" = 10", null)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .runs();
+
+      modelWithView("select \"name\", \"empid\" as e, \"salary\" "
+              + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" IN (10, 20)",
+          true)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .throws_(
+              "View is not modifiable. No value is supplied for NOT NULL "
+                  + "column 'deptno' of base table 'MUTABLE_EMPLOYEES'");
+
+      // Deduce "deptno = 10" from the constraint, and add a further
+      // condition "deptno < 20 OR commission > 1000".
+      modelWithView("select \"name\", \"empid\" as e, \"salary\" "
+              + "from \"MUTABLE_EMPLOYEES\"\n"
+              + "where \"deptno\" = 10 AND (\"deptno\" < 20 OR \"commission\" > 1000)",
+          true)
+          .query("insert into \"adhoc\".v values ('n',1,2)")
+          .explainContains(""
+              + "EnumerableTableModify(table=[[adhoc, MUTABLE_EMPLOYEES]], operation=[INSERT], updateColumnList=[[]], flattened=[false])\n"
+              + "  EnumerableCalc(expr#0..2=[{inputs}], expr#3=[CAST($t1):JavaType(int) NOT NULL], expr#4=[10], expr#5=[CAST($t0):JavaType(class java.lang.String)], expr#6=[CAST($t2):JavaType(float) NOT NULL], expr#7=[null], expr#8=[20], expr#9=[<($t4, $t8)], expr#10=[1000], expr#11=[>($t7, $t10)], expr#12=[OR($t9, $t11)], empid=[$t3], deptno=[$t4], name=[$t5], salary=[$t6], commission=[$t7], $condition=[$t12])\n"
+              + "    EnumerableValues(tuples=[[{ 'n', 1, 2 }]])");
+
+      modelWithView(
+          "select \"name\", \"empid\" as e, \"salary\" "
+              + "from \"MUTABLE_EMPLOYEES\"\n"
+              + "where \"commission\" = 100 AND \"deptno\" = 20",
+          true)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .runs();
+
+      modelWithView(
+          "select \"name\", \"empid\" as e, \"salary\", \"empid\" + 3 as e3, 1 as one\n"
+              + "from \"MUTABLE_EMPLOYEES\"\n"
+              + "where \"commission\" = 100 AND \"deptno\" = 20",
+          true)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .runs();
+
+      Util.discard(RESOURCE.moreThanOneMappedColumn(null, null));
+      modelWithView(
+          "select \"name\", \"empid\" as e, \"salary\", \"name\" as n2 "
+              + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" IN (10, 20)",
+          true)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .throws_(
+              "View is not modifiable. More than one expression maps to "
+              + "column 'name' of base table 'MUTABLE_EMPLOYEES'");
+
+      // no error if we do not claim that the view is modifiable
+      modelWithView(
+          "select \"name\", \"empid\" as e, \"salary\", \"name\" as n2 "
+              + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" IN (10, 20)",
+          null)
+          .query("select \"name\" from \"adhoc\".V order by \"name\"")
+          .runs();
+    } finally {
+      EmpDeptTableFactory.THREAD_COLLECTION.remove();
+    }
+  }
+
   /**
    * Tests a table function with literal arguments.
    */
@@ -3264,10 +3429,10 @@ public class JdbcTest {
             + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[+($t3, $t0)], proj#0..1=[{exprs}], salary=[$t3], $3=[$t5])\n"
             + "      EnumerableTableScan(table=[[hr, emps]])\n")
         .returnsUnordered(
-            "deptno=10; empid=100; S=10100.0; FIVE=5; M=10000.0; C=1"
-            , "deptno=10; empid=110; S=21710.0; FIVE=5; M=10000.0; C=2"
-            , "deptno=10; empid=150; S=18760.0; FIVE=5; M=7000.0; C=2"
-            , "deptno=20; empid=200; S=8200.0; FIVE=5; M=8000.0; C=1")
+            "deptno=10; empid=100; S=10100.0; FIVE=5; M=10000.0; C=1",
+            "deptno=10; empid=110; S=21710.0; FIVE=5; M=10000.0; C=2",
+            "deptno=10; empid=150; S=18760.0; FIVE=5; M=7000.0; C=2",
+            "deptno=20; empid=200; S=8200.0; FIVE=5; M=8000.0; C=1")
         .planContains(CalcitePrepareImpl.DEBUG
             ? "_list.add(new Object[] {\n"
             + "        row[0],\n" // box-unbox is optimized
@@ -4822,7 +4987,8 @@ public class JdbcTest {
             "Cannot define view; parent schema 'adhoc' is not mutable");
   }
 
-  private CalciteAssert.AssertThat modelWithView(String view) {
+  private CalciteAssert.AssertThat modelWithView(String view,
+      Boolean modifiable) {
     final Class<EmpDeptTableFactory> clazz = EmpDeptTableFactory.class;
     return CalciteAssert.model("{\n"
         + "  version: '1.0',\n"
@@ -4837,9 +5003,16 @@ public class JdbcTest {
         + "           operand: {'foo': true, 'bar': 345}\n"
         + "         },\n"
         + "         {\n"
+        + "           name: 'MUTABLE_EMPLOYEES',\n"
+        + "           type: 'custom',\n"
+        + "           factory: '" + clazz.getName() + "',\n"
+        + "           operand: {'foo': false}\n"
+        + "         },\n"
+        + "         {\n"
         + "           name: 'V',\n"
         + "           type: 'view',\n"
-        + "           sql: '" + view + "'\n"
+        + (modifiable == null ? "" : " modifiable: " + modifiable + ",\n")
+        + "           sql: " + new JsonBuilder().toJsonString(view) + "\n"
         + "         }\n"
         + "       ]\n"
         + "     }\n"
@@ -4850,7 +5023,8 @@ public class JdbcTest {
   /** Tests a JDBC connection that provides a model that contains a view. */
   @Test public void testModelView() throws Exception {
     final CalciteAssert.AssertThat with =
-        modelWithView("select * from \"EMPLOYEES\" where \"deptno\" = 10");
+        modelWithView("select * from \"EMPLOYEES\" where \"deptno\" = 10",
+            null);
 
     with.query("select * from \"adhoc\".V order by \"name\" desc")
         .returns(""
@@ -4868,6 +5042,7 @@ public class JdbcTest {
               // all table types
               assertEquals(
                   "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=EMPLOYEES; TABLE_TYPE=TABLE; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n"
+                      + "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=MUTABLE_EMPLOYEES; TABLE_TYPE=TABLE; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n"
                       + "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=V; TABLE_TYPE=VIEW; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n",
                   CalciteAssert.toString(
                       metaData.getTables(null, "adhoc", null, null)));
@@ -4931,7 +5106,7 @@ public class JdbcTest {
   @Test public void testOrderByView() throws Exception {
     final CalciteAssert.AssertThat with =
         modelWithView("select * from \"EMPLOYEES\" where \"deptno\" = 10 "
-            + "order by \"empid\" limit 2");
+            + "order by \"empid\" limit 2", null);
     with
         .query("select \"name\" from \"adhoc\".V order by \"name\"")
         .returns("name=Bill\n"
@@ -6050,7 +6225,9 @@ public class JdbcTest {
     assertThat(a2CalciteSchema.getTable("table1", false), notNullValue());
     assertThat(a2CalciteSchema.getTable("taBle1", true), nullValue());
     assertThat(a2CalciteSchema.getTable("taBle1", false), notNullValue());
-    final TableMacro function = ViewTable.viewMacro(a2Schema, "values 1", null);
+    final TableMacro function =
+        ViewTable.viewMacro(a2Schema, "values 1", null, null);
+    Util.discard(function);
 
     connection.close();
   }
@@ -6186,7 +6363,7 @@ public class JdbcTest {
 
     assertThat(rs.next(), is(true));
     assertThat((Integer) rs.getObject("ID"), equalTo(2));
-    assertThat((Double) rs.getObject("VALS"), nullValue());
+    assertThat(rs.getObject("VALS"), nullValue());
 
     assertThat(rs.next(), is(true));
     assertThat(rs.getObject("ID"), nullValue());
@@ -6420,8 +6597,19 @@ public class JdbcTest {
     }
   }
 
+  /** Abstract base class for implementations of {@link ModifiableView}. */
+  public abstract static class AbstractModifiableView
+      extends AbstractTable implements ModifiableView {
+    protected AbstractModifiableView() {
+      super();
+    }
+  }
+
   /** Factory for EMP and DEPT tables. */
   public static class EmpDeptTableFactory implements TableFactory<Table> {
+    public static final ThreadLocal<List<Employee>> THREAD_COLLECTION =
+        new ThreadLocal<>();
+
     public Table create(
         SchemaPlus schema,
         String name,
@@ -6429,12 +6617,23 @@ public class JdbcTest {
         RelDataType rowType) {
       final Class clazz;
       final Object[] array;
-      if (name.equals("EMPLOYEES")) {
+      switch (name) {
+      case "EMPLOYEES":
         clazz = Employee.class;
         array = new HrSchema().emps;
-      } else {
+        break;
+      case "MUTABLE_EMPLOYEES":
+        List<Employee> employees = THREAD_COLLECTION.get();
+        if (employees == null) {
+          employees = Collections.emptyList();
+        }
+        return JdbcFrontLinqBackTest.mutable(name, employees);
+      case "DEPARTMENTS":
         clazz = Department.class;
         array = new HrSchema().depts;
+        break;
+      default:
+        throw new AssertionError(name);
       }
       return new AbstractQueryableTable(clazz) {
         public RelDataType getRowType(RelDataTypeFactory typeFactory) {


[11/13] incubator-calcite git commit: [CALCITE-723] Document lattices

Posted by jh...@apache.org.
[CALCITE-723] Document lattices


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

Branch: refs/heads/master
Commit: aee32bc498737720428ee51e779473f8deedf51c
Parents: 06a192a
Author: Julian Hyde <jh...@apache.org>
Authored: Mon May 11 18:30:48 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon May 11 18:39:48 2015 -0700

----------------------------------------------------------------------
 README.md      |   1 +
 doc/lattice.md | 125 ++++++++++++++++++++++++++++++++++++++++++++++++++++
 doc/model.md   |   2 +
 3 files changed, 128 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/aee32bc4/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 61e210d..2b64932 100644
--- a/README.md
+++ b/README.md
@@ -213,6 +213,7 @@ For more details, see the <a href="doc/reference.md">Reference guide</a>.
 * <a href="doc/howto.md">HOWTO</a>
 * <a href="doc/model.md">JSON model</a>
 * <a href="doc/reference.md">Reference guide</a>
+* <a href="doc/lattice.md">Lattices</a>
 * <a href="doc/stream.md">Streaming SQL</a>
 * <a href="doc/avatica.md">Avatica JDBC framework</a>
 * <a href="doc/history.md">Release notes and history</a>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/aee32bc4/doc/lattice.md
----------------------------------------------------------------------
diff --git a/doc/lattice.md b/doc/lattice.md
new file mode 100644
index 0000000..d7d4642
--- /dev/null
+++ b/doc/lattice.md
@@ -0,0 +1,125 @@
+<!--
+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.
+-->
+# Lattice
+
+A lattice is a framework for creating and populating materialized views,
+and for recognizing that a materialized view can be used to solve a
+particular query.
+
+A lattice represents a star (or snowflake) schema, not a general
+schema. In particular, all relationships must be many-to-one, heading
+from a fact table at the center of the star.
+
+The name derives from the mathematics: a
+<a href="http://en.wikipedia.org/wiki/Lattice_(order)">lattice</a>
+is a
+<a href="http://en.wikipedia.org/wiki/Partially_ordered_set">partially
+ordered set</a> where any two elements have a unique greatest lower
+bound and least upper bound.
+
+[<a href="#user-content-ref-hru96">HRU96</a>] observed that the set of possible
+materializations of a data cube forms a lattice, and presented an
+algorithm to choose a good set of materializations. Calcite's
+recommendation algorithm is derived from this.
+
+The lattice definition uses a SQL statement to represent the star. SQL
+is a useful short-hand to represent several tables joined together,
+and assigning aliases to the column names (it more convenient than
+inventing a new language to represent relationships, join conditions
+and cardinalities).
+
+Unlike regular SQL, order is important. If you put A before B in the
+FROM clause, and make a join between A and B, you are saying that
+there is a many-to-one foreign key relationship from A to B. (E.g. in
+the example lattice, the Sales fact table occurs before the Time
+dimension table, and before the Product dimension table. The Product
+dimension table occurs before the ProductClass outer dimension table,
+further down an arm of a snowflake.)
+
+A lattice implies constraints. In the A to B relationship, there is a
+foreign key on A (i.e. every value of A's foreign key has a
+corresponding value in B's key), and a unique key on B (i.e. no key
+value occurs more than once). These constraints are really important,
+because it allows the planner to remove joins to tables whose columns
+are not being used, and know that the query results will not change.
+
+Calcite does not check these constraints. If they are violated,
+Calcite will return wrong results.
+
+A lattice is a big, virtual join view. It is not materialized (it
+would be several times larger than the star schema, because of
+denormalization) and you probably wouldn't want to query it (far too
+many columns). So what is it useful for? As we said above, (a) the
+lattice declares some very useful primary and foreign key constraints,
+(b) it helps the query planner map user queries onto
+filter-join-aggregate materialized views (the most useful kind of
+materialized view for DW queries), (c) gives Calcite a framework
+within which to gather stats about data volumes and user queries, (d)
+allows Calcite to automatically design and populate materialized
+views.
+
+Most star schema models force you to choose whether a column is a
+dimension or a measure. In a lattice, every column is a dimension
+column. (That is, it can become one of the columns in the GROUP BY clause
+to query the star schema at a particular dimensionality). Any column
+can also be used in a measure; you define measures by giving the
+column and an aggregate function.
+
+If "unit_sales" tends to be used much more often as a measure rather
+than a dimension, that's fine. Calcite's algorithm should notice that
+it is rarely aggregated, and not be inclined to create tiles that
+aggregate on it. (By "should" I mean "could and one day will". The
+algorithm does not currently take query history into account when
+designing tiles.)
+
+But someone might want to know whether orders with fewer than 5 items
+were more or less profitable than orders with more than 100. All of a
+sudden, "unit_sales" has become a dimension. If there's virtually zero
+cost to declaring a column a dimension column, I figured let's make
+them all dimension columns.
+
+The model allows for a particular table to be used more than once,
+with a different table alias. You could use this to model say
+OrderDate and ShipDate, with two uses to the Time dimension table.
+
+Most SQL systems require that the column names in a view are unique.
+This is hard to achieve in a lattice, because you often include
+primary and foreign key columns in a join. So Calcite lets you refer
+to columns in two ways. If the column is unique, you can use its name,
+["unit_sales"]. Whether or not it is unique in the lattice, it will be
+unique in its table, so you can use it qualified by its table alias.
+Examples:
+* ["sales", "unit_sales"]
+* ["ship_date", "time_id"]
+* ["order_date", "time_id"]
+
+A "tile" is a materialized table in a lattice, with a particular
+dimensionality. (What Kylin calls a "cuboid".) The "tiles" attribute
+of the <a href="model.json#lattice">lattice JSON element</a>
+defines an initial set of tiles to materialize.
+
+If you run the algorithm, you can omit the tiles attribute. Calcite
+will choose an initial set. If you include the tiles attribute, the
+algorithm will start with that list and then start finding other tiles
+that are complementary (i.e. "fill in the gaps" left by the initial
+tiles).
+
+### References
+
+* <a name="ref-hru96">[HRU96]</a> V. Harinarayan, A. Rajaraman and J. Ullman.
+  <a href="http://web.eecs.umich.edu/~jag/eecs584/papers/implementing_data_cube.pdf">Implementing
+  data cubes efficiently</a>. In _Proc. ACM SIGMOD Conf._, Montreal, 1996.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/aee32bc4/doc/model.md
----------------------------------------------------------------------
diff --git a/doc/model.md b/doc/model.md
index 484a302..1b2666d 100644
--- a/doc/model.md
+++ b/doc/model.md
@@ -362,6 +362,8 @@ just 'count(*)':
 [ { name: 'count' } ]
 ```
 
+See also: <a href="lattice.md">Lattices</a>.
+
 ### Tile
 
 Occurs within `root.schemas.lattices.tiles`.


[06/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/history.md
----------------------------------------------------------------------
diff --git a/doc/history.md b/doc/history.md
new file mode 100644
index 0000000..b4ee1cb
--- /dev/null
+++ b/doc/history.md
@@ -0,0 +1,1475 @@
+<!--
+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.
+-->
+# Apache Calcite release history
+
+For a full list of releases, see
+<a href="https://github.com/apache/incubator-calcite/releases">github</a>.
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-1.2.0-incubating">1.2.0-incubating</a> / 2015-04-07
+
+A short release, less than a month after 1.1.
+
+There have been many changes to Avatica, hugely improving its coverage of the
+JDBC API and overall robustness. A new provider, `JdbcMeta`, allows
+you to remote an existing JDBC driver.
+
+[<a href="https://issues.apache.org/jira/browse/CALCITE-606">CALCITE-606</a>]
+improves how the planner propagates traits such as collation and
+distribution among relational expressions.
+
+[<a href="https://issues.apache.org/jira/browse/CALCITE-613">CALCITE-613</a>]
+and [<a href="https://issues.apache.org/jira/browse/CALCITE-307">CALCITE-307</a>]
+improve implicit and explicit conversions in SQL.
+
+New features
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-366">CALCITE-366</a>]
+  Support Aggregate push down in bushy joins (Jesus Camacho Rodriguez)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-613">CALCITE-613</a>]
+  Implicitly convert character values in comparisons
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-307">CALCITE-307</a>]
+  Implement `CAST` between date-time types
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-634">CALCITE-634</a>]
+  Allow `ORDER BY` aggregate function in `SELECT DISTINCT`, provided that it
+  occurs in `SELECT` clause (Sean Hsuan-Yi Chu)
+* In linq4j, implement `firstOrDefault`, `single`, and `singleOrDefault` methods
+  (Daniel Cooper)
+* JDBC adapter
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-631">CALCITE-631</a>]
+    Push theta joins down to JDBC adapter (Ng Jiunn Jye)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-657">CALCITE-657</a>]
+    `NullPointerException` when executing `JdbcAggregate.implement`
+    method (Yuri Au Yong)
+* Metadata
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-659">CALCITE-659</a>]
+    Missing types in `averageTypeValueSize` method in `RelMdSize`
+    (Jesus Camacho Rodriguez)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-650">CALCITE-650</a>]
+    Add metadata for average size of a tuple in `SemiJoin` (Jesus
+    Camacho Rodriguez)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-649">CALCITE-649</a>]
+    Extend `splitCondition` method in `RelOptUtil` to handle multiple
+    joins on the same key (Jesus Camacho Rodriguez)
+
+Avatica features and bug fixes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-670">CALCITE-670</a>]
+  `AvaticaPreparedStatement` should support `execute()` and
+  `executeUpdate()` (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-641">CALCITE-641</a>]
+  Implement logging throughout Avatica server (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-646">CALCITE-646</a>]
+  `AvaticaStatement.execute` method broken over remote JDBC (Yeong Wei
+  and Julian Hyde)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-660">CALCITE-660</a>]
+  Improve Avatica date support
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-655">CALCITE-655</a>]
+  Implement `ConnectionSync` RPC (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-654">CALCITE-654</a>]
+  Tighten up `AvaticaStatement.execute` semantics (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-658">CALCITE-658</a>]
+  Cleanup dependency usage (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-652">CALCITE-652</a>]
+  Move server pieces of `avatica` into `avatica-server` (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-651">CALCITE-651</a>]
+  In `JdbcMeta`, convert property definitions to an enum (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-640">CALCITE-640</a>]
+  Avatica server should expire stale connections/statements (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-639">CALCITE-639</a>]
+  Open up permissions on avatica server components (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-637">CALCITE-637</a>]
+  Implement Avatica `CloseConnection` RPC (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-636">CALCITE-636</a>]
+  Connection isolation for Avatica clients (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-626">CALCITE-626</a>]
+  Implement `CloseStatement` RPC (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-630">CALCITE-630</a>]
+  Flesh out `AvaticaParameter.setObject` (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-627">CALCITE-627</a>]
+  Add Avatica support for `getTableTypes`, `getColumns` (Xavier FH Leong)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-618">CALCITE-618</a>]
+  Add Avatica support for `getTables` (Julian Hyde and Nick Dimiduk)
+
+API changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-617">CALCITE-617</a>]
+  Check at initialization time in `CachingInvocationHandler` that MD provider
+  is not null (Jesus Camacho Rodriguez)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-638">CALCITE-638</a>]
+  SQL standard `REAL` is 4 bytes, `FLOAT` is 8 bytes
+
+Bug-fixes and internal changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-672">CALCITE-672</a>]
+  SQL `ANY` type should be nullable (Jinfeng Ni)
+* Disable tests, pending
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-673">CALCITE-673</a>]
+  Timeout executing joins against MySQL
+* Fix traits in MongoDB adapter, and `NullPointerException` in `JdbcTest`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-662">CALCITE-662</a>]
+  Query validation fails when an `ORDER BY` clause is used with `WITH CLAUSE`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-606">CALCITE-606</a>]
+  Fix trait propagation and add test case
+* Remove checkstyle Eclipse properties from git tracking
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-644">CALCITE-644</a>]
+  Increase check style line limit to 100 chars (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-648">CALCITE-648</a>]
+  Update `ProjectMergeRule` description for new naming convention (Jinfeng Ni)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-625">CALCITE-625</a>]
+  `README.md` linking to the wrong page of `optiq-csv` (hongbin ma)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-632">CALCITE-632</a>]
+  Sort order returned by `SUPERCLASS_COMPARATOR` in
+  `ReflectiveRelMetadataProvider` is inconsistent (Jesus Camacho
+  Rodriguez)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-335">CALCITE-335</a>]
+  Remove uses of linq4j `Functions.adapt`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-592">CALCITE-592</a>]
+  Upgrade to Guava 14.0.1
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-596">CALCITE-596</a>]
+  JDBC adapter incorrectly reads null values as 0 (Ng Jiunn Jye)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-633">CALCITE-633</a>]
+  `WITH ... ORDER BY` cannot find table
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-614">CALCITE-614</a>]
+  `IN` clause in `CASE` in `GROUP BY` gives `AssertionError`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-619">CALCITE-619</a>]
+  Slim down dependencies in parent POM
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-1.1.0-incubating">1.1.0-incubating</a> / 2015-03-13
+
+This Calcite release makes it possible to exploit physical properties
+of relational expressions to produce more efficient plans, introducing
+collation and distribution as traits, `Exchange` relational operator,
+and several new forms of metadata.
+
+We add experimental support for streaming SQL.
+
+This release drops support for JDK 1.6; Calcite now requires 1.7 or
+later.
+
+We have introduced static `create` methods for many sub-classes of
+`RelNode`. We strongly suggest that you use these rather than
+calling constructors directly.
+
+New features
+* SQL
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-602">CALCITE-602</a>]
+    Streaming queries (experimental)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-588">CALCITE-588</a>]
+    Allow `TableMacro` to consume maps and collections
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-583">CALCITE-583</a>]
+    Operator `||` mishandles `ANY` type (Sean Hsuan-Yi Chu)
+* Planner rule improvements
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-445">CALCITE-445</a>]
+    Pull up filters rejected by a `ProjectableFilterableTable`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-600">CALCITE-600</a>]
+    Use `SetOpFactory` in rules containing `Union` operator (Jesus
+    Camacho Rodriguez)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-603">CALCITE-603</a>]
+    Metadata providers for size, memory, parallelism
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-607">CALCITE-607</a>]
+      Change visibility of constructor in metadata providers for size,
+      memory, parallelism (Jesus Camacho Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-608">CALCITE-608</a>]
+      Exception is thrown when `RelMdDistribution` for `Project`
+      operator is called (Jesus Camacho Rodriguez)
+* Collation and distribution as traits
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-88">CALCITE-88</a>]
+    Add collation as a trait and a kind of `RelNode` metadata
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-569">CALCITE-569</a>]
+    `ArrayIndexOutOfBoundsException` when deducing collation (Aman Sinha)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-581">CALCITE-581</a>]
+    Add `LogicalSort` relational expression, and make `Sort` abstract
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-526">CALCITE-526</a>]
+    Add `EnumerableMergeJoin`, which exploits sorted inputs
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-71">CALCITE-71</a>]
+    Provide a way to declare that tables are sorted
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-576">CALCITE-576</a>]
+    Make `RelCollation` trait and `AbstractRelNode.getCollationList` consistent
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-254">CALCITE-254</a>]
+    Propagate `RelCollation` on aliased columns in `JoinRule`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-569">CALCITE-569</a>]
+    `ArrayIndexOutOfBoundsException` when deducing collation
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-594">CALCITE-594</a>]
+    Add `RelDistribution` trait and `Exchange` relational expression
+
+API changes
+* Many sub-classes of `RelNode` now have a static `create` method
+  which automatically sets up traits such as collation and
+  distribution. The constructors are not marked deprecated, but we
+  strongly suggest that you use the `create` method if it exists.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-591">CALCITE-591</a>]
+  Drop support for Java 1.6 (and JDBC 4.0)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-587">CALCITE-587</a>]
+  Upgrade `jetty-server` to 9.2.7.v20150116 and port avatica-server `HttpServer`
+  (Trevor Hartman)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-577">CALCITE-577</a>]
+  Revert temporary API changes introduced in
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-575">CALCITE-575</a>]
+* Add means to create `Context` instances by wrapping objects and by chaining
+  contexts
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-599">CALCITE-599</a>]
+  `EquiJoin` in wrong package (Jesus Camacho Rodriguez)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-573">CALCITE-573</a>]
+  Use user-given names in `RelOptUtil.createProject` and `createRename`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-572">CALCITE-572</a>]
+  Remove `Project.flags` (methods are deprecated, to be removed before 2.0)
+
+Bug-fixes and internal changes
+* Remove the `LICENSE` file of calcite-example-csv (the former
+  optiq-csv) and move its history into main history
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-615">CALCITE-615</a>]
+  AvaticaParameter should be Jackson serializable (Nick Dimiduk)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-612">CALCITE-612</a>]
+  Update AvaticaStatement to handle cancelled queries (Parth Chandra)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-605">CALCITE-605</a>]
+  Reduce dependency on third-party maven repositories
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-611">CALCITE-611</a>]
+  Method `setAggChildKeys` should take into account indicator columns of
+  `Aggregate` operator (Jesus Camacho Rodriguez)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-566">CALCITE-566</a>]
+  `ReduceExpressionsRule` requires planner to have an `Executor`
+* Refactor `TableScanNode.create` method
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-593">CALCITE-593</a>]
+  Validator in `Frameworks` should expand identifiers (Jinfeng Ni)
+* Australian time-zones changed in `tzdata2014f`, Java 1.8.0_31
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-580">CALCITE-580</a>]
+  Average aggregation on an `Integer` column throws `ClassCastException`
+* In Travis, ask Surefire to print results to screen
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-586">CALCITE-586</a>]
+  Prevent JSON serialization of `Signature.internalParameters`
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-1.0.0-incubating">1.0.0-incubating</a> / 2015-01-31
+
+Calcite's first major release.
+
+Since the previous release we have re-organized the into the `org.apache.calcite`
+namespace. To make migration of your code easier, we have described the
+<a href="https://issues.apache.org/jira/secure/attachment/12681620/mapping.txt">mapping from old to new class names</a>
+as an attachment to
+[<a href="https://issues.apache.org/jira/browse/CALCITE-296">CALCITE-296</a>].
+
+The release adds SQL support for `GROUPING SETS`, `EXTEND`, `UPSERT` and sequences;
+a remote JDBC driver;
+improvements to the planner engine and built-in planner rules;
+improvements to the algorithms that implement the relational algebra,
+including an interpreter that can evaluate queries without compilation;
+and fixes about 30 bugs.
+
+New features
+* SQL
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-494">CALCITE-494</a>]
+    Support `NEXT`/`CURRENT VALUE FOR` syntax for using sequences
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-492">CALCITE-492</a>]
+    Support `UPSERT` statement in parser
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-493">CALCITE-493</a>]
+    Add `EXTEND` clause, for defining columns and their types at query/DML time
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-497">CALCITE-497</a>]
+    Support optional qualifier for column name references
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-356">CALCITE-356</a>]
+    Allow column references of the form `schema.table.column`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-462">CALCITE-462</a>]
+    Allow table functions in `LATERAL` expression
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-282">CALCITE-282</a>]
+    Add `{fn QUARTER(date)}` function (Benoy Antony)
+  * Grouping sets
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-370">CALCITE-370</a>]
+      Support `GROUPING SETS`, `CUBE`, `ROLLUP` in SQL and algebra
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-512">CALCITE-512</a>]
+      Add `GROUP_ID`,`GROUPING_ID`, `GROUPING` functions
+* Planner rule improvements
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-92">CALCITE-92</a>]
+    Optimize away `Project` that merely renames fields
+  * Detect and merge duplicate predicates `AND(x, y, x)` to `AND(x, y)` in more
+    circumstances
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-557">CALCITE-557</a>]
+    Speed up planning by never creating `AbstractConverter`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-545">CALCITE-545</a>]
+    When a projected expression can only have one value, replace with that
+    constant
+  * Grouping sets
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-542">CALCITE-542</a>]
+      Support for `Aggregate` with grouping sets in `RelMdColumnOrigins` (Jesus
+      Camacho Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-533">CALCITE-533</a>]
+      Support for grouping sets in `FilterAggregateTransposeRule` (Jesus Camacho
+      Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-532">CALCITE-532</a>]
+      Support for grouping sets in `AggregateFilterTransposeRule` (Jesus Camacho
+      Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-513">CALCITE-513</a>]
+      Support for grouping sets in `AggregateProjectMergeRule` (Jesus Camacho
+      Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-510">CALCITE-510</a>]
+      Support for grouping sets in `AggregateExpandDistinctAggregatesRule` (Jesus
+      Camacho Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-502">CALCITE-502</a>]
+      Support for grouping sets in `AggregateUnionTransposeRule` (Jesus Camacho
+      Rodriguez)
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-503">CALCITE-503</a>]
+      Tests to check rules on `Aggregate` operator without grouping sets (Jesus
+      Camacho Rodriguez)
+* Algorithms
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-451">CALCITE-451</a>]
+    Implement theta join, inner and outer, in enumerable convention
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-489">CALCITE-489</a>]
+    Update `Correlate` mechanics and implement `EnumerableCorrelate` (aka nested
+    loops join)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-544">CALCITE-544</a>]
+    Implement `Union` in interpreter
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-562">CALCITE-562</a>]
+    Implement inner `Join` in interpreter and improve handling of scalar expressions
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-543">CALCITE-543</a>]
+    Implement `Aggregate` (including `GROUPING SETS`) in interpreter (Jacques
+    Nadeau)
+  * In progress towards
+    [<a href="https://issues.apache.org/jira/browse/CALCITE-558">CALCITE-558</a>]
+    add `BINDABLE` convention (but `ENUMERABLE` is still the default), and add
+    `ArrayBindable` and `Scalar` interfaces
+* Remote driver
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-93">CALCITE-93</a>]
+    Calcite RPC server
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-94">CALCITE-94</a>]
+    Remote JDBC driver
+  * Make `JsonHandler` and `JsonService` thread-safe
+
+API changes
+* The great code re-org
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-296">CALCITE-296</a>]
+    Re-organize package structure
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-419">CALCITE-419</a>]
+    Naming convention for planner rules
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-306">CALCITE-306</a>]
+    Standardize code style for "import package.*;"
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-474">CALCITE-474</a>]
+    Clean up rule naming in order to support enabling/disabling rules
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-460">CALCITE-460</a>]
+    Add `ImmutableBitSet` and replace uses of `BitSet`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-479">CALCITE-479</a>]
+    Migrate `RelNode.getChildExps` to `RelNode.accept(RexShuttle)`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-527">CALCITE-527</a>]
+    Drop `rowType` field and constructor/copy argument of `Calc`
+* Add linq4j and example-csv modules
+  * Remove unused packages in linq4j, and fix checkstyle issues in linq4j and csv
+  * Add calcite-linq4j and calcite-example-csv as POM sub-modules
+  * Import 'optiq-csv' project as 'example/csv/', and add Apache headers
+  * Import 'linq4j' project, and add Apache headers
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-478">CALCITE-478</a>]
+    Move CSV tutorial (Siva Narayanan)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-464">CALCITE-464</a>]
+  Make parser accept configurable max length for SQL identifier
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-465">CALCITE-465</a>]
+  Remove `OneRow` and `Empty` relational expressions; `Values` will suffice
+
+Bug-fixes and internal changes
+* Build improvements
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-541">CALCITE-541</a>]
+    Update maven-source-plugin to 2.4 to get speedup in jdk 1.8
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-537">CALCITE-537</a>]
+    Skip overwrite of `NOTICE`, `DEPENDENCIES`, and `LICENSE` files
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-538">CALCITE-538</a>]
+    Generate `Parser.jj` only at first build
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-539">CALCITE-539</a>]
+    Avoid rewrite of `org-apache-calcite-jdbc.properties`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-540">CALCITE-540</a>]
+    Create git.properties file only at first build. This saves time in
+    development at a cost of stale `git.properties`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-536">CALCITE-536</a>]
+    Add `@PackageMarker` to `package-info.java` so maven-compiler skips
+    compilation when the sources are unchanged
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-535">CALCITE-535</a>]
+    Support skip overwrite in hydromatic-resource
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-582">CALCITE-582</a>]
+  `EnumerableTableScan` broken when table has single column
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-575">CALCITE-575</a>]
+  Variant of `ProjectRemoveRule` that considers a project trivial only if its
+  field names are identical (John Pullokkaran)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-571">CALCITE-571</a>]
+  `ReduceExpressionsRule` tries to reduce `SemiJoin` condition to non-equi
+  condition
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-568">CALCITE-568</a>]
+  Upgrade to a version of `pentaho-aggdesigner` that does not pull in
+  `servlet-api`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-567">CALCITE-567</a>]
+  Make `quidem` dependency have scope "test"
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-570">CALCITE-570</a>]
+  `ReduceExpressionsRule` throws "duplicate key" exception
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-561">CALCITE-561</a>]
+  Upgrade parent POM
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-458">CALCITE-458</a>]
+  ArrayIndexOutOfBoundsException when using just a single column in interpreter
+* Fix spurious extra row from `FULL JOIN`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-554">CALCITE-554</a>]
+  Outer join over NULL keys generates wrong result
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-489">CALCITE-489</a>]
+  Teach `CalciteAssert` to respect multiple settings
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-516">CALCITE-516</a>]
+  `GROUP BY` on a `CASE` expression containing `IN` predicate fails (Aman Sinha)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-552">CALCITE-552</a>]
+  Upgrade tpcds (which depends on an old version of guava)
+* Copy identifier when fully-qualifying, so column aliases have the right case
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-548">CALCITE-548</a>]
+  Extend `induce` method to return `CUBE` and `ROLLUP` (Jesus Camacho Rodriguez)
+  * Simplify `Group.induce` by assuming that group sets are sorted
+* Test case for
+  [<a  href="https://issues.apache.org/jira/browse/CALCITE-212">CALCITE-212</a>]
+  Join condition with `OR`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-550">CALCITE-550</a>]
+  Case-insensitive matching of sub-query columns fails
+  * Add more unit tests (Jinfeng Ni)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-448">CALCITE-448</a>]
+  `FilterIntoJoinRule` creates filters containing invalid `RexInputRef`
+* When registering a `RelNode`, be tolerant if it is equivalent to a `RelNode`
+  with different traits
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-547">CALCITE-547</a>]
+  Set nullability while inferring return type of `item(any,...)` operator
+* In Travis CI, enable containers, and cache `.m2` directory
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-534">CALCITE-534</a>]
+  Missing implementation of `ResultSetMetaData.getColumnClassName` (Knut
+  Forkalsrud)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-506">CALCITE-506</a>]
+  Update `EnumerableRelImplementor.stash` so it is suitable for all kinds of
+  classes
+* Merge join algorithm for `Enumerable`s
+* Efficient `Enumerable` over random-access list
+* Add a test that calls all functions with arguments of all types that they
+  claim to accept
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-511">CALCITE-511</a>]
+  `copy` method in `LogicalAggregate` not copying the indicator value properly
+* Add a model that has lattices and works against HSQLDB
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-509">CALCITE-509</a>]
+  `RelMdColumnUniqueness` uses `ImmutableBitSet.Builder` twice, gets
+  `NullPointerException`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-488">CALCITE-488</a>]
+  `Enumerable<Holder>` does not work if where `Holder` is a custom class
+  with a single field; Calcite tries to treat it as `SCALAR` due to premature
+  `JavaRowFormat.optimize`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-352">CALCITE-352</a>]
+  Throw exception if `ResultSet.next()` is called after `close()`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-403">CALCITE-403</a>]
+  `Enumerable` gives `NullPointerException` with `NOT` on nullable expression
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-469">CALCITE-469</a>]
+  Update example/csv README.md instructions
+* Document `WITH`, `LATERAL`, `GROUPING SETS`, `CUBE`, `ROLLUP`;
+  add descriptions for all built-in functions and operators
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-470">CALCITE-470</a>]
+  Print warning when column type hint is not understood;
+  Update `EMPS.deptno` column Integer &rarr; int
+* Fix `Linq4j.product`; the cartesian product of 0 attributes is one row of 0
+  attributes
+* Update link optiq-mat-plugin &rarr; mat-calcite-plugin
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-467">CALCITE-467</a>]
+  Incorrect namespace in `package-info.java`
+* Add headers, to appease the RAT
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-446">CALCITE-446</a>]
+  CSV adapter should read from directory relative to the model file
+* Add examples of scannable and filterable tables, matching
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-436">CALCITE-436</a>]
+  Simpler SPI to query Table
+* Fix `JdbcTest.testVersion` now that version is 1.0
+* Update release HOWTO
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-0.9.2-incubating">0.9.2-incubating</a> / 2014-11-05
+
+A fairly minor release, and last release before we rename all of the
+packages and lots of classes, in what we expect to call 1.0. If you
+have an existing application, it's worth upgrading to this first,
+before you move on to 1.0.
+
+New features
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-436">CALCITE-436</a>]
+  Simpler SPI to query `Table`
+
+API changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-447">CALCITE-447</a>]
+  Change semi-join rules to make use of factories
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-442">CALCITE-442</a>
+  Add `RelOptRuleOperand` constructor that takes a predicate
+
+Bug-fixes and internal changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-397">CALCITE-397</a>]
+  `SELECT DISTINCT *` on reflective schema gives `ClassCastException` at runtime
+* Various lattice improvements.
+* sqlline: Looking for class-path in inconsistent locations.
+* Re-order test suite, so that fast tests are run first.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-444">CALCITE-444</a>]
+  Filters wrongly pushed into full outer join
+* Make it more convenient to unit test `RelMetadataQuery`, and add some more
+  tests for
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-443">CALCITE-443</a>]
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-443">CALCITE-443</a>]
+  `getPredicates` from a Union is not correct
+* Update references to web sites, git repositories, jira, mailing lists,
+  travis CI now that [INFRA-8413] is fixed
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-434">CALCITE-435</a>]
+  `FilterAggregateTransposeRule` loses conditions that cannot be pushed
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-435">CALCITE-435</a>]
+  `LoptOptimizeJoinRule` incorrectly re-orders outer joins
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-439">CALCITE-439</a>]
+  `SqlValidatorUtil.uniquify()` may not terminate under some conditions
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-438">CALCITE-438</a>]
+  Push predicates through `SemiJoinRel`
+* Add test case for `LIKE ... ESCAPE`.
+* HOWTO: Modify release instructions.
+* Update `DiffRepository` documentation.
+* Add tests for windowed aggregates without `ORDER BY`. (Works already.)
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-0.9.1-incubating">0.9.1-incubating</a> / 2014-10-02
+
+This is the first release as Calcite. (The project was previously called Optiq.)
+
+New features
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-430">CALCITE-430</a>]
+  Rename project from Optiq to Calcite
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-426">CALCITE-426</a>]
+  Pool JDBC data sources, to make it easier to pool connections
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-416">CALCITE-416</a>]
+  Execute logical `RelNode`s using an interpreter
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-376">CALCITE-376</a>]
+  Move `SqlRun` into its own artifact,
+  <a href="https://github.com/julianhyde/quidem">Quidem</a>.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-269">CALCITE-269</a>]
+  MongoDB result sets larger than 16MB
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-373">CALCITE-373</a>]
+  `NULL` values in `NOT IN` sub-queries
+* SQL functions:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-422">CALCITE-422</a>]
+    Add `REGR_SXX` and `REGR_SYY` regression functions
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-421">CALCITE-421</a>]
+    Add `COVAR_POP` and `COVAR_SAMP` aggregate functions
+* Planner rules:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-425">CALCITE-425</a>]
+    Add `FilterAggregateTransposeRule`, that pushes a filter through an
+    aggregate
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-399">CALCITE-399</a>]
+    Factorize common `AND` factors out of `OR` predicates
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-404">CALCITE-404</a>]
+    `MergeProjectRule` should not construct `RexProgram`s for simple mappings
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-394">CALCITE-394</a>]
+    Add `RexUtil.toCnf()`, to convert expressions to conjunctive normal form
+    (CNF)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-389">CALCITE-389</a>]
+    `MergeFilterRule` should flatten `AND` condition
+* Lattices:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-428">CALCITE-428</a>]
+    Use optimization algorithm to suggest which tiles of a lattice to
+    materialize
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-410">CALCITE-410</a>]
+    Allow lattice tiles to satisfy a query by rolling up
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-406">CALCITE-406</a>]
+    Add tile and measure elements to lattice model element
+  * Now, a lattice can materialize an aggregate-join and use it in a subsequent
+    query.
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-402">CALCITE-402</a>]
+    Lattice should create materializations on demand
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-344">CALCITE-344</a>]
+    Lattice data structure
+* Field trimmer:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-408">CALCITE-408</a>]
+    Make `FieldTrimmer` work with `RelNode` base classes
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-388">CALCITE-388</a>]
+    Handle semi-joins in field trimmer
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-395">CALCITE-395</a>]
+    Make `FieldTrimmer.trimFields(SetOp)` generate `ProjectRel` instead of
+    `CalcRel`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-393">CALCITE-393</a>]
+    If no fields are projected from a table, field trimmer should project a
+    dummy expression
+
+API changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-413">CALCITE-413</a>]
+  Add `RelDataTypeSystem` plugin, allowing different max precision of a
+  `DECIMAL`
+* In `Planner`, query de-correlation no longer requires state in a
+  `SqlToRelConverter`.
+* Factories:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-392">CALCITE-392</a>]
+    `RelFieldTrimmer` should use factory to create new rel nodes
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-382">CALCITE-382</a>]
+    Refactoring rules to use factories
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-398">CALCITE-398</a>]
+    Move `CalcRel.createProject` methods to `RelOptUtil`
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-396">CALCITE-396</a>]
+    Change return type of `JoinFactory.createJoin()`; add `SemiJoinFactory`
+
+Bug-fixes and internal changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-386">CALCITE-386</a>]
+  Fix NOTICE
+* Add tests inspired by Derby bugs.
+* Add recent presentation to README.md.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-427">CALCITE-427</a>]
+  Off-by-one issues in `RemoveDistinctAggregateRule`,
+  `AggregateFilterTransposeRule`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-414">CALCITE-414</a>]
+  Bad class name in `sqlline` shell script
+* Bad package name in `package-info.java` was causing errors in Eclipse.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-412">CALCITE-412</a>]
+  `RelFieldTrimmer`: when trimming `SortRel`, the collation and trait set don't
+  match
+* Add test case for
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-411">CALCITE-411</a>]
+  Duplicate column aliases
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-407">CALCITE-407</a>]
+  `RemoveTrivialProjectRule` drops child node's traits
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-409">CALCITE-409</a>]
+  `PushFilterPastProjectRule` should not push filters past windowed aggregates
+* Fix tests on Windows.
+* Don't load `FoodMartQuerySet` unless we have to. It's big.
+* Enable connection pooling in test suite.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-384">CALCITE-384</a>]
+  Add `apache-` prefix to tarball and directory within tarball
+* Freeze hive fmpp > freemarker plugin dependency.
+* Upgrade Janino
+* Removed hardcoded foodmart schema information
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-387">CALCITE-387</a>]
+  CompileException when cast TRUE to nullable boolean
+* Temporary fix for
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-390">CALCITE-390</a>]
+  Transitive inference (`RelMdPredicates`) doesn't handle semi-join
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-385">CALCITE-385</a>]
+  Change comment style for Java headers
+* Disable test that is inconistent between JDK 1.7 and 1.8.
+* Fix `git-commit-id-plugin` error when running in Travis-CI.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-381">CALCITE-381</a>]
+  Remove plugin versions from the `&lt;plugins&gt;` tag in root pom
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-383">CALCITE-383</a>]
+  Each jar should have a `git.properties` file describing its exact version
+* Fix `mvn site` on JDK 1.8 and enable in Travis-CI.
+* Status icon based on master branch, not whichever branch happened to build
+  most recently.
+* HOWTO:
+  * Document how to build from git, and how to get jars from maven repo.
+  * Optiq web site
+  * Template emails for Apache votes
+  * Update JIRA cases following release
+  * Instructions for making and verifying a release
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-0.9.0-incubating">0.9.0-incubating</a> / 2014-08-19
+
+This is the first release under the Apache incubator process.
+
+New features
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-371">CALCITE-371</a>]
+  Implement `JOIN` whose `ON` clause contains mixed equi and theta
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-369">CALCITE-369</a>]
+  Add `EnumerableSemiJoinRel`, implementation of semi-join in enumerable
+  convention
+* Add class `Strong`, for detecting null-rejecting predicates.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-368">CALCITE-368</a>]
+  Add SemiJoinRule, planner rule to convert project-join-aggregate into semi-join
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-367">CALCITE-367</a>]
+  `PushFilterPastJoinRule` should strengthen join type
+* Add `EquiJoinRel`, base class for joins known to be equi-joins.
+* Implement `CAST(&lt;string&gt; AS &lt;datetime&gt;)` and
+  `&lt;datetime&gt; + &lt;interval&gt;`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-360">CALCITE-360</a>]
+  Introduce a rule to infer predicates from equi-join conditions
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-349">CALCITE-349</a>]
+  Add heuristic join-optimizer that can generate bushy joins
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-346">CALCITE-346</a>]
+  Add commutative join rule
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-347">CALCITE-347</a>]
+  In `SqlRun`, add `!plan` command
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-314">CALCITE-314</a>]
+  Allow simple UDFs based on methods
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-327">CALCITE-327</a>]
+  Rules should use base class to find rule match & use factory for object
+  creation
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-316">CALCITE-316</a>]
+  In `SqlRun`, match output regardless of order if `ORDER BY` not present
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-300">CALCITE-300</a>]
+  Support multiple parameters in `COUNT(DISTINCT x, y, ...)`
+
+API changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-343">CALCITE-343</a>]
+  RelDecorrelator should build its own mappings, not inherit from SqlToRelConverter
+* Remove deprecated methods.
+* Convert `Hook` to use Guava `Function` (was linq4j `Function1`).
+* Add fluent method `withHook`, to more easily add hooks in tests.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-321">CALCITE-321</a>]
+  Add support for overriding implementation of `CompoundIdentifier` in
+  `SqlParser`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-322">CALCITE-322</a>]
+  Add support for `SqlExplain`, `SqlOrderBy` and `SqlWith` to support
+  `SqlShuttle` use.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-323">CALCITE-323</a>]
+  Override `SqlUnresolvedFunction.inferReturnType()` to return `ANY` type
+  so framework implementors can support late bound function implementations.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-324">CALCITE-324</a>]
+  Add `ViewExpander` for `Planner` in `Frameworks`. Expose additional
+  properties of `ViewTable` to allow subclassing.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-247">CALCITE-247</a>]
+  Add `Context` and `FrameworkConfig`
+
+Bug-fixes and internal changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-380">CALCITE-380</a>]
+  Downgrade to Guava 11.0.2
+* Move several .md files into new 'doc' directory, to keep the root directory simple.
+* Add DISCLAIMER
+* Update history and HOWTO
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-377">CALCITE-377</a>]
+  UnregisteredDriver should catch, log and re-throw NoClassDefFoundError
+* Inherit maven-release-plugin from Apache POM.
+* Test case for
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-373">CALCITE-373</a>]
+  NOT IN and NULL values
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-372">CALCITE-372</a>]
+  Change `LoptOptimizeJoinRule` &amp; `PushFilterPast`* rules to use factory
+* Upgrade `maven-checkstyle-plugin`.
+* Add class `Holder`, a mutable slot that can contain one object.
+* Remove the 2-minute wait at the top of the hour for tests of
+  `CURRENT_TIME`, etc.
+* Tune `ImmutableIntList`'s iterators.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-364">CALCITE-364</a>]
+  Validator rejects valid `WITH ... ORDER BY` query
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-363">CALCITE-363</a>]
+  Use `dependencyManagement` and `pluginManagement` in POM files
+* Add `FilterFactory`.
+* Add `README` file, incubation disclaimers, and how-to build and running tests.
+* Add `KEYS` and start how-to for making snapshots and releases.
+* Capital case component names; inherit license info from Apache parent POM.
+* Only run `apache-rat` and `git-commit-id-plugin` in "release" maven profile.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-348">CALCITE-348</a>]
+  Add Apache RAT as maven plugin
+* Change license headers from "Julian Hyde" to "ASF"; add headers where missing.
+* Fix build breakage on JDK 1.6 due to missing method `BitSet.previousClearBit`.
+* Refactor test infrastructure to allow testing against heuristic bushy-join
+  optimizer.
+* Add methods and tests for BitSets, and re-organize tests.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-354">CALCITE-354</a>]
+  Change maven groupId to "org.apache.optiq"
+* Specify return type when calling `RexBuilder.makeCall`, if possible.
+* Eliminate duplicate conditions in `RexProgramBuilder.addCondition`, not
+  `RexBuilder.makeCall` as previously.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-345">CALCITE-345</a>]
+  `AssertionError` in `RexToLixTranslator` comparing to date literal
+* Restore `PushFilterPastJoinRule` to `RelDecorrelator`; interim pending
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-343">CALCITE-343</a>]
+  fix.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-340">CALCITE-340</a>]
+  Fix bug in `SqlToRelConverter` when push expressions in join conditions into
+  `ProjectRel`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-313">CALCITE-313</a>]
+  Query decorrelation fails
+* While unifying a `RelNode` tree with a materialized view expression,
+  switch representation to `MutableRel`s.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-305">CALCITE-305</a>]
+  Unit test failure on release candidates
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-325">CALCITE-325</a>]
+  Use Java list instead of Guava list to avoid null checks in case of
+  `SqlTypeExplicitPrecedenceList`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-326">CALCITE-326</a>]
+  Fix `RelOptUtil` `ANY` type check.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-303">CALCITE-303</a>]
+  Migrate issue URLs
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-331">CALCITE-331</a>]
+  Precision/scale compatibility checks should always succeed for `ANY` type
+* In `SqlRun`, allow `!plan` after `!ok` for same SQL statement.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-318">CALCITE-318</a>]
+  Add unit test for `SqlRun`
+* Fix a bug where composite `SELECT DISTINCT` would return duplicate rows.
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.8">0.8</a> / 2014-06-27
+
+New features
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-310">CALCITE-310</a>]
+   Implement LEAD, LAG and NTILE windowed aggregates
+* Reduce `COUNT(not-null-expression)` to `COUNT()`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-292">CALCITE-292</a>]
+   Improve windowed aggregate return types
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-302">CALCITE-302</a>]
+   Use heuristic rule to plan queries with large numbers of joins
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-283">CALCITE-283</a>]
+  Add TPC-DS data generator
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-294">CALCITE-294</a>]
+  Implement DENSE_RANK windowed aggregate function
+* SqlRun utility
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-290">CALCITE-290</a>]
+    Add `SqlRun`, an idempotent utility for running SQL test scripts
+  * Add "!skip" command to SqlRun.
+  * Add MySQL formatting mode to SqlRun.
+
+API changes
+* Re-organize planner initialization,
+  to make it easier to use heuristic join order.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-301">CALCITE-301</a>]
+  Add `Program` interface, a planner phase more general than current `RuleSet`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-263">CALCITE-263</a>]
+  Add operand type that will cause a rule to fire when a new subset is created
+* Clean up and document SqlKind.
+  * Add `IS_NOT_TRUE` and `IS_NOT_FALSE` `SqlKind` enums.
+  * Add `SqlKind.IS_NOT_NULL` enum value, and use where possible,
+    including for `IS_NOT_UNKNOWN` operator.
+
+Bug-fixes and internal changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-312">CALCITE-312</a>]
+  Trim non-required fields before `WindowRel`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-311">CALCITE-311</a>]
+  Wrong results when filtering the results of windowed aggregation
+* More tests for `WITH ... ORDER BY`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-309">CALCITE-309</a>]
+  `WITH ... ORDER BY` query gives `AssertionError`
+* Enable `MultiJoinRel` and some other planner rule tests.
+* Add `ImmutableNullableList` and `UnmodifiableArrayList`,
+  and remove calls to `Arrays.asList`.
+* Add method `IntPair.zip`.
+* Reimplement regular and windowed aggregates
+* Switch from github to Apache JIRA for issues tracking.
+  * In release history, update issue URLs from github to Apache JIRA
+* The Apache mailing list is now the official mailing list. Add presentations.
+* Add test for overloaded UDF.
+* Add tests for `NOT IN` where sub-query returns NULL values.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-288">CALCITE-288</a>]
+  Add tests for windowed aggregation based on Postgres reference queries
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-286">CALCITE-286</a>]
+  Error casting MongoDB date
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-284">CALCITE-284</a>]
+  Window functions range defaults to `CURRENT ROW`
+* [<a href=https://issues.apache.org/jira/browse/CALCITE-285">CALCITE-285</a>]
+  Window functions throw exception without `ORDER BY`
+* Test case for
+  [<a href=““https://issues.apache.org/jira/browse/CALCITE-285”>CALCITE-285</a>].
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-281">CALCITE-281</a>]
+  `EXTRACT` function's SQL return type is `BIGINT` but implemented as Java `int`
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.7">0.7</a> / 2014-05-13
+
+New features
+* Implement table functions.
+* Arrays and multi-sets:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-267">CALCITE-267</a>]
+    Improve support for ARRAY data type
+  * Better type information for JDBC Array; nested array now possible.
+  * Implement `JOIN LATERAL` and `JOIN UNNEST`.
+  * Implement the `UNNEST` relational operator, and various improvements
+    to `ARRAY` and `MULTISET` data types.
+  * Represent `ARRAY` columns as Java lists.
+  * Implement `CARDINALITY(ARRAY)` SQL operator.
+* Implement scalar sub-query in `SELECT` clause.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-273">CALCITE-273</a>]
+  Support column alias in WITH queries (common table expressions)
+* Windowed aggregates:
+  * Aggregate over constants, e.g. `SUM(1) OVER (ROWS 10 PRECEDING)`;
+  * `UNBOUNDED PRECEDING` window range;
+  * Windowed aggregates computed over primitive scalars.
+* Fix return type inference for aggregate calls. If the `GROUP BY` clause is
+  empty, `SUM` may return null.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-37">CALCITE-37</a>]
+  Document JSON model file format (as <a href="model.md">model.md</a>).
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-238">CALCITE-238</a>]
+  Add adapter that generates TPC-H data
+* Improve exception message in `AvaticaConnection`; add
+  `ExceptionMessageTest`.
+* Implement micro-benchmarks via
+  <a href="http://openjdk.java.net/projects/code-tools/jmh/">JMH</a>.
+
+API changes
+* Provide an option to create root schema without the "metadata" schema.
+* Schema SPI:
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-175">CALCITE-175</a>]
+    Modify Schema SPI to allow caching
+  * Get sub-schemas defined by a Schema SPI, and cache their `OptiqSchema`
+    wrappers. (Tobi Vollebregt and Julian Hyde)
+* SqlAdvisor callable from client via JDBC.
+
+Bug-fixes and internal changes
+* Add Apache incubator proposal.
+* Rename RELEASE.md to HISTORY.md.
+* Upgrade maven-release-plugin.
+* Upgrade to linq4j-0.3.
+* Code generation improvements:
+ * Move code-generation optimizer to linq4j;
+ * Improve translation of strict functions;
+ * Mark most methods in `SqlFunctions` as `@Deterministic`;
+ * Support `static final` constants generated by linq4j.
+ * Avoid excessive box and unbox of primitives when using `Object[]` storage.
+ * In JDBC result set, avoid row computation on each accessor call.
+* Test composite join conditions in various flavors of outer join.
+* Use `fromTrait` of the just previously converted `RelNode` instead
+  of the original `RelNode`.
+* Disable a MongoDB test, pending
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-270">CALCITE-270</a>].
+* Hush warnings from `SplunkAdapterTest` if Splunk is not available.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-252">CALCITE-252</a>]
+  Scalar sub-query that returns 0 rows should become NULL value
+* `SplunkAdapterTest` now uses the same Foodmart database as `JdbcTest`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-242">CALCITE-242</a>]
+  SplunkAdapterTest fails
+* Remove some obsolete classes.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-205">CALCITE-205</a>]
+  Suspicious map.get in VolcanoPlanner.reregister
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.6">0.6</a> / 2014-04-11
+
+New features
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-214">CALCITE-214</a>]
+  Modify Frameworks to allow Schema to be re-used
+  Obsoletes `name` field of `ReflectiveSchema`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-237">CALCITE-237</a>]
+  Allow user-defined aggregate functions (UDAs) to be defined in a model
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-227">CALCITE-227</a>]
+  Extend `EXTRACT` function to support `DATE`, `TIME` and `TIMESTAMP` values
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-222">CALCITE-222</a>]
+  User-defined table macros
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-179">CALCITE-179</a>]
+  Optiq on Windows
+  * Add `sqlline.bat` and fix issues running `sqlline` under Cygwin.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-195">CALCITE-195</a>]
+  Push aggregation into MongoDB adapter
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-193">CALCITE-193</a>]
+  Implement OFFSET and LIMIT in MongoDB adapter
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-164">CALCITE-164</a>]
+  Improve query performance of optiq over MongoDB
+* Add Phoenix (HBase) SQL dialect (Bruno Dumon)
+
+API changes
+* Obsolete `RexImpTable.AggregateImplementor` and rename `AggImplementor2`.
+  (**This is a breaking change**.)
+* Convert `CombinedParser.jj` into freemarker template to allow
+  custom parser implementations. (Venki Korukanti)
+* Extend `Planner` to pass a custom `ConvertletTable` and custom SQL parser.
+* In `Frameworks`, add a way to specify list of `TraitDef`s that will be used
+  by planner. (Jinfeng Ni)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-198">CALCITE-198</a>]
+  Use `RexExecutor` to evaluate projections and filters
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-219">CALCITE-219</a>]
+  Parse `ALTER scope SET option = value` statement
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-215">CALCITE-215</a>]
+  A Schema should not have to remember its name and parent
+  (**This is a breaking change**.)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-180">CALCITE-180</a>]
+  Common base class for TableFunction, ScalarFunction
+  (**This is a breaking change**.)
+* Add methods for dealing with symbols; deprecate
+  `SqlLiteral.booleanValue(SqlNode)`, `SqlLiteral.symbolValue(SqlNode)`.
+* Add `RelOptPlanner.clear()`; now it is safe to call `transform` twice.
+  (Jinfeng Ni)
+* Remove APIs deprecated for 0.5.
+* Move around some operator classes and singletons.
+
+Bug fixes and internal changes
+* Upgrade to linq4j-0.2.
+* `FETCH` and `LIMIT` are ignored during SQL-to-RelNode translation.
+  (Venki Korukanti)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-245">CALCITE-245</a>]
+  Off-by-one translation of ON clause of JOIN
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-191">CALCITE-191</a>]
+  Rotate time/date/timestamp vals to local timezone
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-244">CALCITE-244</a>]
+  `RelOptTableImpl.create` always expects `QueryableTable` type in
+  `OptiqSchema.TableEntry`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-225">CALCITE-225</a>]
+  Optiq doesn't correctly decorrelate queries
+* Clean up package-info.  Remove duplicates in test packages so they
+  don't conflict with those in non-test packages.
+* Add `Pair.adjacents(Iterable)`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-199">CALCITE-199</a>]
+  Various `ANY` type conditions aren't correctly being considered
+  (Jacques Nadeau)
+* Add files to `.gitignore` that shouldn't be checked in when using
+  Eclipse. (Jacques Nadeau)
+* Add class `ControlFlowException`, and make it base class of
+  existing control-flow exception classes.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-232">CALCITE-232</a>]
+  Sum and avg of empty set should be null as per SQL specification
+* Add `SqlUnresolvedFunction`, to improve how return type of
+  user-defined functions is resolved. (Vladimir Sitnikov)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-228">CALCITE-228</a>]
+  Error while compiling generated Java code when using UDF in expression
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-226">CALCITE-226</a>]
+  User-defined functions should work without explicit schema prefix
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-229">CALCITE-229</a>]
+  Join between different JDBC schemas not implementable
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-230">CALCITE-230</a>]
+  RemoveSortRule derives trait set from sort, should derive it from sort's child
+* Test view and sub-query with `ORDER BY` and `LIMIT`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-223">CALCITE-223</a>]
+  Add `NOTICE` and `LICENSE` files in all generated JAR files
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-209">CALCITE-209</a>]
+  Consistent strategy for line-endings in tests
+  Convert uses of `NL` in tests to Linux newline "\n".
+  This makes string constants simpler.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-218">CALCITE-218</a>]
+  Functions case sensitive when using `Lex.MYSQL`
+* Add tests that a query with aggregate expressions in the `SELECT`
+  clause is considered an aggregate query, even if there is no `GROUP BY`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-216">CALCITE-216</a>]
+  Inconsistent use of provided operator table causes inability to
+  add aggregate functions
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-200">CALCITE-200</a>]
+  Javadoc generation fails under JDK 1.8
+* Add class `XmlOutput` (based on `org.eigenbase.xom.XMLOutput`) and remove
+  dependency on eigenbase-xom.
+* Performance: Don't create stack-trace for exceptions used for control-flow.
+  (Vladimir Sitnikov)
+* Performance: Tune `RexProgramBuilder` by using `Pair` rather than `String` as
+  expression key. (Vladimir Sitnikov)
+* Fix NPE using TRIM function with JDBC. (Bruno Dumon)
+* Add dependency on
+  <a href="https://github.com/julianhyde/hydromatic-resource">hydromatic-resource-maven-plugin</a>
+  and obsolete our copy of the resource framework.
+* Fix race condition in `SpaceList`.
+* In planner, use `RelTrait.subsumes` rather than `equals` in an assert.
+  (Jinfeng Ni)
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.5">0.5</a> / 2014-03-14
+
+New features
+* Allow `quoting`, `quotedCasing`, `unquotedCasing`, and `caseSensitive`
+  properties to be specified explicitly (Vladimir Sitnikov)
+* Recognize more kinds of materializations, including filter-on-project (where
+  project contains expressions) and some kinds of aggregation.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-128">CALCITE-128</a>]
+  Support `WITH` queries (common table expressions)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-53">CALCITE-53</a>]
+  Allow `WHEN` clause in simple `CASE` expression to have multiple values
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-156">CALCITE-156</a>]
+  Optiq should recognize 'SYSTEM TABLE', 'JOIN', 'INDEX' as table types
+* Support querying ARRAY columns from JDBC source. (Gabriel Reid)
+
+API changes
+* Add `ProjectRelBase.copy(RelTraitSet, RelNode, List&lt;RexNode&gt;,
+  RelDataType)` and make `ProjectRelBase.copy(RelTraitSet, RelNode)` final.
+  (**This is a breaking change** for sub-classes of `ProjectRelBase`.)
+* Change `RexBuilder.makeRangeReference` parameter type.
+* `RexBuilder.makeInputRef` replaces `RelOptUtil.createInputRef`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-160">CALCITE-160</a>]
+  Allow comments in schema definitions
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-147">CALCITE-147</a>]
+  Create a new kind of `SqlCall` that keeps operands in fields, not an operands
+  array
+  * Very widely used parse tree nodes with complex operands, including
+    `SqlSelect`, `SqlJoin`, `SqlInsert`, and a new node type `SqlOrderBy`, are
+    now sub-classes of `SqlCall` but not `SqlBasicCall`.
+  * (**This is a breaking change** to code that assumes that, say,
+    `SqlSelect` has an `operands` field.)
+* Convert all enum constants to upper-case.
+  (**This is a breaking change**.)
+
+Bug-fixes and internal changes
+* Generate optiq-core-VERSION-tests.jar not parent-VERSION-tests.jar.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-176">CALCITE-176</a>]
+  ORDER BY expression doesn't work with SELECT \*
+* Fix VARCHAR casts sent to hsqldb source (Bruno Dumon)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-143">CALCITE-143</a>]
+  Remove dependency on eigenbase-resgen
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-173">CALCITE-173</a>]
+  Case-insensitive table names are not supported for `Casing.UNCHANGED`
+* `DATE.getLimit` now returns `Calendar` in GMT time zone (Vladimir Sitnikov)
+* Set `en_US` locale in tests that match against error numbers, dates
+  (Vladimir Sitnikov)
+* Use 1 test thread per CPU to avoid thread starvation on dual core CPUs
+  (Vladimir Sitnikov)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-174">CALCITE-174</a>]
+  Move hsqldb to test scope
+* Add unit tests for `RexExecutorImpl`.
+* Correct JSON model examples in Javadoc comments. (Karel Vervaeke)
+* Move test reference logs from `src/test/java` to `src/test/resources`
+  (reduces the number of 'untracked files' reported by git)
+* Tune `Util.SpaceList`, fix race condition, and move into new utility class
+  `Spaces`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-163">CALCITE-163</a>]
+  Equi-join warning
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-157">CALCITE-157</a>]
+  Handle `SQLFeatureNotSupported` when calling `setQueryTimeout`
+  (Karel Vervaeke)
+* Fix Optiq on Windows. (All tests and checkstyle checks pass.)
+* In checkstyle, support Windows-style file separator, otherwise build fails in
+  Windows due to suppressions not used. (Vladimir Sitnikov)
+* Enable MongoDB tests when `-Dcalcite.test.mongodb=true`.
+* Cleanup cache exception-handling and an assert.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-153">CALCITE-153</a>]
+  Error using MongoDB adapter: Failed to set setXIncludeAware(true)
+* Disable spark engine unless Spark libraries are on the class path and
+  `spark=true` is specified in the connect string.
+* Fix path to `mongo-zips-model.json` in HOWTO. (Mariano Luna)
+* Fix bug deriving the type of a join-key.
+* Fix the value of `ONE_MINUS_EPSILON`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-158">CALCITE-158</a>]
+  Optiq fails when call `Planner.transform()` multiple times, each with
+  different ruleset
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-148">CALCITE-148</a>]
+ Less verbose description of collation. Also, optimize `RelTraitSet` creation
+ and amortize `RelTraitSet.toString()`.
+* Add generics to SQL parser.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-145">CALCITE-145</a>]
+  Unexpected upper-casing of keywords when using java lexer
+* Remove duplicate `maven-source-plugin`.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-141">CALCITE-141</a>]
+  Downgrade to guava-11.0.2. (This is necessary for Hadoop compatibility.
+  Later versions of Guava can also be used.)
+* Upgrade to spark-0.9.0. (Because this version of spark is available from
+  maven-central, we can make optiq-spark part of the regular build, and remove
+  the spark profile.)
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.18">0.4.18</a> / 2014-02-14
+
+API and functionality changes
+* Configurable lexical policy
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-33">CALCITE-33</a>]
+      SQL parser should allow different identifier quoting
+    * [<a href="https://issues.apache.org/jira/browse/CALCITE-34">CALCITE-34</a>]
+      Policy for case-sensitivity of identifiers should be configurable
+    * New connect-string parameter "lex", with allowable values
+      "ORACLE", "MYSQL", "SQL_SERVER", "JAVA" sets policy to be like those
+      databases, in terms of quote string, whether quoted and unquoted
+      identifiers are converted to upper/lower case, and whether
+      identifiers are matched case-sensitively. "JAVA" is case-sensitive,
+      even for unquoted identifiers. It should be possible
+      for each connection to have its own settings for these. Objects
+      shared between sessions (views, materialized views) might
+      require more work.
+    * Added various internals to make it easy for developers to do the
+      right thing. When you need to look up a schema, table or
+      column/field name, you should use a catalog reader, and it will
+      apply the right case-sensitivity policy.
+    * Enable optiq consumer to utilize different lexical settings in
+      Frameworks/Planner. (Jacques Nadeau)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-135">CALCITE-115</a>]
+  Add a PARSE_TREE hook point with SqlNode parameter
+* Change planner rules to use `ProjectFactory` for creating
+  projects. (John Pullokkaran)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-131">CALCITE-131</a>]
+  Add interfaces for metadata (statistics)
+  (**This is a breaking change**.)
+* Update Avatica to allow `Cursor` & `Accessor` implementations to throw
+  `SQLException`. (Jacques Nadeau)
+* Separate cost model (`RelOptCostFactory`) from planner. Allow
+  `VolcanoPlanner` to be sub-classed with different cost factory.
+    * Remove references to VolcanoCost from RelSubset, so clients can
+      use a different `RelOptCost`. (Harish Butani)
+    * Make `VolcanoCost` immutable.
+* Break `SqlTypeStrategies` into `OperandTypes`, `ReturnTypes` and
+  `InferTypes`, and rename its static members to upper-case, per
+  checkstyle. (**This is a breaking change**.)
+* Add a mechanism for defining configuration parameters and have them
+  appear in the responses to `AvaticaDatabaseMetaData` methods.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-113">CALCITE-113</a>]
+  User-defined scalar functions
+* Add rules to short-cut a query if `LIMIT 0` is present. Also remove
+  sort, aggregation, join if their inputs are known to be empty, and
+  propagate the fact that the relational expressions are known to be
+  empty up the tree. (We already do this for union, filter, project.)
+* `RexNode` and its sub-classes are now immutable.
+
+Bug fixes and internal changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-61">CALCITE-16</a>]
+  Upgrade to janino-2.7
+* Upgrade to guava-15.0 (guava-14.0.1 still allowed), sqlline-1.1.7,
+  maven-surefire-plugin-2.16, linq4j-0.1.13.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-136">CALCITE-136</a>]
+  Support Hive dialect
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-138">CALCITE-138</a>]
+  SqlDataTypeSpec.clone handles collection types wrong
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-137">CALCITE-137</a>]
+  If a subset is created that is subsumed by an existing subset, its
+  'best' is not assigned
+    * If best rel in a Volcano subset doesn't have metadata, see if
+      other rels have metadata.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-127">CALCITE-127</a>]
+  EnumerableCalcRel can't support 3+ AND conditions (Harish Butani)
+* Fix push-down of datetime literals to JDBC data sources.
+* Add `Util.startsWith(List, List)` and `Util.hashCode(double)`.
+* Add maven-checkstyle-plugin, enable in "verify" phase, and fix exceptions.
+* Fix `SqlValidator` to rely on `RelDataType` to do field name matching.  Fix
+  `RelDataTypeImpl` to correctly use the case sensitive flag rather than
+  ignoring it.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-119">CALCITE-119</a>]
+  Comparing Java type long with SQL type INTEGER gives wrong answer
+* Enable multi-threaded testing, and fix race conditions.
+    * Two of the race conditions involved involving trait caches. The
+      other was indeterminacy in type system when precision was not
+      specified but had a default; now we canonize TIME to TIME(0), for
+      instance.
+* Convert files to `us-ascii`.
+* Work around
+  [<a href="http://jira.codehaus.org/browse/JANINO-169">JANINO-169</a>].
+* Refactor SQL validator testing infrastructure so SQL parser is
+  configurable.
+* Add `optiq-mat-plugin` to README.
+* Fix the check for duplicate subsets in a rule match.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-112">CALCITE-112</a>]
+  Java boolean column should be treated as SQL boolean
+* Fix escaped unicode characters above 0x8000. Add tests for unicode
+  strings.
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.17">0.4.17</a> / 2014-01-13
+
+API changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-106">CALCITE-106</a>]
+  Make `Schema` and `Table` SPIs simpler to implement, and make them
+  re-usable across connections
+  (**This is a breaking change**.)
+* Make it easier to define sub-classes of rule operands. The new class
+  `RelOptRuleOperandChildren` contains the children of an operand and
+  the policy for dealing with them. Existing rules now use the new
+  methods to construct operands: `operand()`, `leaf()`, `any()`, `none()`,
+  `unordered()`. The previous methods are now deprecated and will be
+  removed before 0.4.18. (**This is a breaking change**.)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-101">CALCITE-101</a>]
+  Enable phased access to the Optiq engine
+* List-handling methods in `Util`: add methods `skipLast`, `last`, `skip`;
+  remove `subList`, `butLast`.
+* Convert `SqlIdentifier.names` from `String[]` to `ImmutableList<String>`.
+* Rename `OptiqAssert.assertThat()` to `that()`, to avoid clash with junit's
+  `Assert.assertThat()`.
+* Usability improvements for `RelDataTypeFactory.FieldInfoBuilder`. It
+  now has a type-factory, so you can just call `build()`.
+* Rework `HepProgramBuilder` into a fluent API.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-105">CALCITE-105</a>]
+  Externalize RelNode to and from JSON
+
+Tuning
+* If `EnumerableAggregateRel` has no aggregate functions, generate a
+   call to `Enumerable.distinct()`, thereby saving the effort of
+   building trivial accumulators.
+* Default rule set now does not introduce `CalcRel` until a later phase
+  of planning. This reduces the number of trivial projects and calcs
+  created, merged, and elimated.
+* Reduce the amount of time spent creating record types that
+  already exist.
+* More efficient implementation of `Util.isDistinct` for small lists.
+* When an internal record has 0 fields, rather than generating a
+  synthetic class and lots of instances that are all the same, use the
+  new `Unit` class, which is a singleton.
+* To take advantage of asymmetric hash join added recently in linq4j,
+  tweak cost of `EnumerableJoinRel` so that join is cheaper if the
+  larger input is on the left, and more expensive if it is a cartesian
+  product.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-70">CALCITE-70</a>]
+  Joins seem to be very expensive in memory
+* Make planning process more efficient by not sorting the list of
+  matched rules each cycle. It is sorted if tracing is enabled;
+  otherwise we scan to find the most important element. For this list,
+  replace `LinkedList` with `ChunkList`, which has an O(1) remove and add,
+  a fast O(n) get, and fast scan.
+
+Other
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-87">CALCITE-87</a>]
+  Constant folding
+  * Rules for constant-expression reduction, and to simplify/eliminate
+    `VALUES` operator.
+* Graph algorithms: Implement breadth-first iterator and cycle-detector.
+* Fix bug in planner which occurred when two `RelNode`s have identical
+  digest but different row-type.
+* Fix link to optiq-csv tutorial.
+* Fix bugs in `RemoveTrivialProjectRule.strip`, `JdbcProjectRel.implement`
+  and `SortRel.computeSelfCost`.
+* Reformat code, and remove `@author` tags.
+* Upgrade to eigenbase-xom-1.3.4, eigenbase-properties-1.1.4,
+  eigenbase-resgen-1.3.6.
+* Upgrade to linq4j-0.1.12.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-97">CALCITE-97</a>]
+  Correlated EXISTS
+* Fix a bug in `VolcanoCost`.
+* Add class `FoodMartQuerySet`, that contains the 6,700 foodmart queries.
+* Fix factory class names in `UnregisteredDriver`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-96">CALCITE-96</a>]
+  LIMIT against a table in a clone schema causes UnsupportedOperationException
+* Disable spark module by default.
+* Allow `CloneSchema` to be specified in terms of url, driver, user,
+  password; not just dataSource.
+* Wrap internal error in `SQLException`.
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.16">0.4.16</a> / 2013-11-24
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-69">CALCITE-69</a>]
+  Can't join on string columns and other problems with expressions in the join
+  condition
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-74">CALCITE-74</a>]
+  JOIN ... USING fails in 3-way join with UnsupportedOperationException.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-65">CALCITE-65</a>]
+  Fix issues in the JDBC driver, and in particular to DatabaseMetaData methods,
+  to make Squirrel-SQL run better.
+* Fix JDBC column, table, schema names for when the table is not in a schema of
+  depth 1.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-85">CALCITE-85</a>]
+  Adding a table to the root schema causes breakage in OptiqPrepareImpl
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-84">CALCITE-84</a>]
+  Extract Optiq's JDBC driver as a new JDBC driver framework, Avatica.
+  Other projects can use this to implement a JDBC driver by implementing
+  just a few methods. If you wish to use Optiq's JDBC driver, you will
+  now need to include optiq-avatica.jar in addition to optiq-core.jar.
+  Avatica does not depend on anything besides the standard Java library.
+* Support for parameters in PreparedStatement.
+* First steps in recognizing complex materializations. Internally we introduce a
+  concept called a "star table", virtual table composed of real tables joined
+  together via many-to-one relationships. The queries that define
+  materializations and end-user queries are canonized in terms of star tables.
+  Matching (not done yet) will then be a matter of looking for sort, groupBy,
+  project. It is not yet possible to define a star in an Optiq model file.
+* Add section to <a href="howto.md">HOWTO</a> on implementing adapters.
+* Fix data type conversions when creating a clone table in memory.
+* Fix how strings are escaped in JsonBuilder.
+* Test suite now depends on an embedded hsqldb database, so you can run
+  <code>mvn test</code> right after pulling from git. You can instead use a
+  MySQL database if you specify '-Dcalcite.test.db=mysql', but you need to
+  manually populate it.
+* Fix a planner issue which occurs when the left and right children of join are
+  the same relational expression, caused by a self-join query.
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-76">CALCITE-76</a>]
+  Precedence of the item operator, <code>map[index]</code>; remove the space
+  before '[' when converting parse tree to string.
+* Allow <code>CAST(expression AS ANY)</code>, and fix an issue with the ANY type
+  and NULL values.
+* Handle null timestamps and dates coming out of JDBC adapter.
+* Add <code>jdbcDriver</code> attribute to JDBC schema in model, for drivers
+  that do not auto-register.
+* Allow join rules to match any subclass of JoinRelBase.
+* Push projects, filters and sorts down to MongoDB. (Fixes
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-57">CALCITE-57</a>],
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-60">CALCITE-60</a>] and
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-72">CALCITE-72</a>].)
+* Add instructions for loading FoodMart data set into MongoDB, and how to enable
+  tracing.
+* Now runs on JDK 1.8 (still runs on JDK 1.6 and JDK 1.7).
+* Upgrade to junit-4.11 (avoiding the dodgy junit-4.1.12).
+* Upgrade to linq4j-0.1.11.
+
+## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.15">0.4.15</a> / 2013-10-14
+
+* Lots of good stuff that this margin is too small to contain. See
+  <a href="reference.md">SQL language reference</a> and
+  <a href="model.md">JSON model reference</a>.
+
+# Optiq-csv release history
+
+Optiq-csv-0.3 was the last independent release of optiq-csv. From
+calcite-0.9.2 onwards, the code was included as the
+calcite-example-csv module.
+
+* Upgrade to calcite-0.9.1
+* Support gzip-compressed CSV and JSON files (recognized by '.gz' suffix)
+* Cleanup, and fix minor timezone issue in a test
+* Support for date types (date, time, timestamp) (Martijn van den Broek)
+* Upgrade to optiq-0.8, optiq-avatica-0.8, linq4j-0.4
+* Add support for JSON files (recognized by '.json' suffix)
+* Upgrade maven-release-plugin to version 2.4.2
+* Upgrade to optiq-0.6, linq4j-0.2
+* Add NOTICE and LICENSE files in generated JAR file
+
+## <a href="https://github.com/julianhyde/optiq-csv/releases/tag/optiq-csv-0.3">0.3</a> / 2014-03-21
+
+* Upgrade to optiq-0.5
+* Add workaround to
+  [<a href="https://github.com/jline/jline2/issues/62">JLINE2-62</a>]
+  to `sqlline.bat` (windows) and `sqlline` (windows using cygwin)
+* Fix classpath construction: `sqlline.bat` copies dependencies to
+  `target/dependencies`; `sqlline` constructs `target/classpath.txt`
+* Build, checkstyle and tests now succeed on windows (both native and cygwin)
+* Models can now contain comments
+* [<a href="https://github.com/julianhyde/optiq-csv/issues/2">OPTIQ-CSV-2</a>]
+  Update tutorial to reflect changes to Optiq's JDBC adapter
+
+## <a href="https://github.com/julianhyde/optiq-csv/releases/tag/optiq-csv-0.2">0.2</a> / 2014-02-18
+
+* Add test case for
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-112">CALCITE-112</a>]
+* Add `sqlline.bat`, Windows SQL shell (based on fix for
+  [<a href="https://issues.apache.org/jira/browse/DRILL-338">DRILL-338</a>])
+* Upgrade to optiq-0.4.18, sqlline-1.1.7
+* Return a single object for single-col enumerator (Gabriel Reid)
+* Enable maven-checkstyle-plugin; fix checkstyle exceptions
+
+## <a href="https://github.com/julianhyde/optiq-csv/releases/tag/optiq-csv-0.1">0.1</a> / 2014-01-13
+
+* Add release notes and history
+* Enable maven-release-plugin
+* Upgrade to optiq-0.4.17, linq4j-0.1.12, sqlline-1.1.6
+* Upgrade tutorial for new Schema and Table SPIs
+* Fixes for optiq SPI changes in
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-106">CALCITE-106</a>]
+* Enable oraclejdk8 in Travis CI
+* Fix bug where non-existent directory would give NPE; instead print warning
+* Add an example of a planner rule
+* Add `CsvTableFactory`, an example of a custom table
+* Add a view to tutorial
+* Split into scenario with a "simple" schema that generates tables
+  (`CsvTable`) that just execute and a "smart" schema that generates
+  tables (`CsvSmartTable`) that undergo optimization
+* Make `CsvEnumerator` a top-level class
+* Implement the algorithms to sniff names and types from the first
+  row, and to return an enumerator of all rows
+* Read column types from header of CSV file
+
+# Linq4j release history
+
+Linq4j-0.4 was the last independent release of linq4j. From
+calcite-0.9.2 onwards, the code was included as calcite-linq4j, and
+features added to linq4j in a particular calcite release are described
+with the other changes in that release.
+
+## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.4">0.4</a> / 2014-05-28
+
+* Fix <a href="https://github.com/julianhyde/linq4j/issues/27">#27</a>,
+  "Incorrectly inlines non-final variable".
+* Maven build process now deploys web site.
+* Implement `Enumerable` methods: `any`, `all`,
+  `contains` with `EqualityComparer`, `first`, `first` with predicate.
+
+## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.3">0.3</a> / 2014-04-21
+
+* Move optimizer visitor from optiq to linq4j; add
+  `ExpressionType.modifiesLvalue` to avoid invalid inlining.
+* Fix <a href="https://github.com/julianhyde/linq4j/issues/17">#17</a>,
+  "Assign constant expressions to 'static final' members";
+  add `@Deterministic` annotation to help deduce which expressions are
+  constant.
+* Multi-pass optimization: some of the variables might be avoided and
+  inlined after the first pass.
+* Various other peephole optimizations: `Boolean.valueOf(const)`,
+  'not' expressions (`!const`, `!!a`, `!(a==b)`, `!(a!=b)`, `!(a>b)`,
+  etc.),
+  '?' expressions coming from `CASE` (`a ? booleanConstant : b` and `a
+  ? b : booleanConstant`).
+* Implement left, right and full outer join.
+* Clean build on cygwin/Windows.
+
+## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.2">0.2</a> / 2014-04-11
+
+* Fix <a href="https://github.com/julianhyde/linq4j/issues/8">#8</a>,
+  "Javadoc generation fails under JDK 1.8".
+* Fix <a href="https://github.com/julianhyde/linq4j/issues/15">#15</a>,
+  "`Expressions.ifThenElse` does not work".
+* Use `HashMap` for searching of declarations to reuse; consider both
+  `optimizing` and `optimize` flags when reusing.
+* Implement `equals` and `hashCode` for expressions. Hash codes for
+  complex expressions are cached into a field of the expression.
+* Add example, `com.example.Linq4jExample`.
+* Fix optimizing away parameter declarations in assignment target.
+* Support Windows path names in checkstyle-suppresions.
+* Support `Statement.toString` via `ExpressionWriter`.
+* Use `AtomicInteger` for naming of `ParameterExpression`s to avoid
+  conflicts in multithreaded usage
+* Cleanup: use `Functions.adapt` rather than `new AbstractList`
+* Add `NOTICE` and `LICENSE` files in generated JAR file.
+* Optimize `select()` if selector is identity.
+* Enable checkstyle.
+
+## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.1.13">0.1.13</a> / 2014-01-20
+
+* Remove spurious "null" generated when converting expression to string.
+* Allow a field declaration to not have an initializer.
+* Add `Primitive.defaultValue`.
+* Enable `oraclejdk8` in <a href="https://travis-ci.org/julianhyde/linq4j">Travis CI</a>.
+
+## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.1.12">0.1.12</a> / 2013-12-07
+
+* Add release notes.
+* Fix implementation of `Enumerable.asEnumerable` in
+  `DefaultQueryable` (inherited by most classes that implement
+  `Queryable`).
+
+## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.1.11">0.1.11</a> / 2013-11-06
+
+* Initial commit
+


[04/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/stream.md
----------------------------------------------------------------------
diff --git a/doc/stream.md b/doc/stream.md
new file mode 100644
index 0000000..4052ac1
--- /dev/null
+++ b/doc/stream.md
@@ -0,0 +1,631 @@
+<!--
+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.
+-->
+# Calcite SQL extensions for streaming
+
+## Introduction
+
+Streams are collections to records that flow continuously, and forever.
+Unlike tables, they are not typically stored on disk, but flow over the
+network and are held for short periods of time in memory.
+
+Streams complement tables because they represent what is happening in the
+present and future of the enterprise whereas tables represent the past.
+It is very common for a stream to be archived into a table.
+
+Like tables, you often want to query streams in a high-level language
+based on relational algebra, validated according to a schema, and optimized
+to take advantage of available resources and algorithms.
+
+Calcite's SQL is an extension to standard SQL, not another 'SQL-like' language.
+The distinction is important, for several reasons:
+* Streaming SQL is easy to learn for anyone who knows regular SQL.
+* The semantics are clear, because we aim to produce the same results on a
+  stream as if the same data were in a table.
+* You can write queries that combine streams and tables (or the history of
+  a stream, which is basically an in-memory table).
+* Lots of existing tools can generate standard SQL.
+
+## An example schema
+
+Our streaming SQL examples use the following schema:
+* `Orders (rowtime, productId, orderId, units)` - a stream and a table
+* `Products (rowtime, productId, name)` - a table
+* `Shipments (rowtime, orderId)` - a stream
+
+## A simple query
+
+Let's start with the simplest streaming query:
+
+```sql
+SELECT STREAM *
+FROM Orders;
+
+  rowtime | productId | orderId | units
+----------+-----------+---------+-------
+ 10:17:00 |        30 |       5 |     4
+ 10:17:05 |        10 |       6 |     1
+ 10:18:05 |        20 |       7 |     2
+ 10:18:07 |        30 |       8 |    20
+ 11:02:00 |        10 |       9 |     6
+ 11:04:00 |        10 |      10 |     1
+ 11:09:30 |        40 |      11 |    12
+ 11:24:11 |        10 |      12 |     4
+```
+
+This query reads all columns and rows from the `Orders` stream.
+Like any streaming query, it never terminates. It outputs a record whenever
+a record arrives in `Orders`.
+
+Type `Control-C` to terminate the query.
+
+The `STREAM` keyword is the main extension in streaming SQL. It tells the
+system that you are interested in incoming orders, not existing ones. The query
+
+```sql
+SELECT *
+FROM Orders;
+
+  rowtime | productId | orderId | units
+----------+-----------+---------+-------
+ 08:30:00 |        10 |       1 |     3
+ 08:45:10 |        20 |       2 |     1
+ 09:12:21 |        10 |       3 |    10
+ 09:27:44 |        30 |       4 |     2
+
+4 records returned.
+```
+
+is also valid, but will print out all existing orders and then terminate. We
+call it a *relational* query, as opposed to *streaming*. It has traditional
+SQL semantics.
+
+`Orders` is special, in that it has both a stream and a table. If you try to run
+a streaming query on a table, or a relational query on a stream, Calcite gives
+an error:
+
+```sql
+> SELECT * FROM Shipments;
+ERROR: Cannot convert stream 'SHIPMENTS' to a table
+
+> SELECT STREAM * FROM Products;
+ERROR: Cannot convert table 'PRODUCTS' to a stream
+```
+
+# Filtering rows
+
+Just as in regular SQL, you use a `WHERE` clause to filter rows:
+
+```sql
+SELECT STREAM *
+FROM Orders
+WHERE units > 3;
+
+  rowtime | productId | orderId | units
+----------+-----------+---------+-------
+ 10:17:00 |        30 |       5 |     4
+ 10:18:07 |        30 |       8 |    20
+ 11:02:00 |        10 |       9 |     6
+ 11:09:30 |        40 |      11 |    12
+ 11:24:11 |        10 |      12 |     4
+```
+
+# Projecting expressions
+
+Use expressions in the `SELECT` clause to choose which columns to return or
+compute expressions:
+
+```sql
+SELECT STREAM rowtime,
+  'An order for ' || units || ' '
+    || CASE units WHEN 1 THEN 'unit' ELSE 'units' END
+    || ' of product #' || productId AS description
+FROM Orders;
+
+  rowtime | description
+----------+---------------------------------------
+ 10:17:00 | An order for 4 units of product #30
+ 10:17:05 | An order for 1 unit of product #10
+ 10:18:05 | An order for 2 units of product #20
+ 10:18:07 | An order for 20 units of product #30
+ 11:02:00 | An order by 6 units of product #10
+ 11:04:00 | An order by 1 unit of product #10
+ 11:09:30 | An order for 12 units of product #40
+ 11:24:11 | An order by 4 units of product #10
+```
+
+We recommend that you always include the `rowtime` column in the `SELECT`
+clause. Having a sorted timestamp in each stream and streaming query makes it
+possible to do advanced calculations later, such as `GROUP BY` and `JOIN`.
+
+# Tumbling windows
+
+There are several ways to compute aggregate functions on streams. The
+differences are:
+* How many rows come out for each row in?
+* Does each incoming value appear in one total, or more?
+* What defines the "window", the set of rows that contribute to a given output row?
+* Is the result a stream or a relation?
+
+First we'll look a *tumbling window*, which is defined by a streaming
+`GROUP BY`. Here is an example:
+
+```sql
+SELECT STREAM FLOOR(rowtime TO HOUR) AS rowtime,
+  productId,
+  COUNT(*) AS c,
+  SUM(units) AS units
+FROM Orders
+GROUP BY FLOOR(rowtime TO HOUR), productId;
+
+  rowtime | productId |       c | units
+----------+-----------+---------+-------
+ 10:00:00 |        30 |       2 |    24
+ 10:00:00 |        10 |       1 |     1
+ 10:00:00 |        20 |       1 |     7
+ 11:00:00 |        10 |       3 |    11
+ 11:00:00 |        40 |       1 |    12
+```
+
+The result is a stream. At 11 o'clock, Calcite emits a sub-total for every
+`productId` that had an order since 10 o'clock. At 12 o'clock, it will emit
+the orders that occurred between 11:00 and 12:00. Each input row contributes to
+only one output row.
+
+How did Calcite know that the 10:00:00 sub-totals were complete at 11:00:00,
+so that it could emit them? It knows that `rowtime` is increasing, and it knows
+that `FLOOR(rowtime TO HOUR)` is also increasing. So, once it has seen a row
+at or after 11:00:00, it will never see a row that will contribute to a 10:00:00
+total.
+
+A column or expression that is increasing or decreasing is said to be
+*monotonic*. Without a monotonic expression in the `GROUP BY` clause, Calcite is
+not able to make progress, and it will not allow the query:
+
+```sql
+> SELECT STREAM productId,
+>   COUNT(*) AS c,
+>   SUM(units) AS units
+> FROM Orders
+> GROUP BY productId;
+ERROR: Streaming aggregation requires at least one monotonic expression in GROUP BY clause
+```
+
+Monotonic columns need to be declared in the schema. The monotonicity is
+enforced when records enter the stream and assumed by queries that read from
+that stream. We recommend that you give each stream a timestamp column called
+`rowtime`, but you can declare others, `orderId`, for example.
+
+# Filtering after aggregation
+
+As in standard SQL, you can apply a `HAVING` clause to filter rows emitted by
+a streaming `GROUP BY`:
+
+```sql
+SELECT STREAM FLOOR(rowtime TO HOUR) AS rowtime,
+  productId
+FROM Orders
+GROUP BY FLOOR(rowtime TO HOUR), productId
+HAVING COUNT(*) > 2 OR SUM(units) > 10;
+
+  rowtime | productId
+----------+-----------
+ 10:00:00 |        30
+ 11:00:00 |        10
+ 11:00:00 |        40
+```
+
+# Sub-queries, views and SQL's closure property
+
+The previous `HAVING` query can be expressed using a `WHERE` clause on a
+sub-query:
+
+```sql
+SELECT STREAM rowtime, productId
+FROM (
+  SELECT FLOOR(rowtime TO HOUR) AS rowtime,
+    productId,
+    COUNT(*) AS c,
+    SUM(units) AS su
+  FROM Orders
+  GROUP BY FLOOR(rowtime TO HOUR), productId)
+WHERE c > 2 OR su > 10;
+
+  rowtime | productId
+----------+-----------
+ 10:00:00 |        30
+ 11:00:00 |        10
+ 11:00:00 |        40
+```
+
+`HAVING` was introduced in the early days of SQL, when a way was needed to
+perform a filter *after* aggregation. (Recall that `WHERE` filters rows before
+they enter the `GROUP BY` clause.)
+
+Since then, SQL has become a mathematically closed language, which means that
+any operation you can perform on a table can also perform on a query.
+
+The *closure property* of SQL is extremely powerful. Not only does it render
+`HAVING` obsolete (or, at least, reduce it to syntactic sugar), it makes views
+possible:
+
+```sql
+CREATE VIEW HourlyOrderTotals (rowtime, productId, c, su) AS
+  SELECT FLOOR(rowtime TO HOUR),
+    productId,
+    COUNT(*),
+    SUM(units)
+  FROM Orders
+  GROUP BY FLOOR(rowtime TO HOUR), productId;
+
+SELECT STREAM rowtime, productId
+FROM HourlyOrderTotals
+WHERE c > 2 OR su > 10;
+
+  rowtime | productId
+----------+-----------
+ 10:00:00 |        30
+ 11:00:00 |        10
+ 11:00:00 |        40
+```
+
+Sub-queries in the `FROM` clause are sometimes referred to as "inline views",
+but really, nested queries are more fundamental. Views are just a convenient
+way to carve your SQL into manageable chunks.
+
+Many people find that nested queries and views are even more useful on streams
+than they are on relations. Streaming queries are pipelines of
+operators all running continuously, and often those pipelines get quite long.
+Nested queries and views help to express and manage those pipelines.
+
+And, by the way, a `WITH` clause can accomplish the same as a sub-query or
+a view:
+
+```sql
+WITH HourlyOrderTotals (rowtime, productId, c, su) AS (
+  SELECT FLOOR(rowtime TO HOUR),
+    productId,
+    COUNT(*),
+    SUM(units)
+  FROM Orders
+  GROUP BY FLOOR(rowtime TO HOUR), productId)
+SELECT STREAM rowtime, productId
+FROM HourlyOrderTotals
+WHERE c > 2 OR su > 10;
+
+  rowtime | productId
+----------+-----------
+ 10:00:00 |        30
+ 11:00:00 |        10
+ 11:00:00 |        40
+```
+
+## Converting between streams and relations
+
+Look back at the definition of the `HourlyOrderTotals` view.
+Is the view a stream or a relation?
+
+It does not contain the `STREAM` keyword, so it is a relation.
+However, it is a relation that can be converted into a stream.
+
+You can use it in both relational and streaming queries:
+
+```sql
+# A relation; will query the historic Orders table.
+# Returns the largest number of product #10 ever sold in one hour.
+SELECT max(su)
+FROM HourlyOrderTotals
+WHERE productId = 10;
+
+# A stream; will query the Orders stream.
+# Returns every hour in which at least one product #10 was sold.
+SELECT STREAM rowtime
+FROM HourlyOrderTotals
+WHERE productId = 10;
+```
+
+This approach is not limited to views and sub-queries.
+Following the approach set out in CQL [<a href="#ref1">1</a>], every query
+in streaming SQL is defined as a relational query and converted to a stream
+using the `STREAM` keyword in the top-most `SELECT`.
+
+If the `STREAM` keyword is present in sub-queries or view definitions, it has no
+effect.
+
+At query preparation time, Calcite figures out whether the relations referenced
+in the query can be converted to streams or historical relations.
+
+Sometimes a stream makes available some of its history (say the last 24 hours of
+data in an Apache Kafka [<a href="#ref2">2</a>] topic)
+but not all. At run time, Calcite figures out whether there is sufficient
+history to run the query, and if not, gives an error.
+
+## Hopping windows
+
+Previously we saw how to define a tumbling window using a `GROUP BY` clause.
+Each record contributed to a single sub-total record, the one containing its
+hour and product id.
+
+But suppose we want to emit, every hour, the number of each product ordered over
+the past three hours. To do this, we use `SELECT ... OVER` and a sliding window
+to combine multiple tumbling windows.
+
+```sql
+SELECT STREAM rowtime,
+  productId,
+  SUM(su) OVER w AS su,
+  SUM(c) OVER w AS c
+FROM HourlyTotals
+WINDOW w AS (
+  ORDER BY rowtime
+  PARTITION BY productId
+  RANGE INTERVAL '2' HOUR PRECEDING)
+```
+
+This query uses the `HourlyOrderTotals` view defined previously.
+The 2 hour interval combines the totals timestamped 09:00:00, 10:00:00 and
+11:00:00 for a particular product into a single total timestamped 11:00:00 and
+summarizing orders for that product between 09:00:00 and 12:00:00.
+
+## Limitations of tumbling and hopping windows
+
+In the present syntax, we acknowledge that it is not easy to create certain
+kinds of windows.
+
+First, let's consider tumbling windows over complex periods.
+
+The `FLOOR` and `CEIL` functions make is easy to create a tumbling window that
+emits on a whole time unit (say every hour, or every minute) but less easy to
+emit, say, every 15 minutes. One could imagine an extension to the `FLOOR`
+function that emits unique values on just about any periodic basis (say in 11
+minute intervals starting from midnight of the current day).
+
+Next, let's consider hopping windows whose retention period is not a multiple
+of its emission period. Say we want to output, at the top of each hour, the
+orders for the previous 7,007 seconds. If we were to simulate this hopping
+window using a sliding window over a tumbling window, as before, we would have
+to sum lots of 1-second windows (because 3,600 and 7,007 are co-prime).
+This is a lot of effort for both the system and the person writing the query.
+
+Calcite could perhaps solve this generalizing `GROUP BY` syntax, but we would
+be destroying the principle that an input row into a `GROUP BY` appears in
+precisely one output row.
+
+Calcite's SQL extensions for streaming queries are evolving. As we learn more
+about how people wish to query streams, we plan to make the language more
+expressive while remaining compatible with standard SQL and consistent with
+its principles, look and feel.
+
+## Sorting
+
+The story for `ORDER BY` is similar to `GROUP BY`.
+The syntax looks like regular SQL, but Calcite must be sure that it can deliver
+timely results. It therefore requires a monotonic expression on the leading edge
+of your `ORDER BY` key.
+
+```sql
+SELECT STREAM FLOOR(rowtime TO hour) AS rowtime, productId, orderId, units
+FROM Orders
+ORDER BY FLOOR(rowtime TO hour) ASC, units DESC;
+
+  rowtime | productId | orderId | units
+----------+-----------+---------+-------
+ 10:00:00 |        30 |       8 |    20
+ 10:00:00 |        30 |       5 |     4
+ 10:00:00 |        20 |       7 |     2
+ 10:00:00 |        10 |       6 |     1
+ 11:00:00 |        40 |      11 |    12
+ 11:00:00 |        10 |       9 |     6
+ 11:00:00 |        10 |      12 |     4
+ 11:00:00 |        10 |      10 |     1
+```
+
+Most queries will return results in the order that they were inserted,
+because the engine is using streaming algorithms, but you should not rely on it.
+For example, consider this:
+
+```sql
+SELECT STREAM *
+FROM Orders
+WHERE productId = 10
+UNION ALL
+SELECT STREAM *
+FROM Orders
+WHERE productId = 30;
+
+  rowtime | productId | orderId | units
+----------+-----------+---------+-------
+ 10:17:05 |        10 |       6 |     1
+ 10:17:00 |        30 |       5 |     4
+ 10:18:07 |        30 |       8 |    20
+ 11:02:00 |        10 |       9 |     6
+ 11:04:00 |        10 |      10 |     1
+ 11:24:11 |        10 |      12 |     4
+```
+
+The rows with `productId` = 30 are apparently out of order, probably because
+the `Orders` stream was partitioned on `productId` and the partitioned streams
+sent their data at different times.
+
+If you require a particular ordering, add an explicit `ORDER BY`:
+
+```sql
+SELECT STREAM *
+FROM Orders
+WHERE productId = 10
+UNION ALL
+SELECT STREAM *
+FROM Orders
+WHERE productId = 30
+ORDER BY rowtime;
+
+  rowtime | productId | orderId | units
+----------+-----------+---------+-------
+ 10:17:00 |        30 |       5 |     4
+ 10:17:05 |        10 |       6 |     1
+ 10:18:07 |        30 |       8 |    20
+ 11:02:00 |        10 |       9 |     6
+ 11:04:00 |        10 |      10 |     1
+ 11:24:11 |        10 |      12 |     4
+```
+
+Calcite will probably implement the `UNION ALL` by merging using `rowtime`,
+which is only slightly less efficient.
+
+You only need to add an `ORDER BY` to the outermost query. If you need to,
+say, perform `GROUP BY` after a `UNION ALL`, Calcite will add an `ORDER BY`
+implicitly, in order to make the GROUP BY algorithm possible.
+
+## Table constructor
+
+The `VALUES` clause creates an inline table with a given set of rows.
+
+Streaming is disallowed. The set of rows never changes, and therefore a stream
+would never return any rows.
+
+```sql
+> SELECT STREAM * FROM (VALUES (1, 'abc'));
+
+ERROR: Cannot stream VALUES
+```
+
+## Sliding windows
+
+Standard SQL features so-called "analytic functions" that can be used in the
+`SELECT` clause. Unlike `GROUP BY`, these do not collapse records. For each
+record that goes in, one record comes out. But the aggregate function is based
+on a window of many rows.
+
+Let's look at an example.
+
+```sql
+SELECT STREAM rowtime,
+  productId,
+  units,
+  SUM(units) OVER (ORDER BY rowtime RANGE INTERVAL '1' HOUR PRECEDING) unitsLastHour
+FROM Orders;
+```
+
+The feature packs a lot of power with little effort. You can have multiple
+functions in the `SELECT` clause, based on multiple window specifications.
+
+The following example returns orders whose average order size over the last
+10 minutes is greater than the average order size for the last week.
+
+```sql
+SELECT STREAM *
+FROM (
+  SELECT STREAM rowtime,
+    productId,
+    units,
+    AVG(units) OVER product (RANGE INTERVAL '10' MINUTE PRECEDING) AS m10,
+    AVG(units) OVER product (RANGE INTERVAL '7' DAY PRECEDING) AS d7
+  FROM Orders
+  WINDOW product AS (
+    ORDER BY rowtime
+    PARTITION BY productId))
+WHERE m10 > d7;
+```
+
+For conciseness, here we use a syntax where you partially define a window
+using a `WINDOW` clause and then refine the window in each `OVER` clause.
+You could also define all windows in the `WINDOW` clause, or all windows inline,
+if you wish.
+
+But the real power goes beyond syntax. Behind the scenes, this query is
+maintaining two tables, and adding and removing values from sub-totals using
+with FIFO queues. But you can access those tables without introducing a join
+into the query.
+
+Some other features of the windowed aggregation syntax:
+* You can define windows based on row count.
+* The window can reference rows that have not yet arrived.
+  (The stream will wait until they have arrived).
+* You can compute order-dependent functions such as `RANK` and median.
+
+## Cascading windows
+
+What if we want a query that returns a result for every record, like a
+sliding window, but resets totals on a fixed time period, like a
+tumbling window? Such a pattern is called a *cascading window*. Here
+is an example:
+
+```sql
+SELECT STREAM rowtime,
+  productId,
+  units,
+  SUM(units) OVER (PARTITION BY FLOOR(rowtime TO HOUR)) AS unitsSinceTopOfHour
+FROM Orders;
+```
+
+It looks similar to a sliding window query, but the monotonic
+expression occurs within the `PARTITION BY` clause of the window. As
+the rowtime moves from from 10:59:59 to 11:00:00, `FLOOR(rowtime TO
+HOUR)` changes from 10:00:00 to 11:00:00, and therefore a new
+partition starts. The first row to arrive in the new hour will start a
+new total; the second row will have a total that consists of two rows,
+and so on.
+
+Calcite knows that the old partition will never be used again, so
+removes all sub-totals for that partition from its internal storage.
+
+Analytic functions that using cascading and sliding windows can be
+combined in the same query.
+
+## State of the stream
+
+Not all concepts in this article have been implemented in Calcite.
+And others may be implemented in Calcite but not in a particular adapter
+such as Samza SQL [<a href="#ref3">3</a>].
+
+### Implemented
+* Streaming SELECT, WHERE, GROUP BY, HAVING, UNION ALL, ORDER BY
+* FLOOR and CEILING functions
+* Monotonicity
+* Streaming VALUES is disallowed
+
+### Not implemented
+* Stream-to-stream JOIN
+* Stream-to-table JOIN
+* Stream on view
+* Streaming UNION ALL with ORDER BY (merge)
+* Relational query on stream
+* Streaming windowed aggregation (sliding and cascading windows)
+* Check that STREAM in sub-queries and views is ignored
+* Check that streaming ORDER BY cannot have OFFSET or LIMIT
+* Limited history; at run time, check that there is sufficient history
+  to run the query.
+
+### To do in this document
+* Re-visit whether you can stream VALUES
+* OVER clause to define window on stream
+* Windowed aggregation
+* Punctuation
+* Stream-to-table join
+** Stream-to-table join where table is changing
+* Stream-to-stream join
+* Relational queries on streams (e.g. "pie chart" query)
+* Diagrams for various window types
+
+## References
+
+* [<a name="ref1">1</a>]
+  <a href="http://ilpubs.stanford.edu:8090/758/">Arasu, Arvind and Babu,
+  Shivnath and Widom, Jennifer (2003) The CQL Continuous Query
+  Language: Semantic Foundations and Query Execution</a>.
+* [<a name="ref2">2</a>]
+  <a href="http://kafka.apache.org/documentation.html">Apache Kafka</a>.
+* [<a name="ref3">3</a>] <a href="http://samza.apache.org">Apache Samza</a>.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/tutorial.md
----------------------------------------------------------------------
diff --git a/doc/tutorial.md b/doc/tutorial.md
new file mode 100644
index 0000000..91ddef1
--- /dev/null
+++ b/doc/tutorial.md
@@ -0,0 +1,753 @@
+<!--
+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.
+-->
+# CSV Adapter Tutorial
+
+Calcite-example-CSV is a fully functional adapter for
+<a href="https://github.com/apache/incubator-calcite">Calcite</a> that reads
+text files in
+<a href="http://en.wikipedia.org/wiki/Comma-separated_values">CSV
+(comma-separated values)</a> format. It is remarkable that a couple of
+hundred lines of Java code are sufficient to provide full SQL query
+capability.
+
+CSV also serves as a template for building adapters to other
+data formats. Even though there are not many lines of code, it covers
+several important concepts:
+* user-defined schema using SchemaFactory and Schema interfaces;
+* declaring schemas in a model JSON file;
+* declaring views in a model JSON file;
+* user-defined table using the Table interface;
+* determining the record type of a table;
+* a simple implementation of Table, using the ScannableTable interface, that
+  enumerates all rows directly;
+* a more advanced implementation that implements FilterableTable, and can
+  filter out rows according to simple predicates;
+* advanced implementation of Table, using TranslatableTable, that translates
+  to relational operators using planner rules.
+
+## Download and build
+
+You need Java (1.7 or higher; 1.8 preferred), git and maven (3.2.1 or later).
+
+```bash
+$ git clone https://github.com/apache/incubator-calcite.git
+$ cd incubator-calcite
+$ mvn install -DskipTests -Dcheckstyle.skip=true
+$ cd example/csv
+```
+
+## First queries
+
+Now let's connect to Calcite using
+<a href="https://github.com/julianhyde/sqlline">sqlline</a>, a SQL shell
+that is included in this project.
+
+```bash
+$ ./sqlline
+sqlline> !connect jdbc:calcite:model=target/test-classes/model.json admin admin
+```
+
+(If you are running Windows, the command is `sqlline.bat`.)
+
+Execute a metadata query:
+
+```bash
+sqlline> !tables
++------------+--------------+-------------+---------------+----------+------+
+| TABLE_CAT  | TABLE_SCHEM  | TABLE_NAME  |  TABLE_TYPE   | REMARKS  | TYPE |
++------------+--------------+-------------+---------------+----------+------+
+| null       | SALES        | DEPTS       | TABLE         | null     | null |
+| null       | SALES        | EMPS        | TABLE         | null     | null |
+| null       | SALES        | HOBBIES     | TABLE         | null     | null |
+| null       | metadata     | COLUMNS     | SYSTEM_TABLE  | null     | null |
+| null       | metadata     | TABLES      | SYSTEM_TABLE  | null     | null |
++------------+--------------+-------------+---------------+----------+------+
+```
+
+(JDBC experts, note: sqlline's <code>!tables</code> command is just executing
+<a href="http://docs.oracle.com/javase/7/docs/api/java/sql/DatabaseMetaData.html#getTables(java.lang.String, java.lang.String, java.lang.String, java.lang.String[])"><code>DatabaseMetaData.getTables()</code></a>
+behind the scenes.
+It has other commands to query JDBC metadata, such as <code>!columns</code> and <code>!describe</code>.)
+
+As you can see there are 5 tables in the system: tables
+<code>EMPS</code>, <code>DEPTS</code> and <code>HOBBIES</code> in the current
+<code>SALES</code> schema, and <code>COLUMNS</code> and
+<code>TABLES</code> in the system <code>metadata</code> schema. The
+system tables are always present in Calcite, but the other tables are
+provided by the specific implementation of the schema; in this case,
+the <code>EMPS</code> and <code>DEPTS</code> tables are based on the
+<code>EMPS.csv</code> and <code>DEPTS.csv</code> files in the
+<code>target/test-classes</code> directory.
+
+Let's execute some queries on those tables, to show that Calcite is providing
+a full implementation of SQL. First, a table scan:
+
+```bash
+sqlline> SELECT * FROM emps;
++--------+--------+---------+---------+----------------+--------+-------+---+
+| EMPNO  |  NAME  | DEPTNO  | GENDER  |      CITY      | EMPID  |  AGE  | S |
++--------+--------+---------+---------+----------------+--------+-------+---+
+| 100    | Fred   | 10      |         |                | 30     | 25    | t |
+| 110    | Eric   | 20      | M       | San Francisco  | 3      | 80    | n |
+| 110    | John   | 40      | M       | Vancouver      | 2      | null  | f |
+| 120    | Wilma  | 20      | F       |                | 1      | 5     | n |
+| 130    | Alice  | 40      | F       | Vancouver      | 2      | null  | f |
++--------+--------+---------+---------+----------------+--------+-------+---+
+```
+
+Now JOIN and GROUP BY:
+
+```bash
+sqlline> SELECT d.name, COUNT(*)
+. . . .> FROM emps AS e JOIN depts AS d ON e.deptno = d.deptno
+. . . .> GROUP BY d.name;
++------------+---------+
+|    NAME    | EXPR$1  |
++------------+---------+
+| Sales      | 1       |
+| Marketing  | 2       |
++------------+---------+
+```
+
+Last, the VALUES operator generates a single row, and is a convenient
+way to test expressions and SQL built-in functions:
+
+```bash
+sqlline> VALUES CHAR_LENGTH('Hello, ' || 'world!');
++---------+
+| EXPR$0  |
++---------+
+| 13      |
++---------+
+```
+
+Calcite has many other SQL features. We don't have time to cover them
+here. Write some more queries to experiment.
+
+## Schema discovery
+
+Now, how did Calcite find these tables? Remember, core Calcite does not
+know anything about CSV files. (As a "database without a storage
+layer", Calcite doesn't know about any file formats.) Calcite knows about
+those tables because we told it to run code in the calcite-example-csv
+project.
+
+There are a couple of steps in that chain. First, we define a schema
+based on a schema factory class in a model file. Then the schema
+factory creates a schema, and the schema creates several tables, each
+of which knows how to get data by scanning a CSV file. Last, after
+Calcite has parsed the query and planned it to use those tables, Calcite
+invokes the tables to read the data as the query is being
+executed. Now let's look at those steps in more detail.
+
+On the JDBC connect string we gave the path of a model in JSON
+format. Here is the model:
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'SALES',
+  schemas: [
+    {
+      name: 'SALES',
+      type: 'custom',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
+      operand: {
+        directory: 'target/test-classes/sales'
+      }
+    }
+  ]
+}
+```
+
+The model defines a single schema called 'SALES'. The schema is
+powered by a plugin class,
+<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java">org.apache.calcite.adapter.csv.CsvSchemaFactory</a>,
+which is part of the
+calcite-example-csv project and implements the Calcite interface
+<a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/SchemaFactory.html">SchemaFactory</a>.
+Its <code>create</code> method instantiates a
+schema, passing in the <code>directory</code> argument from the model file:
+
+```java
+public Schema create(SchemaPlus parentSchema, String name,
+    Map<String, Object> operand) {
+  String directory = (String) operand.get("directory");
+  String flavorName = (String) operand.get("flavor");
+  CsvTable.Flavor flavor;
+  if (flavorName == null) {
+    flavor = CsvTable.Flavor.SCANNABLE;
+  } else {
+    flavor = CsvTable.Flavor.valueOf(flavorName.toUpperCase());
+  }
+  return new CsvSchema(
+      new File(directory),
+      flavor);
+}
+```
+
+Driven by the model, the schema factory instantiates a single schema
+called 'SALES'.  The schema is an instance of
+<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java">org.apache.calcite.adapter.csv.CsvSchema</a>
+and implements the Calcite interface <a
+href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/Schema.html">Schema</a>.
+
+A schema's job is to produce a list of tables. (It can also list sub-schemas and
+table-functions, but these are advanced features and calcite-example-csv does
+not support them.) The tables implement Calcite's
+<a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/Table.html">Table</a>
+interface. <code>CsvSchema</code> produces tables that are instances of
+<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java">CsvTable</a>
+and its sub-classes.
+
+Here is the relevant code from <code>CsvSchema</code>, overriding the
+<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/impl/AbstractSchema.html#getTableMap()">getTableMap()</a></code>
+method in the <code>AbstractSchema</code> base class.
+
+```java
+protected Map<String, Table> getTableMap() {
+  // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
+  // ".json.gz".
+  File[] files = directoryFile.listFiles(
+      new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+          final String nameSansGz = trim(name, ".gz");
+          return nameSansGz.endsWith(".csv")
+              || nameSansGz.endsWith(".json");
+        }
+      });
+  if (files == null) {
+    System.out.println("directory " + directoryFile + " not found");
+    files = new File[0];
+  }
+  // Build a map from table name to table; each file becomes a table.
+  final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+  for (File file : files) {
+    String tableName = trim(file.getName(), ".gz");
+    final String tableNameSansJson = trimOrNull(tableName, ".json");
+    if (tableNameSansJson != null) {
+      JsonTable table = new JsonTable(file);
+      builder.put(tableNameSansJson, table);
+      continue;
+    }
+    tableName = trim(tableName, ".csv");
+    final Table table = createTable(file);
+    builder.put(tableName, table);
+  }
+  return builder.build();
+}
+
+/** Creates different sub-type of table based on the "flavor" attribute. */
+private Table createTable(File file) {
+  switch (flavor) {
+  case TRANSLATABLE:
+    return new CsvTranslatableTable(file, null);
+  case SCANNABLE:
+    return new CsvScannableTable(file, null);
+  case FILTERABLE:
+    return new CsvFilterableTable(file, null);
+  default:
+    throw new AssertionError("Unknown flavor " + flavor);
+  }
+}
+```
+
+The schema scans the directory and finds all files whose name ends
+with ".csv" and creates tables for them. In this case, the directory
+is <code>target/test-classes/sales</code> and contains files
+<code>EMPS.csv</code> and <code>DEPTS.csv</code>, which these become
+the tables <code>EMPS</code> and <code>DEPTS</code>.
+
+## Tables and views in schemas
+
+Note how we did not need to define any tables in the model; the schema
+generated the tables automatically.
+
+You can define extra tables,
+beyond those that are created automatically,
+using the <code>tables</code> property of a schema.
+
+Let's see how to create
+an important and useful type of table, namely a view.
+
+A view looks like a table when you are writing a query, but it doesn't store data.
+It derives its result by executing a query.
+The view is expanded while the query is being planned, so the query planner
+can often perform optimizations like removing expressions from the SELECT
+clause that are not used in the final result.
+
+Here is a schema that defines a view:
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'SALES',
+  schemas: [
+    {
+      name: 'SALES',
+      type: 'custom',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
+      operand: {
+        directory: 'target/test-classes/sales'
+      },
+      tables: [
+        {
+          name: 'FEMALE_EMPS',
+          type: 'view',
+          sql: 'SELECT * FROM emps WHERE gender = \'F\''
+        }
+      ]
+    }
+  ]
+}
+```
+
+The line <code>type: 'view'</code> tags <code>FEMALE_EMPS</code> as a view,
+as opposed to a regular table or a custom table.
+Note that single-quotes within the view definition are escaped using a
+back-slash, in the normal way for JSON.
+
+JSON doesn't make it easy to author long strings, so Calcite supports an
+alternative syntax. If your view has a long SQL statement, you can instead
+supply a list of lines rather than a single string:
+
+```json
+        {
+          name: 'FEMALE_EMPS',
+          type: 'view',
+          sql: [
+            'SELECT * FROM emps',
+            'WHERE gender = \'F\''
+          ]
+        }
+```
+
+Now we have defined a view, we can use it in queries just as if it were a table:
+
+```sql
+sqlline> SELECT e.name, d.name FROM female_emps AS e JOIN depts AS d on e.deptno = d.deptno;
++--------+------------+
+|  NAME  |    NAME    |
++--------+------------+
+| Wilma  | Marketing  |
++--------+------------+
+```
+
+## Custom tables
+
+Custom tables are tables whose implementation is driven by user-defined code.
+They don't need to live in a custom schema.
+
+There is an example in <code>model-with-custom-table.json</code>:
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'CUSTOM_TABLE',
+  schemas: [
+    {
+      name: 'CUSTOM_TABLE',
+      tables: [
+        {
+          name: 'EMPS',
+          type: 'custom',
+          factory: 'org.apache.calcite.adapter.csv.CsvTableFactory',
+          operand: {
+            file: 'target/test-classes/sales/EMPS.csv.gz',
+            flavor: "scannable"
+          }
+        }
+      ]
+    }
+  ]
+}
+```
+
+We can query the table in the usual way:
+
+```sql
+sqlline> !connect jdbc:calcite:model=target/test-classes/model-with-custom-table.json admin admin
+sqlline> SELECT empno, name FROM custom_table.emps;
++--------+--------+
+| EMPNO  |  NAME  |
++--------+--------+
+| 100    | Fred   |
+| 110    | Eric   |
+| 110    | John   |
+| 120    | Wilma  |
+| 130    | Alice  |
++--------+--------+
+```
+
+The schema is a regular one, and contains a custom table powered by
+<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java">org.apache.calcite.adapter.csv.CsvTableFactory</a>,
+which implements the Calcite interface
+<a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/TableFactory.html">TableFactory</a>.
+Its <code>create</code> method instantiates a <code>CsvScannableTable</code>,
+passing in the <code>file</code> argument from the model file:
+
+```java
+public CsvTable create(SchemaPlus schema, String name,
+    Map<String, Object> map, RelDataType rowType) {
+  String fileName = (String) map.get("file");
+  final File file = new File(fileName);
+  final RelProtoDataType protoRowType =
+      rowType != null ? RelDataTypeImpl.proto(rowType) : null;
+  return new CsvScannableTable(file, protoRowType);
+}
+```
+
+Implementing a custom table is often a simpler alternative to implementing
+a custom schema. Both approaches might end up creating a similar implementation
+of the <code>Table</code> interface, but for the custom table you don't
+need to implement metadata discovery. (<code>CsvTableFactory</code>
+creates a <code>CsvScannableTable</code>, just as <code>CsvSchema</code> does,
+but the table implementation does not scan the filesystem for .csv files.)
+
+Custom tables require more work for the author of the model (the author
+needs to specify each table and its file explicitly) but also give the author
+more control (say, providing different parameters for each table).
+
+## Comments in models
+
+Models can include comments using `/* ... */` and `//` syntax:
+
+```json
+{
+  version: '1.0',
+  /* Multi-line
+     comment. */
+  defaultSchema: 'CUSTOM_TABLE',
+  // Single-line comment.
+  schemas: [
+    ..
+  ]
+}
+```
+
+(Comments are not standard JSON, but are a harmless extension.)
+
+## Optimizing queries using planner rules
+
+The table implementations we have seen so far are fine as long as the tables
+don't contain a great deal of data. But if your customer table has, say, a
+hundred columns and a million rows, you would rather that the system did not
+retrieve all of the data for every query. You would like Calcite to negotiate
+with the adapter and find a more efficient way of accessing the data.
+
+This negotiation is a simple form of query optimization. Calcite supports query
+optimization by adding <i>planner rules</i>. Planner rules operate by
+looking for patterns in the query parse tree (for instance a project on top
+of a certain kind of table), and
+
+Planner rules are also extensible, like schemas and tables. So, if you have a
+data store that you want to access via SQL, you first define a custom table or
+schema, and then you define some rules to make the access efficient.
+
+To see this in action, let's use a planner rule to access
+a subset of columns from a CSV file. Let's run the same query against two very
+similar schemas:
+
+```sql
+sqlline> !connect jdbc:calcite:model=target/test-classes/model.json admin admin
+sqlline> explain plan for select name from emps;
++-----------------------------------------------------+
+| PLAN                                                |
++-----------------------------------------------------+
+| EnumerableCalcRel(expr#0..9=[{inputs}], NAME=[$t1]) |
+|   EnumerableTableAccessRel(table=[[SALES, EMPS]])   |
++-----------------------------------------------------+
+sqlline> !connect jdbc:calcite:model=target/test-classes/smart.json admin admin
+sqlline> explain plan for select name from emps;
++-----------------------------------------------------+
+| PLAN                                                |
++-----------------------------------------------------+
+| EnumerableCalcRel(expr#0..9=[{inputs}], NAME=[$t1]) |
+|   CsvTableScan(table=[[SALES, EMPS]])               |
++-----------------------------------------------------+
+```
+
+What causes the difference in plan? Let's follow the trail of evidence. In the
+<code>smart.json</code> model file, there is just one extra line:
+
+```json
+flavor: "translatable"
+```
+
+This causes a <code>CsvSchema</code> to be created with
+<code>flavor = TRANSLATABLE</code>,
+and its <code>createTable</code> method creates instances of
+<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java">CsvTranslatableTable</a>
+rather than a <code>CsvScannableTable</code>.
+
+<code>CsvTranslatableTable</code> implements the
+<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/TranslatableTable.html#toRel()">TranslatableTable.toRel()</a></code>
+method to create
+<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java">CsvTableScan</a>.
+Table scans are the leaves of a query operator tree.
+The usual implementation is
+<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/adapter/enumerable/EnumerableTableScan.html">EnumerableTableScan</a></code>,
+but we have created a distinctive sub-type that will cause rules to fire.
+
+Here is the rule in its entirety:
+
+```java
+public class CsvProjectTableScanRule extends RelOptRule {
+  public static final CsvProjectTableScanRule INSTANCE =
+      new CsvProjectTableScanRule();
+
+  private CsvProjectTableScanRule() {
+    super(
+        operand(Project.class,
+            operand(CsvTableScan.class, none())),
+        "CsvProjectTableScanRule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final Project project = call.rel(0);
+    final CsvTableScan scan = call.rel(1);
+    int[] fields = getProjectFields(project.getProjects());
+    if (fields == null) {
+      // Project contains expressions more complex than just field references.
+      return;
+    }
+    call.transformTo(
+        new CsvTableScan(
+            scan.getCluster(),
+            scan.getTable(),
+            scan.csvTable,
+            fields));
+  }
+
+  private int[] getProjectFields(List<RexNode> exps) {
+    final int[] fields = new int[exps.size()];
+    for (int i = 0; i < exps.size(); i++) {
+      final RexNode exp = exps.get(i);
+      if (exp instanceof RexInputRef) {
+        fields[i] = ((RexInputRef) exp).getIndex();
+      } else {
+        return null; // not a simple projection
+      }
+    }
+    return fields;
+  }
+}
+```
+
+The constructor declares the pattern of relational expressions that will cause
+the rule to fire.
+
+The <code>onMatch</code> method generates a new relational expression and calls
+<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/plan/RelOptRuleCall.html#transformTo(org.apache.calcite.rel.RelNode)">RelOptRuleCall.transformTo()</a></code>
+to indicate that the rule has fired successfully.
+
+## The query optimization process
+
+There's a lot to say about how clever Calcite's query planner is, but we won't
+say it here. The cleverness is designed to take the burden off you, the writer
+of planner rules.
+
+First, Calcite doesn't fire rules in a prescribed order. The query optimization
+process follows many branches of a branching tree, just like a chess playing
+program examines many possible sequences of moves. If rules A and B both match a
+given section of the query operator tree, then Calcite can fire both.
+
+Second, Calcite uses cost in choosing between plans, but the cost model doesn't
+prevent rules from firing which may seem to be more expensive in the short term.
+
+Many optimizers have a linear optimization scheme. Faced with a choice between
+rule A and rule B, as above, such an optimizer needs to choose immediately. It
+might have a policy such as "apply rule A to the whole tree, then apply rule B
+to the whole tree", or apply a cost-based policy, applying the rule that
+produces the cheaper result.
+
+Calcite doesn't require such compromises.
+This makes it simple to combine various sets of rules.
+If, say you want to combine rules to recognize materialized views with rules to
+read from CSV and JDBC source systems, you just give Calcite the set of all
+rules and tell it to go at it.
+
+Calcite does use a cost model. The cost model decides which plan to ultimately
+use, and sometimes to prune the search tree to prevent the search space from
+exploding, but it never forces you to choose between rule A and rule B. This is
+important, because it avoids falling into local minima in the search space that
+are not actually optimal.
+
+Also (you guessed it) the cost model is pluggable, as are the table and query
+operator statistics it is based upon. But that can be a subject for later.
+
+## JDBC adapter
+
+The JDBC adapter maps a schema in a JDBC data source as a Calcite schema.
+
+For example, this schema reads from a MySQL "foodmart" database:
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'FOODMART',
+  schemas: [
+    {
+      name: 'FOODMART',
+      type: 'custom',
+      factory: 'org.apache.calcite.adapter.jdbc.JdbcSchema$Factory',
+      operand: {
+        jdbcDriver: 'com.mysql.jdbc.Driver',
+        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
+        jdbcUser: 'foodmart',
+        jdbcPassword: 'foodmart'
+      }
+    }
+  ]
+}
+```
+
+(The FoodMart database will be familiar to those of you who have used
+the Mondrian OLAP engine, because it is Mondrian's main test data
+set. To load the data set, follow <a
+href="http://mondrian.pentaho.com/documentation/installation.php#2_Set_up_test_data">Mondrian's
+installation instructions</a>.)
+
+<b>Current limitations</b>: The JDBC adapter currently only pushes
+down table scan operations; all other processing (filtering, joins,
+aggregations and so forth) occurs within Calcite. Our goal is to push
+down as much processing as possible to the source system, translating
+syntax, data types and built-in functions as we go. If a Calcite query
+is based on tables from a single JDBC database, in principle the whole
+query should go to that database. If tables are from multiple JDBC
+sources, or a mixture of JDBC and non-JDBC, Calcite will use the most
+efficient distributed query approach that it can.
+
+## The cloning JDBC adapter
+
+The cloning JDBC adapter creates a hybrid database. The data is
+sourced from a JDBC database but is read into in-memory tables the
+first time each table is accessed. Calcite evaluates queries based on
+those in-memory tables, effectively a cache of the database.
+
+For example, the following model reads tables from a MySQL
+"foodmart" database:
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'FOODMART_CLONE',
+  schemas: [
+    {
+      name: 'FOODMART_CLONE',
+      type: 'custom',
+      factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',
+      operand: {
+        jdbcDriver: 'com.mysql.jdbc.Driver',
+        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
+        jdbcUser: 'foodmart',
+        jdbcPassword: 'foodmart'
+      }
+    }
+  ]
+}
+```
+
+Another technique is to build a clone schema on top of an existing
+schema. You use the <code>source</code> property to reference a schema
+defined earlier in the model, like this:
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'FOODMART_CLONE',
+  schemas: [
+    {
+      name: 'FOODMART',
+      type: 'custom',
+      factory: 'org.apache.calcite.adapter.jdbc.JdbcSchema$Factory',
+      operand: {
+        jdbcDriver: 'com.mysql.jdbc.Driver',
+        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
+        jdbcUser: 'foodmart',
+        jdbcPassword: 'foodmart'
+      }
+    },
+    {
+      name: 'FOODMART_CLONE',
+      type: 'custom',
+      factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',
+      operand: {
+        source: 'FOODMART'
+      }
+    }
+  ]
+}
+```
+
+You can use this approach to create a clone schema on any type of
+schema, not just JDBC.
+
+The cloning adapter isn't the be-all and end-all. We plan to develop
+more sophisticated caching strategies, and a more complete and
+efficient implementation of in-memory tables, but for now the cloning
+JDBC adapter shows what is possible and allows us to try out our
+initial implementations.
+
+## Further topics
+
+### Defining a custom schema
+
+(To be written.)
+
+### Modifying data
+
+How to enable DML operations (INSERT, UPDATE and DELETE) on your schema.
+
+(To be written.)
+
+### Calling conventions
+
+(To be written.)
+
+### Statistics and cost
+
+(To be written.)
+
+### Defining and using user-defined functions
+
+(To be written.)
+
+###  Defining tables in a schema
+
+(To be written.)
+
+### Defining custom tables
+
+(To be written.)
+
+### Built-in SQL implementation
+
+How does Calcite implement SQL, if an adapter does not implement all of the core
+relational operators?
+
+(To be written.)
+
+### Table functions
+
+(To be written.)
+
+## Further resources
+
+* <a href="http://calcite.incubator.apache.org">Apache Calcite</a> home
+  page

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
index 7d63293..d3b37e1 100644
--- a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
+++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
@@ -40,7 +40,7 @@ import static org.junit.Assert.assertThat;
  * Tests for the {@code org.apache.calcite.adapter.mongodb} package.
  *
  * <p>Before calling this test, you need to populate MongoDB with the "zips"
- * data set (as described in HOWTO.md)
+ * data set (as described in howto.md)
  * and "foodmart" data set, as follows:</p>
  *
  * <blockquote><code>


[07/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/STREAM.md
----------------------------------------------------------------------
diff --git a/doc/STREAM.md b/doc/STREAM.md
deleted file mode 100644
index 4052ac1..0000000
--- a/doc/STREAM.md
+++ /dev/null
@@ -1,631 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# Calcite SQL extensions for streaming
-
-## Introduction
-
-Streams are collections to records that flow continuously, and forever.
-Unlike tables, they are not typically stored on disk, but flow over the
-network and are held for short periods of time in memory.
-
-Streams complement tables because they represent what is happening in the
-present and future of the enterprise whereas tables represent the past.
-It is very common for a stream to be archived into a table.
-
-Like tables, you often want to query streams in a high-level language
-based on relational algebra, validated according to a schema, and optimized
-to take advantage of available resources and algorithms.
-
-Calcite's SQL is an extension to standard SQL, not another 'SQL-like' language.
-The distinction is important, for several reasons:
-* Streaming SQL is easy to learn for anyone who knows regular SQL.
-* The semantics are clear, because we aim to produce the same results on a
-  stream as if the same data were in a table.
-* You can write queries that combine streams and tables (or the history of
-  a stream, which is basically an in-memory table).
-* Lots of existing tools can generate standard SQL.
-
-## An example schema
-
-Our streaming SQL examples use the following schema:
-* `Orders (rowtime, productId, orderId, units)` - a stream and a table
-* `Products (rowtime, productId, name)` - a table
-* `Shipments (rowtime, orderId)` - a stream
-
-## A simple query
-
-Let's start with the simplest streaming query:
-
-```sql
-SELECT STREAM *
-FROM Orders;
-
-  rowtime | productId | orderId | units
-----------+-----------+---------+-------
- 10:17:00 |        30 |       5 |     4
- 10:17:05 |        10 |       6 |     1
- 10:18:05 |        20 |       7 |     2
- 10:18:07 |        30 |       8 |    20
- 11:02:00 |        10 |       9 |     6
- 11:04:00 |        10 |      10 |     1
- 11:09:30 |        40 |      11 |    12
- 11:24:11 |        10 |      12 |     4
-```
-
-This query reads all columns and rows from the `Orders` stream.
-Like any streaming query, it never terminates. It outputs a record whenever
-a record arrives in `Orders`.
-
-Type `Control-C` to terminate the query.
-
-The `STREAM` keyword is the main extension in streaming SQL. It tells the
-system that you are interested in incoming orders, not existing ones. The query
-
-```sql
-SELECT *
-FROM Orders;
-
-  rowtime | productId | orderId | units
-----------+-----------+---------+-------
- 08:30:00 |        10 |       1 |     3
- 08:45:10 |        20 |       2 |     1
- 09:12:21 |        10 |       3 |    10
- 09:27:44 |        30 |       4 |     2
-
-4 records returned.
-```
-
-is also valid, but will print out all existing orders and then terminate. We
-call it a *relational* query, as opposed to *streaming*. It has traditional
-SQL semantics.
-
-`Orders` is special, in that it has both a stream and a table. If you try to run
-a streaming query on a table, or a relational query on a stream, Calcite gives
-an error:
-
-```sql
-> SELECT * FROM Shipments;
-ERROR: Cannot convert stream 'SHIPMENTS' to a table
-
-> SELECT STREAM * FROM Products;
-ERROR: Cannot convert table 'PRODUCTS' to a stream
-```
-
-# Filtering rows
-
-Just as in regular SQL, you use a `WHERE` clause to filter rows:
-
-```sql
-SELECT STREAM *
-FROM Orders
-WHERE units > 3;
-
-  rowtime | productId | orderId | units
-----------+-----------+---------+-------
- 10:17:00 |        30 |       5 |     4
- 10:18:07 |        30 |       8 |    20
- 11:02:00 |        10 |       9 |     6
- 11:09:30 |        40 |      11 |    12
- 11:24:11 |        10 |      12 |     4
-```
-
-# Projecting expressions
-
-Use expressions in the `SELECT` clause to choose which columns to return or
-compute expressions:
-
-```sql
-SELECT STREAM rowtime,
-  'An order for ' || units || ' '
-    || CASE units WHEN 1 THEN 'unit' ELSE 'units' END
-    || ' of product #' || productId AS description
-FROM Orders;
-
-  rowtime | description
-----------+---------------------------------------
- 10:17:00 | An order for 4 units of product #30
- 10:17:05 | An order for 1 unit of product #10
- 10:18:05 | An order for 2 units of product #20
- 10:18:07 | An order for 20 units of product #30
- 11:02:00 | An order by 6 units of product #10
- 11:04:00 | An order by 1 unit of product #10
- 11:09:30 | An order for 12 units of product #40
- 11:24:11 | An order by 4 units of product #10
-```
-
-We recommend that you always include the `rowtime` column in the `SELECT`
-clause. Having a sorted timestamp in each stream and streaming query makes it
-possible to do advanced calculations later, such as `GROUP BY` and `JOIN`.
-
-# Tumbling windows
-
-There are several ways to compute aggregate functions on streams. The
-differences are:
-* How many rows come out for each row in?
-* Does each incoming value appear in one total, or more?
-* What defines the "window", the set of rows that contribute to a given output row?
-* Is the result a stream or a relation?
-
-First we'll look a *tumbling window*, which is defined by a streaming
-`GROUP BY`. Here is an example:
-
-```sql
-SELECT STREAM FLOOR(rowtime TO HOUR) AS rowtime,
-  productId,
-  COUNT(*) AS c,
-  SUM(units) AS units
-FROM Orders
-GROUP BY FLOOR(rowtime TO HOUR), productId;
-
-  rowtime | productId |       c | units
-----------+-----------+---------+-------
- 10:00:00 |        30 |       2 |    24
- 10:00:00 |        10 |       1 |     1
- 10:00:00 |        20 |       1 |     7
- 11:00:00 |        10 |       3 |    11
- 11:00:00 |        40 |       1 |    12
-```
-
-The result is a stream. At 11 o'clock, Calcite emits a sub-total for every
-`productId` that had an order since 10 o'clock. At 12 o'clock, it will emit
-the orders that occurred between 11:00 and 12:00. Each input row contributes to
-only one output row.
-
-How did Calcite know that the 10:00:00 sub-totals were complete at 11:00:00,
-so that it could emit them? It knows that `rowtime` is increasing, and it knows
-that `FLOOR(rowtime TO HOUR)` is also increasing. So, once it has seen a row
-at or after 11:00:00, it will never see a row that will contribute to a 10:00:00
-total.
-
-A column or expression that is increasing or decreasing is said to be
-*monotonic*. Without a monotonic expression in the `GROUP BY` clause, Calcite is
-not able to make progress, and it will not allow the query:
-
-```sql
-> SELECT STREAM productId,
->   COUNT(*) AS c,
->   SUM(units) AS units
-> FROM Orders
-> GROUP BY productId;
-ERROR: Streaming aggregation requires at least one monotonic expression in GROUP BY clause
-```
-
-Monotonic columns need to be declared in the schema. The monotonicity is
-enforced when records enter the stream and assumed by queries that read from
-that stream. We recommend that you give each stream a timestamp column called
-`rowtime`, but you can declare others, `orderId`, for example.
-
-# Filtering after aggregation
-
-As in standard SQL, you can apply a `HAVING` clause to filter rows emitted by
-a streaming `GROUP BY`:
-
-```sql
-SELECT STREAM FLOOR(rowtime TO HOUR) AS rowtime,
-  productId
-FROM Orders
-GROUP BY FLOOR(rowtime TO HOUR), productId
-HAVING COUNT(*) > 2 OR SUM(units) > 10;
-
-  rowtime | productId
-----------+-----------
- 10:00:00 |        30
- 11:00:00 |        10
- 11:00:00 |        40
-```
-
-# Sub-queries, views and SQL's closure property
-
-The previous `HAVING` query can be expressed using a `WHERE` clause on a
-sub-query:
-
-```sql
-SELECT STREAM rowtime, productId
-FROM (
-  SELECT FLOOR(rowtime TO HOUR) AS rowtime,
-    productId,
-    COUNT(*) AS c,
-    SUM(units) AS su
-  FROM Orders
-  GROUP BY FLOOR(rowtime TO HOUR), productId)
-WHERE c > 2 OR su > 10;
-
-  rowtime | productId
-----------+-----------
- 10:00:00 |        30
- 11:00:00 |        10
- 11:00:00 |        40
-```
-
-`HAVING` was introduced in the early days of SQL, when a way was needed to
-perform a filter *after* aggregation. (Recall that `WHERE` filters rows before
-they enter the `GROUP BY` clause.)
-
-Since then, SQL has become a mathematically closed language, which means that
-any operation you can perform on a table can also perform on a query.
-
-The *closure property* of SQL is extremely powerful. Not only does it render
-`HAVING` obsolete (or, at least, reduce it to syntactic sugar), it makes views
-possible:
-
-```sql
-CREATE VIEW HourlyOrderTotals (rowtime, productId, c, su) AS
-  SELECT FLOOR(rowtime TO HOUR),
-    productId,
-    COUNT(*),
-    SUM(units)
-  FROM Orders
-  GROUP BY FLOOR(rowtime TO HOUR), productId;
-
-SELECT STREAM rowtime, productId
-FROM HourlyOrderTotals
-WHERE c > 2 OR su > 10;
-
-  rowtime | productId
-----------+-----------
- 10:00:00 |        30
- 11:00:00 |        10
- 11:00:00 |        40
-```
-
-Sub-queries in the `FROM` clause are sometimes referred to as "inline views",
-but really, nested queries are more fundamental. Views are just a convenient
-way to carve your SQL into manageable chunks.
-
-Many people find that nested queries and views are even more useful on streams
-than they are on relations. Streaming queries are pipelines of
-operators all running continuously, and often those pipelines get quite long.
-Nested queries and views help to express and manage those pipelines.
-
-And, by the way, a `WITH` clause can accomplish the same as a sub-query or
-a view:
-
-```sql
-WITH HourlyOrderTotals (rowtime, productId, c, su) AS (
-  SELECT FLOOR(rowtime TO HOUR),
-    productId,
-    COUNT(*),
-    SUM(units)
-  FROM Orders
-  GROUP BY FLOOR(rowtime TO HOUR), productId)
-SELECT STREAM rowtime, productId
-FROM HourlyOrderTotals
-WHERE c > 2 OR su > 10;
-
-  rowtime | productId
-----------+-----------
- 10:00:00 |        30
- 11:00:00 |        10
- 11:00:00 |        40
-```
-
-## Converting between streams and relations
-
-Look back at the definition of the `HourlyOrderTotals` view.
-Is the view a stream or a relation?
-
-It does not contain the `STREAM` keyword, so it is a relation.
-However, it is a relation that can be converted into a stream.
-
-You can use it in both relational and streaming queries:
-
-```sql
-# A relation; will query the historic Orders table.
-# Returns the largest number of product #10 ever sold in one hour.
-SELECT max(su)
-FROM HourlyOrderTotals
-WHERE productId = 10;
-
-# A stream; will query the Orders stream.
-# Returns every hour in which at least one product #10 was sold.
-SELECT STREAM rowtime
-FROM HourlyOrderTotals
-WHERE productId = 10;
-```
-
-This approach is not limited to views and sub-queries.
-Following the approach set out in CQL [<a href="#ref1">1</a>], every query
-in streaming SQL is defined as a relational query and converted to a stream
-using the `STREAM` keyword in the top-most `SELECT`.
-
-If the `STREAM` keyword is present in sub-queries or view definitions, it has no
-effect.
-
-At query preparation time, Calcite figures out whether the relations referenced
-in the query can be converted to streams or historical relations.
-
-Sometimes a stream makes available some of its history (say the last 24 hours of
-data in an Apache Kafka [<a href="#ref2">2</a>] topic)
-but not all. At run time, Calcite figures out whether there is sufficient
-history to run the query, and if not, gives an error.
-
-## Hopping windows
-
-Previously we saw how to define a tumbling window using a `GROUP BY` clause.
-Each record contributed to a single sub-total record, the one containing its
-hour and product id.
-
-But suppose we want to emit, every hour, the number of each product ordered over
-the past three hours. To do this, we use `SELECT ... OVER` and a sliding window
-to combine multiple tumbling windows.
-
-```sql
-SELECT STREAM rowtime,
-  productId,
-  SUM(su) OVER w AS su,
-  SUM(c) OVER w AS c
-FROM HourlyTotals
-WINDOW w AS (
-  ORDER BY rowtime
-  PARTITION BY productId
-  RANGE INTERVAL '2' HOUR PRECEDING)
-```
-
-This query uses the `HourlyOrderTotals` view defined previously.
-The 2 hour interval combines the totals timestamped 09:00:00, 10:00:00 and
-11:00:00 for a particular product into a single total timestamped 11:00:00 and
-summarizing orders for that product between 09:00:00 and 12:00:00.
-
-## Limitations of tumbling and hopping windows
-
-In the present syntax, we acknowledge that it is not easy to create certain
-kinds of windows.
-
-First, let's consider tumbling windows over complex periods.
-
-The `FLOOR` and `CEIL` functions make is easy to create a tumbling window that
-emits on a whole time unit (say every hour, or every minute) but less easy to
-emit, say, every 15 minutes. One could imagine an extension to the `FLOOR`
-function that emits unique values on just about any periodic basis (say in 11
-minute intervals starting from midnight of the current day).
-
-Next, let's consider hopping windows whose retention period is not a multiple
-of its emission period. Say we want to output, at the top of each hour, the
-orders for the previous 7,007 seconds. If we were to simulate this hopping
-window using a sliding window over a tumbling window, as before, we would have
-to sum lots of 1-second windows (because 3,600 and 7,007 are co-prime).
-This is a lot of effort for both the system and the person writing the query.
-
-Calcite could perhaps solve this generalizing `GROUP BY` syntax, but we would
-be destroying the principle that an input row into a `GROUP BY` appears in
-precisely one output row.
-
-Calcite's SQL extensions for streaming queries are evolving. As we learn more
-about how people wish to query streams, we plan to make the language more
-expressive while remaining compatible with standard SQL and consistent with
-its principles, look and feel.
-
-## Sorting
-
-The story for `ORDER BY` is similar to `GROUP BY`.
-The syntax looks like regular SQL, but Calcite must be sure that it can deliver
-timely results. It therefore requires a monotonic expression on the leading edge
-of your `ORDER BY` key.
-
-```sql
-SELECT STREAM FLOOR(rowtime TO hour) AS rowtime, productId, orderId, units
-FROM Orders
-ORDER BY FLOOR(rowtime TO hour) ASC, units DESC;
-
-  rowtime | productId | orderId | units
-----------+-----------+---------+-------
- 10:00:00 |        30 |       8 |    20
- 10:00:00 |        30 |       5 |     4
- 10:00:00 |        20 |       7 |     2
- 10:00:00 |        10 |       6 |     1
- 11:00:00 |        40 |      11 |    12
- 11:00:00 |        10 |       9 |     6
- 11:00:00 |        10 |      12 |     4
- 11:00:00 |        10 |      10 |     1
-```
-
-Most queries will return results in the order that they were inserted,
-because the engine is using streaming algorithms, but you should not rely on it.
-For example, consider this:
-
-```sql
-SELECT STREAM *
-FROM Orders
-WHERE productId = 10
-UNION ALL
-SELECT STREAM *
-FROM Orders
-WHERE productId = 30;
-
-  rowtime | productId | orderId | units
-----------+-----------+---------+-------
- 10:17:05 |        10 |       6 |     1
- 10:17:00 |        30 |       5 |     4
- 10:18:07 |        30 |       8 |    20
- 11:02:00 |        10 |       9 |     6
- 11:04:00 |        10 |      10 |     1
- 11:24:11 |        10 |      12 |     4
-```
-
-The rows with `productId` = 30 are apparently out of order, probably because
-the `Orders` stream was partitioned on `productId` and the partitioned streams
-sent their data at different times.
-
-If you require a particular ordering, add an explicit `ORDER BY`:
-
-```sql
-SELECT STREAM *
-FROM Orders
-WHERE productId = 10
-UNION ALL
-SELECT STREAM *
-FROM Orders
-WHERE productId = 30
-ORDER BY rowtime;
-
-  rowtime | productId | orderId | units
-----------+-----------+---------+-------
- 10:17:00 |        30 |       5 |     4
- 10:17:05 |        10 |       6 |     1
- 10:18:07 |        30 |       8 |    20
- 11:02:00 |        10 |       9 |     6
- 11:04:00 |        10 |      10 |     1
- 11:24:11 |        10 |      12 |     4
-```
-
-Calcite will probably implement the `UNION ALL` by merging using `rowtime`,
-which is only slightly less efficient.
-
-You only need to add an `ORDER BY` to the outermost query. If you need to,
-say, perform `GROUP BY` after a `UNION ALL`, Calcite will add an `ORDER BY`
-implicitly, in order to make the GROUP BY algorithm possible.
-
-## Table constructor
-
-The `VALUES` clause creates an inline table with a given set of rows.
-
-Streaming is disallowed. The set of rows never changes, and therefore a stream
-would never return any rows.
-
-```sql
-> SELECT STREAM * FROM (VALUES (1, 'abc'));
-
-ERROR: Cannot stream VALUES
-```
-
-## Sliding windows
-
-Standard SQL features so-called "analytic functions" that can be used in the
-`SELECT` clause. Unlike `GROUP BY`, these do not collapse records. For each
-record that goes in, one record comes out. But the aggregate function is based
-on a window of many rows.
-
-Let's look at an example.
-
-```sql
-SELECT STREAM rowtime,
-  productId,
-  units,
-  SUM(units) OVER (ORDER BY rowtime RANGE INTERVAL '1' HOUR PRECEDING) unitsLastHour
-FROM Orders;
-```
-
-The feature packs a lot of power with little effort. You can have multiple
-functions in the `SELECT` clause, based on multiple window specifications.
-
-The following example returns orders whose average order size over the last
-10 minutes is greater than the average order size for the last week.
-
-```sql
-SELECT STREAM *
-FROM (
-  SELECT STREAM rowtime,
-    productId,
-    units,
-    AVG(units) OVER product (RANGE INTERVAL '10' MINUTE PRECEDING) AS m10,
-    AVG(units) OVER product (RANGE INTERVAL '7' DAY PRECEDING) AS d7
-  FROM Orders
-  WINDOW product AS (
-    ORDER BY rowtime
-    PARTITION BY productId))
-WHERE m10 > d7;
-```
-
-For conciseness, here we use a syntax where you partially define a window
-using a `WINDOW` clause and then refine the window in each `OVER` clause.
-You could also define all windows in the `WINDOW` clause, or all windows inline,
-if you wish.
-
-But the real power goes beyond syntax. Behind the scenes, this query is
-maintaining two tables, and adding and removing values from sub-totals using
-with FIFO queues. But you can access those tables without introducing a join
-into the query.
-
-Some other features of the windowed aggregation syntax:
-* You can define windows based on row count.
-* The window can reference rows that have not yet arrived.
-  (The stream will wait until they have arrived).
-* You can compute order-dependent functions such as `RANK` and median.
-
-## Cascading windows
-
-What if we want a query that returns a result for every record, like a
-sliding window, but resets totals on a fixed time period, like a
-tumbling window? Such a pattern is called a *cascading window*. Here
-is an example:
-
-```sql
-SELECT STREAM rowtime,
-  productId,
-  units,
-  SUM(units) OVER (PARTITION BY FLOOR(rowtime TO HOUR)) AS unitsSinceTopOfHour
-FROM Orders;
-```
-
-It looks similar to a sliding window query, but the monotonic
-expression occurs within the `PARTITION BY` clause of the window. As
-the rowtime moves from from 10:59:59 to 11:00:00, `FLOOR(rowtime TO
-HOUR)` changes from 10:00:00 to 11:00:00, and therefore a new
-partition starts. The first row to arrive in the new hour will start a
-new total; the second row will have a total that consists of two rows,
-and so on.
-
-Calcite knows that the old partition will never be used again, so
-removes all sub-totals for that partition from its internal storage.
-
-Analytic functions that using cascading and sliding windows can be
-combined in the same query.
-
-## State of the stream
-
-Not all concepts in this article have been implemented in Calcite.
-And others may be implemented in Calcite but not in a particular adapter
-such as Samza SQL [<a href="#ref3">3</a>].
-
-### Implemented
-* Streaming SELECT, WHERE, GROUP BY, HAVING, UNION ALL, ORDER BY
-* FLOOR and CEILING functions
-* Monotonicity
-* Streaming VALUES is disallowed
-
-### Not implemented
-* Stream-to-stream JOIN
-* Stream-to-table JOIN
-* Stream on view
-* Streaming UNION ALL with ORDER BY (merge)
-* Relational query on stream
-* Streaming windowed aggregation (sliding and cascading windows)
-* Check that STREAM in sub-queries and views is ignored
-* Check that streaming ORDER BY cannot have OFFSET or LIMIT
-* Limited history; at run time, check that there is sufficient history
-  to run the query.
-
-### To do in this document
-* Re-visit whether you can stream VALUES
-* OVER clause to define window on stream
-* Windowed aggregation
-* Punctuation
-* Stream-to-table join
-** Stream-to-table join where table is changing
-* Stream-to-stream join
-* Relational queries on streams (e.g. "pie chart" query)
-* Diagrams for various window types
-
-## References
-
-* [<a name="ref1">1</a>]
-  <a href="http://ilpubs.stanford.edu:8090/758/">Arasu, Arvind and Babu,
-  Shivnath and Widom, Jennifer (2003) The CQL Continuous Query
-  Language: Semantic Foundations and Query Execution</a>.
-* [<a name="ref2">2</a>]
-  <a href="http://kafka.apache.org/documentation.html">Apache Kafka</a>.
-* [<a name="ref3">3</a>] <a href="http://samza.apache.org">Apache Samza</a>.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/TUTORIAL.md
----------------------------------------------------------------------
diff --git a/doc/TUTORIAL.md b/doc/TUTORIAL.md
deleted file mode 100644
index f0edafc..0000000
--- a/doc/TUTORIAL.md
+++ /dev/null
@@ -1,753 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# CSV Adapter Tutorial
-
-Calcite-example-CSV is a fully functional adapter for
-<a href="https://github.com/apache/incubator-calcite">Calcite</a> that reads
-text files in
-<a href="http://en.wikipedia.org/wiki/Comma-separated_values">CSV
-(comma-separated values)</a> format. It is remarkable that a couple of
-hundred lines of Java code are sufficient to provide full SQL query
-capability.
-
-CSV also serves as a template for building adapters to other
-data formats. Even though there are not many lines of code, it covers
-several important concepts:
-* user-defined schema using SchemaFactory and Schema interfaces;
-* declaring schemas in a model JSON file;
-* declaring views in a model JSON file;
-* user-defined table using the Table interface;
-* determining the record type of a table;
-* a simple implementation of Table, using the ScannableTable interface, that
-  enumerates all rows directly;
-* a more advanced implementation that implements FilterableTable, and can
-  filter out rows according to simple predicates;
-* advanced implementation of Table, using TranslatableTable, that translates
-  to relational operators using planner rules.
-
-## Download and build
-
-You need Java (1.6 or higher; 1.7 preferred), git and maven (3.0.4 or later).
-
-```bash
-$ git clone https://github.com/apache/incubator-calcite.git
-$ cd incubator-calcite
-$ mvn install -DskipTests -Dcheckstyle.skip=true
-$ cd example/csv
-```
-
-## First queries
-
-Now let's connect to Calcite using
-<a href="https://github.com/julianhyde/sqlline">sqlline</a>, a SQL shell
-that is included in this project.
-
-```bash
-$ ./sqlline
-sqlline> !connect jdbc:calcite:model=target/test-classes/model.json admin admin
-```
-
-(If you are running Windows, the command is `sqlline.bat`.)
-
-Execute a metadata query:
-
-```bash
-sqlline> !tables
-+------------+--------------+-------------+---------------+----------+------+
-| TABLE_CAT  | TABLE_SCHEM  | TABLE_NAME  |  TABLE_TYPE   | REMARKS  | TYPE |
-+------------+--------------+-------------+---------------+----------+------+
-| null       | SALES        | DEPTS       | TABLE         | null     | null |
-| null       | SALES        | EMPS        | TABLE         | null     | null |
-| null       | SALES        | HOBBIES     | TABLE         | null     | null |
-| null       | metadata     | COLUMNS     | SYSTEM_TABLE  | null     | null |
-| null       | metadata     | TABLES      | SYSTEM_TABLE  | null     | null |
-+------------+--------------+-------------+---------------+----------+------+
-```
-
-(JDBC experts, note: sqlline's <code>!tables</code> command is just executing
-<a href="http://docs.oracle.com/javase/7/docs/api/java/sql/DatabaseMetaData.html#getTables(java.lang.String, java.lang.String, java.lang.String, java.lang.String[])"><code>DatabaseMetaData.getTables()</code></a>
-behind the scenes.
-It has other commands to query JDBC metadata, such as <code>!columns</code> and <code>!describe</code>.)
-
-As you can see there are 5 tables in the system: tables
-<code>EMPS</code>, <code>DEPTS</code> and <code>HOBBIES</code> in the current
-<code>SALES</code> schema, and <code>COLUMNS</code> and
-<code>TABLES</code> in the system <code>metadata</code> schema. The
-system tables are always present in Calcite, but the other tables are
-provided by the specific implementation of the schema; in this case,
-the <code>EMPS</code> and <code>DEPTS</code> tables are based on the
-<code>EMPS.csv</code> and <code>DEPTS.csv</code> files in the
-<code>target/test-classes</code> directory.
-
-Let's execute some queries on those tables, to show that Calcite is providing
-a full implementation of SQL. First, a table scan:
-
-```bash
-sqlline> SELECT * FROM emps;
-+--------+--------+---------+---------+----------------+--------+-------+---+
-| EMPNO  |  NAME  | DEPTNO  | GENDER  |      CITY      | EMPID  |  AGE  | S |
-+--------+--------+---------+---------+----------------+--------+-------+---+
-| 100    | Fred   | 10      |         |                | 30     | 25    | t |
-| 110    | Eric   | 20      | M       | San Francisco  | 3      | 80    | n |
-| 110    | John   | 40      | M       | Vancouver      | 2      | null  | f |
-| 120    | Wilma  | 20      | F       |                | 1      | 5     | n |
-| 130    | Alice  | 40      | F       | Vancouver      | 2      | null  | f |
-+--------+--------+---------+---------+----------------+--------+-------+---+
-```
-
-Now JOIN and GROUP BY:
-
-```bash
-sqlline> SELECT d.name, COUNT(*)
-. . . .> FROM emps AS e JOIN depts AS d ON e.deptno = d.deptno
-. . . .> GROUP BY d.name;
-+------------+---------+
-|    NAME    | EXPR$1  |
-+------------+---------+
-| Sales      | 1       |
-| Marketing  | 2       |
-+------------+---------+
-```
-
-Last, the VALUES operator generates a single row, and is a convenient
-way to test expressions and SQL built-in functions:
-
-```bash
-sqlline> VALUES CHAR_LENGTH('Hello, ' || 'world!');
-+---------+
-| EXPR$0  |
-+---------+
-| 13      |
-+---------+
-```
-
-Calcite has many other SQL features. We don't have time to cover them
-here. Write some more queries to experiment.
-
-## Schema discovery
-
-Now, how did Calcite find these tables? Remember, core Calcite does not
-know anything about CSV files. (As a "database without a storage
-layer", Calcite doesn't know about any file formats.) Calcite knows about
-those tables because we told it to run code in the calcite-example-csv
-project.
-
-There are a couple of steps in that chain. First, we define a schema
-based on a schema factory class in a model file. Then the schema
-factory creates a schema, and the schema creates several tables, each
-of which knows how to get data by scanning a CSV file. Last, after
-Calcite has parsed the query and planned it to use those tables, Calcite
-invokes the tables to read the data as the query is being
-executed. Now let's look at those steps in more detail.
-
-On the JDBC connect string we gave the path of a model in JSON
-format. Here is the model:
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'SALES',
-  schemas: [
-    {
-      name: 'SALES',
-      type: 'custom',
-      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
-      operand: {
-        directory: 'target/test-classes/sales'
-      }
-    }
-  ]
-}
-```
-
-The model defines a single schema called 'SALES'. The schema is
-powered by a plugin class,
-<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java">org.apache.calcite.adapter.csv.CsvSchemaFactory</a>,
-which is part of the
-calcite-example-csv project and implements the Calcite interface
-<a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/SchemaFactory.html">SchemaFactory</a>.
-Its <code>create</code> method instantiates a
-schema, passing in the <code>directory</code> argument from the model file:
-
-```java
-public Schema create(SchemaPlus parentSchema, String name,
-    Map<String, Object> operand) {
-  String directory = (String) operand.get("directory");
-  String flavorName = (String) operand.get("flavor");
-  CsvTable.Flavor flavor;
-  if (flavorName == null) {
-    flavor = CsvTable.Flavor.SCANNABLE;
-  } else {
-    flavor = CsvTable.Flavor.valueOf(flavorName.toUpperCase());
-  }
-  return new CsvSchema(
-      new File(directory),
-      flavor);
-}
-```
-
-Driven by the model, the schema factory instantiates a single schema
-called 'SALES'.  The schema is an instance of
-<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java">org.apache.calcite.adapter.csv.CsvSchema</a>
-and implements the Calcite interface <a
-href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/Schema.html">Schema</a>.
-
-A schema's job is to produce a list of tables. (It can also list sub-schemas and
-table-functions, but these are advanced features and calcite-example-csv does
-not support them.) The tables implement Calcite's
-<a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/Table.html">Table</a>
-interface. <code>CsvSchema</code> produces tables that are instances of
-<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java">CsvTable</a>
-and its sub-classes.
-
-Here is the relevant code from <code>CsvSchema</code>, overriding the
-<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/impl/AbstractSchema.html#getTableMap()">getTableMap()</a></code>
-method in the <code>AbstractSchema</code> base class.
-
-```java
-protected Map<String, Table> getTableMap() {
-  // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
-  // ".json.gz".
-  File[] files = directoryFile.listFiles(
-      new FilenameFilter() {
-        public boolean accept(File dir, String name) {
-          final String nameSansGz = trim(name, ".gz");
-          return nameSansGz.endsWith(".csv")
-              || nameSansGz.endsWith(".json");
-        }
-      });
-  if (files == null) {
-    System.out.println("directory " + directoryFile + " not found");
-    files = new File[0];
-  }
-  // Build a map from table name to table; each file becomes a table.
-  final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
-  for (File file : files) {
-    String tableName = trim(file.getName(), ".gz");
-    final String tableNameSansJson = trimOrNull(tableName, ".json");
-    if (tableNameSansJson != null) {
-      JsonTable table = new JsonTable(file);
-      builder.put(tableNameSansJson, table);
-      continue;
-    }
-    tableName = trim(tableName, ".csv");
-    final Table table = createTable(file);
-    builder.put(tableName, table);
-  }
-  return builder.build();
-}
-
-/** Creates different sub-type of table based on the "flavor" attribute. */
-private Table createTable(File file) {
-  switch (flavor) {
-  case TRANSLATABLE:
-    return new CsvTranslatableTable(file, null);
-  case SCANNABLE:
-    return new CsvScannableTable(file, null);
-  case FILTERABLE:
-    return new CsvFilterableTable(file, null);
-  default:
-    throw new AssertionError("Unknown flavor " + flavor);
-  }
-}
-```
-
-The schema scans the directory and finds all files whose name ends
-with ".csv" and creates tables for them. In this case, the directory
-is <code>target/test-classes/sales</code> and contains files
-<code>EMPS.csv</code> and <code>DEPTS.csv</code>, which these become
-the tables <code>EMPS</code> and <code>DEPTS</code>.
-
-## Tables and views in schemas
-
-Note how we did not need to define any tables in the model; the schema
-generated the tables automatically.
-
-You can define extra tables,
-beyond those that are created automatically,
-using the <code>tables</code> property of a schema.
-
-Let's see how to create
-an important and useful type of table, namely a view.
-
-A view looks like a table when you are writing a query, but it doesn't store data.
-It derives its result by executing a query.
-The view is expanded while the query is being planned, so the query planner
-can often perform optimizations like removing expressions from the SELECT
-clause that are not used in the final result.
-
-Here is a schema that defines a view:
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'SALES',
-  schemas: [
-    {
-      name: 'SALES',
-      type: 'custom',
-      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
-      operand: {
-        directory: 'target/test-classes/sales'
-      },
-      tables: [
-        {
-          name: 'FEMALE_EMPS',
-          type: 'view',
-          sql: 'SELECT * FROM emps WHERE gender = \'F\''
-        }
-      ]
-    }
-  ]
-}
-```
-
-The line <code>type: 'view'</code> tags <code>FEMALE_EMPS</code> as a view,
-as opposed to a regular table or a custom table.
-Note that single-quotes within the view definition are escaped using a
-back-slash, in the normal way for JSON.
-
-JSON doesn't make it easy to author long strings, so Calcite supports an
-alternative syntax. If your view has a long SQL statement, you can instead
-supply a list of lines rather than a single string:
-
-```json
-        {
-          name: 'FEMALE_EMPS',
-          type: 'view',
-          sql: [
-            'SELECT * FROM emps',
-            'WHERE gender = \'F\''
-          ]
-        }
-```
-
-Now we have defined a view, we can use it in queries just as if it were a table:
-
-```sql
-sqlline> SELECT e.name, d.name FROM female_emps AS e JOIN depts AS d on e.deptno = d.deptno;
-+--------+------------+
-|  NAME  |    NAME    |
-+--------+------------+
-| Wilma  | Marketing  |
-+--------+------------+
-```
-
-## Custom tables
-
-Custom tables are tables whose implementation is driven by user-defined code.
-They don't need to live in a custom schema.
-
-There is an example in <code>model-with-custom-table.json</code>:
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'CUSTOM_TABLE',
-  schemas: [
-    {
-      name: 'CUSTOM_TABLE',
-      tables: [
-        {
-          name: 'EMPS',
-          type: 'custom',
-          factory: 'org.apache.calcite.adapter.csv.CsvTableFactory',
-          operand: {
-            file: 'target/test-classes/sales/EMPS.csv.gz',
-            flavor: "scannable"
-          }
-        }
-      ]
-    }
-  ]
-}
-```
-
-We can query the table in the usual way:
-
-```sql
-sqlline> !connect jdbc:calcite:model=target/test-classes/model-with-custom-table.json admin admin
-sqlline> SELECT empno, name FROM custom_table.emps;
-+--------+--------+
-| EMPNO  |  NAME  |
-+--------+--------+
-| 100    | Fred   |
-| 110    | Eric   |
-| 110    | John   |
-| 120    | Wilma  |
-| 130    | Alice  |
-+--------+--------+
-```
-
-The schema is a regular one, and contains a custom table powered by
-<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java">org.apache.calcite.adapter.csv.CsvTableFactory</a>,
-which implements the Calcite interface
-<a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/TableFactory.html">TableFactory</a>.
-Its <code>create</code> method instantiates a <code>CsvScannableTable</code>,
-passing in the <code>file</code> argument from the model file:
-
-```java
-public CsvTable create(SchemaPlus schema, String name,
-    Map<String, Object> map, RelDataType rowType) {
-  String fileName = (String) map.get("file");
-  final File file = new File(fileName);
-  final RelProtoDataType protoRowType =
-      rowType != null ? RelDataTypeImpl.proto(rowType) : null;
-  return new CsvScannableTable(file, protoRowType);
-}
-```
-
-Implementing a custom table is often a simpler alternative to implementing
-a custom schema. Both approaches might end up creating a similar implementation
-of the <code>Table</code> interface, but for the custom table you don't
-need to implement metadata discovery. (<code>CsvTableFactory</code>
-creates a <code>CsvScannableTable</code>, just as <code>CsvSchema</code> does,
-but the table implementation does not scan the filesystem for .csv files.)
-
-Custom tables require more work for the author of the model (the author
-needs to specify each table and its file explicitly) but also give the author
-more control (say, providing different parameters for each table).
-
-## Comments in models
-
-Models can include comments using `/* ... */` and `//` syntax:
-
-```json
-{
-  version: '1.0',
-  /* Multi-line
-     comment. */
-  defaultSchema: 'CUSTOM_TABLE',
-  // Single-line comment.
-  schemas: [
-    ..
-  ]
-}
-```
-
-(Comments are not standard JSON, but are a harmless extension.)
-
-## Optimizing queries using planner rules
-
-The table implementations we have seen so far are fine as long as the tables
-don't contain a great deal of data. But if your customer table has, say, a
-hundred columns and a million rows, you would rather that the system did not
-retrieve all of the data for every query. You would like Calcite to negotiate
-with the adapter and find a more efficient way of accessing the data.
-
-This negotiation is a simple form of query optimization. Calcite supports query
-optimization by adding <i>planner rules</i>. Planner rules operate by
-looking for patterns in the query parse tree (for instance a project on top
-of a certain kind of table), and
-
-Planner rules are also extensible, like schemas and tables. So, if you have a
-data store that you want to access via SQL, you first define a custom table or
-schema, and then you define some rules to make the access efficient.
-
-To see this in action, let's use a planner rule to access
-a subset of columns from a CSV file. Let's run the same query against two very
-similar schemas:
-
-```sql
-sqlline> !connect jdbc:calcite:model=target/test-classes/model.json admin admin
-sqlline> explain plan for select name from emps;
-+-----------------------------------------------------+
-| PLAN                                                |
-+-----------------------------------------------------+
-| EnumerableCalcRel(expr#0..9=[{inputs}], NAME=[$t1]) |
-|   EnumerableTableAccessRel(table=[[SALES, EMPS]])   |
-+-----------------------------------------------------+
-sqlline> !connect jdbc:calcite:model=target/test-classes/smart.json admin admin
-sqlline> explain plan for select name from emps;
-+-----------------------------------------------------+
-| PLAN                                                |
-+-----------------------------------------------------+
-| EnumerableCalcRel(expr#0..9=[{inputs}], NAME=[$t1]) |
-|   CsvTableScan(table=[[SALES, EMPS]])               |
-+-----------------------------------------------------+
-```
-
-What causes the difference in plan? Let's follow the trail of evidence. In the
-<code>smart.json</code> model file, there is just one extra line:
-
-```json
-flavor: "translatable"
-```
-
-This causes a <code>CsvSchema</code> to be created with
-<code>flavor = TRANSLATABLE</code>,
-and its <code>createTable</code> method creates instances of
-<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java">CsvTranslatableTable</a>
-rather than a <code>CsvScannableTable</code>.
-
-<code>CsvTranslatableTable</code> implements the
-<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/schema/TranslatableTable.html#toRel()">TranslatableTable.toRel()</a></code>
-method to create
-<a href="../example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java">CsvTableScan</a>.
-Table scans are the leaves of a query operator tree.
-The usual implementation is
-<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/adapter/enumerable/EnumerableTableScan.html">EnumerableTableScan</a></code>,
-but we have created a distinctive sub-type that will cause rules to fire.
-
-Here is the rule in its entirety:
-
-```java
-public class CsvProjectTableScanRule extends RelOptRule {
-  public static final CsvProjectTableScanRule INSTANCE =
-      new CsvProjectTableScanRule();
-
-  private CsvProjectTableScanRule() {
-    super(
-        operand(Project.class,
-            operand(CsvTableScan.class, none())),
-        "CsvProjectTableScanRule");
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final Project project = call.rel(0);
-    final CsvTableScan scan = call.rel(1);
-    int[] fields = getProjectFields(project.getProjects());
-    if (fields == null) {
-      // Project contains expressions more complex than just field references.
-      return;
-    }
-    call.transformTo(
-        new CsvTableScan(
-            scan.getCluster(),
-            scan.getTable(),
-            scan.csvTable,
-            fields));
-  }
-
-  private int[] getProjectFields(List<RexNode> exps) {
-    final int[] fields = new int[exps.size()];
-    for (int i = 0; i < exps.size(); i++) {
-      final RexNode exp = exps.get(i);
-      if (exp instanceof RexInputRef) {
-        fields[i] = ((RexInputRef) exp).getIndex();
-      } else {
-        return null; // not a simple projection
-      }
-    }
-    return fields;
-  }
-}
-```
-
-The constructor declares the pattern of relational expressions that will cause
-the rule to fire.
-
-The <code>onMatch</code> method generates a new relational expression and calls
-<code><a href="http://www.hydromatic.net/calcite/apidocs/org/apache/calcite/plan/RelOptRuleCall.html#transformTo(org.apache.calcite.rel.RelNode)">RelOptRuleCall.transformTo()</a></code>
-to indicate that the rule has fired successfully.
-
-## The query optimization process
-
-There's a lot to say about how clever Calcite's query planner is, but we won't
-say it here. The cleverness is designed to take the burden off you, the writer
-of planner rules.
-
-First, Calcite doesn't fire rules in a prescribed order. The query optimization
-process follows many branches of a branching tree, just like a chess playing
-program examines many possible sequences of moves. If rules A and B both match a
-given section of the query operator tree, then Calcite can fire both.
-
-Second, Calcite uses cost in choosing between plans, but the cost model doesn't
-prevent rules from firing which may seem to be more expensive in the short term.
-
-Many optimizers have a linear optimization scheme. Faced with a choice between
-rule A and rule B, as above, such an optimizer needs to choose immediately. It
-might have a policy such as "apply rule A to the whole tree, then apply rule B
-to the whole tree", or apply a cost-based policy, applying the rule that
-produces the cheaper result.
-
-Calcite doesn't require such compromises.
-This makes it simple to combine various sets of rules.
-If, say you want to combine rules to recognize materialized views with rules to
-read from CSV and JDBC source systems, you just give Calcite the set of all
-rules and tell it to go at it.
-
-Calcite does use a cost model. The cost model decides which plan to ultimately
-use, and sometimes to prune the search tree to prevent the search space from
-exploding, but it never forces you to choose between rule A and rule B. This is
-important, because it avoids falling into local minima in the search space that
-are not actually optimal.
-
-Also (you guessed it) the cost model is pluggable, as are the table and query
-operator statistics it is based upon. But that can be a subject for later.
-
-## JDBC adapter
-
-The JDBC adapter maps a schema in a JDBC data source as a Calcite schema.
-
-For example, this schema reads from a MySQL "foodmart" database:
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'FOODMART',
-  schemas: [
-    {
-      name: 'FOODMART',
-      type: 'custom',
-      factory: 'org.apache.calcite.adapter.jdbc.JdbcSchema$Factory',
-      operand: {
-        jdbcDriver: 'com.mysql.jdbc.Driver',
-        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
-        jdbcUser: 'foodmart',
-        jdbcPassword: 'foodmart'
-      }
-    }
-  ]
-}
-```
-
-(The FoodMart database will be familiar to those of you who have used
-the Mondrian OLAP engine, because it is Mondrian's main test data
-set. To load the data set, follow <a
-href="http://mondrian.pentaho.com/documentation/installation.php#2_Set_up_test_data">Mondrian's
-installation instructions</a>.)
-
-<b>Current limitations</b>: The JDBC adapter currently only pushes
-down table scan operations; all other processing (filtering, joins,
-aggregations and so forth) occurs within Calcite. Our goal is to push
-down as much processing as possible to the source system, translating
-syntax, data types and built-in functions as we go. If a Calcite query
-is based on tables from a single JDBC database, in principle the whole
-query should go to that database. If tables are from multiple JDBC
-sources, or a mixture of JDBC and non-JDBC, Calcite will use the most
-efficient distributed query approach that it can.
-
-## The cloning JDBC adapter
-
-The cloning JDBC adapter creates a hybrid database. The data is
-sourced from a JDBC database but is read into in-memory tables the
-first time each table is accessed. Calcite evaluates queries based on
-those in-memory tables, effectively a cache of the database.
-
-For example, the following model reads tables from a MySQL
-"foodmart" database:
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'FOODMART_CLONE',
-  schemas: [
-    {
-      name: 'FOODMART_CLONE',
-      type: 'custom',
-      factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',
-      operand: {
-        jdbcDriver: 'com.mysql.jdbc.Driver',
-        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
-        jdbcUser: 'foodmart',
-        jdbcPassword: 'foodmart'
-      }
-    }
-  ]
-}
-```
-
-Another technique is to build a clone schema on top of an existing
-schema. You use the <code>source</code> property to reference a schema
-defined earlier in the model, like this:
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'FOODMART_CLONE',
-  schemas: [
-    {
-      name: 'FOODMART',
-      type: 'custom',
-      factory: 'org.apache.calcite.adapter.jdbc.JdbcSchema$Factory',
-      operand: {
-        jdbcDriver: 'com.mysql.jdbc.Driver',
-        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
-        jdbcUser: 'foodmart',
-        jdbcPassword: 'foodmart'
-      }
-    },
-    {
-      name: 'FOODMART_CLONE',
-      type: 'custom',
-      factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',
-      operand: {
-        source: 'FOODMART'
-      }
-    }
-  ]
-}
-```
-
-You can use this approach to create a clone schema on any type of
-schema, not just JDBC.
-
-The cloning adapter isn't the be-all and end-all. We plan to develop
-more sophisticated caching strategies, and a more complete and
-efficient implementation of in-memory tables, but for now the cloning
-JDBC adapter shows what is possible and allows us to try out our
-initial implementations.
-
-## Further topics
-
-### Defining a custom schema
-
-(To be written.)
-
-### Modifying data
-
-How to enable DML operations (INSERT, UPDATE and DELETE) on your schema.
-
-(To be written.)
-
-### Calling conventions
-
-(To be written.)
-
-### Statistics and cost
-
-(To be written.)
-
-### Defining and using user-defined functions
-
-(To be written.)
-
-###  Defining tables in a schema
-
-(To be written.)
-
-### Defining custom tables
-
-(To be written.)
-
-### Built-in SQL implementation
-
-How does Calcite implement SQL, if an adapter does not implement all of the core
-relational operators?
-
-(To be written.)
-
-### Table functions
-
-(To be written.)
-
-## Further resources
-
-* <a href="http://calcite.incubator.apache.org">Apache Calcite</a> home
-  page


[08/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/HOWTO.md
----------------------------------------------------------------------
diff --git a/doc/HOWTO.md b/doc/HOWTO.md
deleted file mode 100644
index 65f2ebd..0000000
--- a/doc/HOWTO.md
+++ /dev/null
@@ -1,801 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# Calcite HOWTO
-
-Here's some miscellaneous documentation about using Calcite and its various
-adapters.
-
-## Building from a source distribution
-
-Prerequisites are maven (3.2.1 or later)
-and Java (JDK 1.7 or later, 1.8 preferred) on your path.
-
-Unpack the source distribution `.tar.gz` or `.zip` file,
-`cd` to the root directory of the unpacked source,
-then build using maven:
-
-```bash
-$ tar xvfz calcite-1.2.0-incubating-source.tar.gz
-$ cd calcite-1.2.0-incubating
-$ mvn install
-```
-
-[Running tests](HOWTO.md#running-tests) describes how to run more or fewer
-tests.
-
-## Building from git
-
-Prerequisites are git, maven (3.2.1 or later)
-and Java (JDK 1.7 or later, 1.8 preferred) on your path.
-
-Create a local copy of the github repository,
-`cd` to its root directory,
-then build using maven:
-
-```bash
-$ git clone git://github.com/apache/incubator-calcite.git
-$ cd incubator-calcite
-$ mvn install
-```
-
-[Running tests](HOWTO.md#running-tests) describes how to run more or fewer
-tests.
-
-## Running tests
-
-The test suite will run by default when you build, unless you specify
-`-DskipTests`:
-
-```bash
-$ mvn clean # Note: mvn clean install does not work, use mvn clean && mvn install
-$ mvn -DskipTests install
-```
-
-There are other options that control which tests are run, and in what
-environment, as follows.
-
-* `-Dcalcite.test.db=DB` (where db is `h2`, `hsqldb`, `mysql`, or `postgresql`) allows you
-  to change the JDBC data source for the test suite. Calcite's test
-  suite requires a JDBC data source populated with the foodmart data
-  set.
-   * `hsqldb`, the default, uses an in-memory hsqldb database.
-   * all others access test virtual machine (see [integration tests](HOWTO.md#Running-integration-tests) below)
-     `mysql` and `postgresql` might be somewhat faster than hsqldb, but you need to populate it (i.e. provision a VM).
-* `-Dcalcite.debug` prints extra debugging information to stdout.
-* `-Dcalcite.test.slow` enables tests that take longer to execute. For
-  example, there are tests that create virtual TPC-H and TPC-DS schemas
-  in-memory and run tests from those benchmarks.
-* `-Dcalcite.test.splunk=true` enables tests that run against Splunk.
-  Splunk must be installed and running.
-
-## Running integration tests
-
-For testing Calcite's external adapters, a test virtual machine should be used.
-The VM includes H2, HSQLDB, MySQL, MongoDB, and PostgreSQL.
-
-Test VM requires 5GiB of disk space and it takes 30 minutes to build.
-
-Note: you can use [calcite-test-dataset](https://github.com/vlsi/calcite-test-dataset)
- to populate your own database, however it is recommended to use test VM so the test environment can be reproduced.
-
-### VM preparation
-
-0) Install dependencies: [Vagrant](https://www.vagrantup.com/) and [VirtualBox](https://www.virtualbox.org/)
-
-1) Clone https://github.com/vlsi/calcite-test-dataset.git at the same level as calcite repository.
-For instance:
-```bash
-code
-  +-- calcite
-  +-- calcite-test-dataset
-```
-
-Note: integration tests search for ../calcite-test-dataset or ../../calcite-test-dataset.
- You can specify full path via calcite.test.dataset system property.
-
-2) Build and start the VM:
-```bash
-cd calcite-test-dataset && mvn install
-```
-
-### VM management
-
-Test VM is provisioned by Vagrant, so regular Vagrant `vagrant up` and `vagrant halt` should be used to start and stop the VM.
-The connection strings for different databases are listed in [calcite-test-dataset](https://github.com/vlsi/calcite-test-dataset) readme.
-
-### Suggested test flow
-
-Note: test VM should be started before you launch integration tests. Calcite itself does not start/stop the VM.
-
-Command line:
-* Executing regular unit tests (does not require external data): no change. `mvn test` or `mvn install`.
-* Executing all tests, for all the DBs: `mvn verify -Pit`. `it` stands for "integration-test". `mvn install -Pit` works as well.
-* Executing just tests for external DBs, excluding unit tests: `mvn -Dtest=foo -DfailIfNoTests=false -Pit verify`
-* Executing just MongoDB tests: `cd mongo; mvn verify -Pit`
-
-From within IDE:
-* Executing regular unit tests: no change.
-* Executing MongoDB tests: run `MongoAdapterIT.java` as usual (no additional properties are required)
-* Executing MySQL tests: run `JdbcTest` and `JdbcAdapterTest` with setting `-Dcalcite.test.db=mysql`
-* Executing PostgreSQL tests: run `JdbcTest` and `JdbcAdapterTest` with setting `-Dcalcite.test.db=postgresql`
-
-### Integration tests technical details
-
-Tests with external data are executed at maven's integration-test phase.
-We do not currently use pre-integration-test/post-integration-test, however we could use that in future.
-The verification of build pass/failure is performed at verify phase.
-Integration tests should be named `...IT.java`, so they are not picked up on unit test execution.
-
-## Contributing
-
-We welcome contributions.
-
-If you are planning to make a large contribution, talk to us first! It
-helps to agree on the general approach. Log a
-[JIRA case](https://issues.apache.org/jira/browse/CALCITE) for your
-proposed feature or start a discussion on the dev list.
-
-Fork the github repository, and create a branch for your feature.
-
-Develop your feature and test cases, and make sure that `mvn
-install` succeeds. (Run extra tests if your change warrants it.)
-
-Commit your change to your branch, and use a comment that starts with
-the JIRA case number, like this:
-
-```
-[CALCITE-345] AssertionError in RexToLixTranslator comparing to date literal
-```
-
-If your change had multiple commits, use `git rebase -i master` to
-combine them into a single commit, and to bring your code up to date
-with the latest on the main line.
-
-Then push your commit(s) to github, and create a pull request from
-your branch to the incubator-calcite master branch. Update the JIRA case
-to reference your pull request, and a committer will review your
-changes.
-
-## Getting started
-
-Calcite is a community, so the first step to joining the project is to introduce yourself.
-Join the [developers list](http://mail-archives.apache.org/mod_mbox/incubator-calcite-dev/)
-and send an email.
-
-If you have the chance to attend a [meetup](http://www.meetup.com/Apache-Calcite/),
-or meet [members of the community](http://calcite.incubator.apache.org/team-list.html)
-at a conference, that's also great.
-
-Choose an initial task to work on. It should be something really simple,
-such as a bug fix or a [Jira task that we have labeled
-"newbie"](https://issues.apache.org/jira/issues/?jql=labels%20%3D%20newbie%20%26%20project%20%3D%20Calcite%20%26%20status%20%3D%20Open).
-Follow the [contributing guidelines](#contributing) to get your change committed.
-
-After you have made several useful contributions we may
-[invite you to become a committer](https://community.apache.org/contributors/).
-We value all contributions that help to build a vibrant community, not just code.
-You can contribute by testing the code, helping verify a release,
-writing documentation or the web site,
-or just by answering questions on the list.
-
-## Tracing
-
-To enable tracing, add the following flags to the java command line:
-
-```
--Dcalcite.debug=true -Djava.util.logging.config.file=core/src/test/resources/logging.properties
-```
-
-The first flag causes Calcite to print the Java code it generates
-(to execute queries) to stdout. It is especially useful if you are debugging
-mysterious problems like this:
-
-```
-Exception in thread "main" java.lang.ClassCastException: Integer cannot be cast to Long
-  at Baz$1$1.current(Unknown Source)
-```
-
-The second flag specifies a config file for
-the <a href="http://docs.oracle.com/javase/7/docs/api/java/util/logging/package-summary.html">java.util.logging</a>
-framework. Put the following into core/src/test/resources/logging.properties:
-
-```properties
-handlers= java.util.logging.ConsoleHandler
-.level= INFO
-org.apache.calcite.plan.RelOptPlanner.level=FINER
-java.util.logging.ConsoleHandler.level=ALL
-```
-
-The line `org.apache.calcite.plan.RelOptPlanner.level=FINER` tells the planner to produce
-fairly verbose output. You can modify the file to enable other loggers, or to change levels.
-For instance, if you change `FINER` to `FINEST` the planner will give you an account of the
-planning process so detailed that it might fill up your hard drive.
-
-## CSV adapter
-
-See the <a href="TUTORIAL.md">tutorial</a>.
-
-## MongoDB adapter
-
-First, download and install Calcite,
-and <a href="http://www.mongodb.org/downloads">install MongoDB</a>.
-
-Note: you can use MongoDB from integration test virtual machine above.
-
-Import MongoDB's zipcode data set into MongoDB:
-
-```bash
-$ curl -o /tmp/zips.json http://media.mongodb.org/zips.json
-$ mongoimport --db test --collection zips --file /tmp/zips.json
-Tue Jun  4 16:24:14.190 check 9 29470
-Tue Jun  4 16:24:14.469 imported 29470 objects
-```
-
-Log into MongoDB to check it's there:
-
-```bash
-$ mongo
-MongoDB shell version: 2.4.3
-connecting to: test
-> db.zips.find().limit(3)
-{ "city" : "ACMAR", "loc" : [ -86.51557, 33.584132 ], "pop" : 6055, "state" : "AL", "_id" : "35004" }
-{ "city" : "ADAMSVILLE", "loc" : [ -86.959727, 33.588437 ], "pop" : 10616, "state" : "AL", "_id" : "35005" }
-{ "city" : "ADGER", "loc" : [ -87.167455, 33.434277 ], "pop" : 3205, "state" : "AL", "_id" : "35006" }
-> exit
-bye
-```
-
-Connect using the
-<a href="https://github.com/apache/incubator-calcite/blob/master/mongodb/src/test/resources/mongo-zips-model.json">mongo-zips-model.json</a>
-Calcite model:
-```bash
-$ ./sqlline
-sqlline> !connect jdbc:calcite:model=mongodb/target/test-classes/mongo-zips-model.json admin admin
-Connecting to jdbc:calcite:model=mongodb/target/test-classes/mongo-zips-model.json
-Connected to: Calcite (version 1.x.x)
-Driver: Calcite JDBC Driver (version 1.x.x)
-Autocommit status: true
-Transaction isolation: TRANSACTION_REPEATABLE_READ
-sqlline> !tables
-+------------+--------------+-----------------+---------------+
-| TABLE_CAT  | TABLE_SCHEM  |   TABLE_NAME    |  TABLE_TYPE   |
-+------------+--------------+-----------------+---------------+
-| null       | mongo_raw    | zips            | TABLE         |
-| null       | mongo_raw    | system.indexes  | TABLE         |
-| null       | mongo        | ZIPS            | VIEW          |
-| null       | metadata     | COLUMNS         | SYSTEM_TABLE  |
-| null       | metadata     | TABLES          | SYSTEM_TABLE  |
-+------------+--------------+-----------------+---------------+
-sqlline> select count(*) from zips;
-+---------+
-| EXPR$0  |
-+---------+
-| 29467   |
-+---------+
-1 row selected (0.746 seconds)
-sqlline> !quit
-Closing: org.apache.calcite.jdbc.FactoryJdbc41$CalciteConnectionJdbc41
-$
-```
-
-## Splunk adapter
-
-To run the test suite and sample queries against Splunk,
-load Splunk's `tutorialdata.zip` data set as described in
-<a href="http://docs.splunk.com/Documentation/Splunk/6.0.2/PivotTutorial/GetthetutorialdataintoSplunk">the Splunk tutorial</a>.
-
-(This step is optional, but it provides some interesting data for the sample
-queries. It is also necessary if you intend to run the test suite, using
-`-Dcalcite.test.splunk=true`.)
-
-## Implementing an adapter
-
-New adapters can be created by implementing `CalcitePrepare.Context`:
-
-```java
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteRootSchema;
-
-public class AdapterContext implements CalcitePrepare.Context {
-  @Override
-  public JavaTypeFactory getTypeFactory() {
-    // adapter implementation
-    return typeFactory;
-  }
-
-  @Override
-  public CalciteRootSchema getRootSchema() {
-    // adapter implementation
-    return rootSchema;
-  }
-}
-```
-
-### Testing adapter in Java
-
-The example below shows how SQL query can be submitted to
-`CalcitePrepare` with a custom context (`AdapterContext` in this
-case). Calcite prepares and implements the query execution, using the
-resources provided by the `Context`. `CalcitePrepare.PrepareResult`
-provides access to the underlying enumerable and methods for
-enumeration. The enumerable itself can naturally be some adapter
-specific implementation.
-
-```java
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.prepare.CalcitePrepareImpl;
-import org.junit.Test;
-
-public class AdapterContextTest {
-  @Test
-  public void testSelectAllFromTable() {
-    AdapterContext ctx = new AdapterContext();
-    String sql = "SELECT * FROM TABLENAME";
-    Class elementType = Object[].class;
-    CalcitePrepare.PrepareResult<Object> prepared =
-        new CalcitePrepareImpl().prepareSql(ctx, sql, null, elementType, -1);
-    Object enumerable = prepared.getExecutable();
-    // etc.
-  }
-}
-```
-
-### JavaTypeFactory
-
-When Calcite compares types (instances of `RelDataType`), it requires them to be the same
-object. If there are two distinct type instances that refer to the
-same Java type, Calcite may fail to recognize that they match.  It is
-recommended to:
-* Use a single instance of `JavaTypeFactory` within the calcite context;
-* Store the types so that the same object is always returned for the same type.
-
-## Set up PGP signing keys (for Calcite committers)
-
-Follow instructions at http://www.apache.org/dev/release-signing to
-create a key pair. (On Mac OS X, I did `brew install gpg` and `gpg
---gen-key`.)
-
-Add your public key to the `KEYS` file by following instructions in
-the `KEYS` file.
-
-## Making a snapshot (for Calcite committers)
-
-Before you start:
-* Set up signing keys as described above.
-* Make sure you are using JDK 1.7 (not 1.8).
-* Make sure build and tests succeed with `-Dcalcite.test.db=hsqldb` (the default)
-
-```bash
-# Set passphrase variable without putting it into shell history
-read -s GPG_PASSPHRASE
-
-# Make sure that there are no junk files in the sandbox
-git clean -xn
-mvn clean
-
-mvn -Papache-release -Dgpg.passphrase=${GPG_PASSPHRASE} install
-```
-
-When the dry-run has succeeded, change `install` to `deploy`.
-
-## Making a release (for Calcite committers)
-
-Before you start:
-* Set up signing keys as described above.
-* Make sure you are using JDK 1.7 (not 1.8).
-* Check that `README`, `README.md` and `HOWTO.md` have the correct version number.
-* Make sure build and tests succeed, including with
-  -Dcalcite.test.db={mysql,hsqldb}, -Dcalcite.test.slow=true,
-  -Dcalcite.test.mongodb=true, -Dcalcite.test.splunk=true.
-* Trigger a
-  <a href="https://scan.coverity.com/projects/2966">Coverity scan</a>
-  by merging the latest code into the `julianhyde/coverity_scan` branch,
-  and when it completes, make sure that there are no important issues.
-* Make sure that
-  <a href="https://issues.apache.org/jira/issues/?jql=project%20%3D%20CALCITE%20AND%20status%20%3D%20Resolved%20and%20fixVersion%20is%20null">
-  every "resolved" JIRA case</a> (including duplicates) has
-  a fix version assigned (most likely the version we are
-  just about to release)
-
-Create a release branch named after the release, e.g. `branch-1.1`, and push it to Apache.
-
-```bash
-$ git checkout -b branch-X.Y
-$ git push -u origin branch-X.Y
-
-We will use the branch for the entire the release process. Meanwhile,
-we do not allow commits to the master branch. After the release is
-final, we can use `git merge --ff-only` to append the changes on the
-release branch onto the master branch. (Apache does not allow reverts
-to the master branch, which makes it difficult to clean up the kind of
-messy commits that inevitably happen while you are trying to finalize
-a release.)
-
-Now, set up your environment and do a dry run. The dry run will not
-commit any changes back to git and gives you the opportunity to verify
-that the release process will complete as expected.
-
-If any of the steps fail, clean up (see below), fix the problem, and
-start again from the top.
-
-```bash
-# Set passphrase variable without putting it into shell history
-read -s GPG_PASSPHRASE
-
-# Make sure that there are no junk files in the sandbox
-git clean -xn
-mvn clean
-
-# Do a dry run of the release:prepare step, which sets version numbers.
-mvn -DdryRun=true -DskipTests -DreleaseVersion=X.Y.Z-incubating -DdevelopmentVersion=X.Y.Z+1-incubating-SNAPSHOT -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:prepare 2>&1 | tee /tmp/prepare-dry.log
-```
-
-Check the artifacts:
-* In the `target` directory should be these 8 files, among others:
-  * apache-calcite-X.Y.Z-incubating-src.tar.gz
-  * apache-calcite-X.Y.Z-incubating-src.tar.gz.asc
-  * apache-calcite-X.Y.Z-incubating-src.tar.gz.md5
-  * apache-calcite-X.Y.Z-incubating-src.tar.gz.sha1
-  * apache-calcite-X.Y.Z-incubating-src.zip
-  * apache-calcite-X.Y.Z-incubating-src.zip.asc
-  * apache-calcite-X.Y.Z-incubating-src.zip.md5
-  * apache-calcite-X.Y.Z-incubating-src.zip.sha1
-* Note that the file names start `apache-calcite-` and include
-  `incubating` in the version.
-* In the two source distros `.tar.gz` and `.zip` (currently there is
-  no binary distro), check that all files belong to a directory called
-  `apache-calcite-X.Y.Z-incubating-src`.
-* That directory must contain files `DISCLAIMER`, `NOTICE`, `LICENSE`,
-  `README`, `README.md`
-  * Check that the version in `README` is correct
-* In each .jar (for example
-  `core/target/calcite-core-X.Y.Z-incubating.jar` and
-  `mongodb/target/calcite-mongodb-X.Y.Z-incubating-sources.jar`), check
-  that the `META-INF` directory contains `DEPENDENCIES`, `LICENSE`,
-  `NOTICE` and `git.properties`
-* In each .jar, check that `org-apache-calcite-jdbc.properties` is
-  present and does not contain un-substituted `${...}` variables
-* Check PGP, per https://httpd.apache.org/dev/verification.html
-
-Now, remove the `-DdryRun` flag and run the release for real.
-
-```bash
-# Prepare sets the version numbers, creates a tag, and pushes it to git.
-mvn -DdryRun=false -DskipTests -DreleaseVersion=X.Y.Z-incubating -DdevelopmentVersion=X.Y.Z+1-incubating-SNAPSHOT -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:prepare 2>&1 | tee /tmp/prepare.log
-
-# Perform checks out the tagged version, builds, and deploys to the staging repository
-mvn -DskipTests -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:perform 2>&1 | tee /tmp/perform.log
-```
-
-Verify the staged artifacts in the Nexus repository:
-* Go to https://repository.apache.org/
-* Under `Build Promotion`, click `Staging Repositories`
-* In the `Staging Repositories` tab there should be a line with profile `org.apache.calcite`
-* Navigate through the artifact tree and make sure the .jar, .pom, .asc files are present
-* Check the box on in the first column of the row,
-  and press the 'Close' button to publish the repository at
-  https://repository.apache.org/content/repositories/orgapachecalcite-1000
-  (or a similar URL)
-
-Upload the artifacts via subversion to a staging area,
-https://dist.apache.org/repos/dist/dev/incubator/calcite/apache-calcite-X.Y.Z-incubating-rcN:
-
-```bash
-# Create a subversion workspace, if you haven't already
-mkdir -p ~/dist/dev
-pushd ~/dist/dev
-svn co https://dist.apache.org/repos/dist/dev/incubator/calcite
-popd
-
-# Move the files into a directory
-cd target
-mkdir ~/dist/dev/calcite/apache-calcite-X.Y.Z-incubating-rcN
-mv apache-calcite-* ~/dist/dev/calcite/apache-calcite-X.Y.Z-incubating-rcN
-
-# Check in
-cd ~/dist/dev/calcite
-svn add apache-calcite-X.Y.Z-incubating-rcN
-svn ci
-```
-
-## Cleaning up after a failed release attempt (for Calcite committers)
-
-```
-# Make sure that the tag you are about to generate does not already
-# exist (due to a failed release attempt)
-git tag
-
-# If the tag exists, delete it locally and remotely
-git tag -d apache-calcite-X.Y.Z-incubating
-git push origin :refs/tags/apache-calcite-X.Y.Z-incubating
-
-# Remove modified files
-mvn release:clean
-
-# Check whether there are modified files and if so, go back to the
-# original git commit
-git status
-git reset --hard HEAD
-```
-
-## Validate a release
-
-```bash
-# Check that the signing key (e.g. 2AD3FAE3) is pushed
-gpg --recv-keys key
-
-# Check keys
-curl -O https://dist.apache.org/repos/dist/release/incubator/calcite/KEYS
-
-# Sign/check md5 and sha1 hashes
-# (Assumes your O/S has 'md5' and 'sha1' commands.)
-function checkHash() {
-  cd "$1"
-  for i in *.{zip,pom,gz}; do
-    if [ ! -f $i ]; then
-      continue
-    fi
-    if [ -f $i.md5 ]; then
-      if [ "$(cat $i.md5)" = "$(md5 -q $i)" ]; then
-        echo $i.md5 present and correct
-      else
-        echo $i.md5 does not match
-      fi
-    else
-      md5 -q $i > $i.md5
-      echo $i.md5 created
-    fi
-    if [ -f $i.sha1 ]; then
-      if [ "$(cat $i.sha1)" = "$(sha1 -q $i)" ]; then
-        echo $i.sha1 present and correct
-      else
-        echo $i.sha1 does not match
-      fi
-    else
-      sha1 -q $i > $i.sha1
-      echo $i.sha1 created
-    fi
-  done
-}
-checkHash apache-calcite-X.Y.Z-incubating-rcN
-```
-
-## Get approval for a release via Apache voting process (for Calcite committers)
-
-Release vote on dev list
-
-```
-To: dev@calcite.incubator.apache.org
-Subject: [VOTE] Release apache-calcite-X.Y.Z-incubating (release candidate N)
-
-Hi all,
-
-I have created a build for Apache Calcite X.Y.Z-incubating, release candidate N.
-
-Thanks to everyone who has contributed to this release.
-<Further details about release.> You can read the release notes here:
-https://github.com/apache/incubator-calcite/blob/XXXX/doc/HISTORY.md
-
-The commit to be voted upon:
-http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/NNNNNN
-
-Its hash is XXXX.
-
-The artifacts to be voted on are located here:
-https://dist.apache.org/repos/dist/dev/incubator/calcite/apache-calcite-X.Y.Z-incubating-rcN/
-
-The hashes of the artifacts are as follows:
-src.tar.gz.md5 XXXX
-src.tar.gz.sha1 XXXX
-src.zip.md5 XXXX
-src.zip.sha1 XXXX
-
-A staged Maven repository is available for review at:
-https://repository.apache.org/content/repositories/orgapachecalcite-NNNN
-
-Release artifacts are signed with the following key:
-https://people.apache.org/keys/committer/jhyde.asc
-
-Please vote on releasing this package as Apache Calcite X.Y.Z-incubating.
-
-The vote is open for the next 72 hours and passes if a majority of
-at least three +1 PPMC votes are cast.
-
-[ ] +1 Release this package as Apache Calcite X.Y.Z-incubating
-[ ]  0 I don't feel strongly about it, but I'm okay with the release
-[ ] -1 Do not release this package because...
-
-
-Here is my vote:
-
-+1 (binding)
-
-Julian
-```
-
-After vote finishes, send out the result:
-
-```
-Subject: [RESULT] [VOTE] Release apache-calcite-X.Y.Z-incubating (release candidate N)
-To: dev@calcite.incubator.apache.org
-
-Thanks to everyone who has tested the release candidate and given
-their comments and votes.
-
-The tally is as follows.
-
-N binding +1s:
-<names>
-
-N non-binding +1s:
-<names>
-
-No 0s or -1s.
-
-Therefore I am delighted to announce that the proposal to release
-Apache Calcite X.Y.Z-incubating has passed.
-
-I'll now start a vote on the general list. Those of you in the IPMC,
-please recast your vote on the new thread.
-
-Julian
-```
-
-Use the [Apache URL shortener](http://s.apache.org) to generate
-shortened URLs for the vote proposal and result emails. Examples:
-[s.apache.org/calcite-1.2-vote](http://s.apache.org/calcite-1.2-vote) and
-[s.apache.org/calcite-1.2-result](http://s.apache.org/calcite-1.2-result).
-
-Propose a vote on the incubator list.
-
-```
-To: general@incubator.apache.org
-Subject: [VOTE] Release Apache Calcite X.Y.Z (incubating)
-
-Hi all,
-
-The Calcite community has voted on and approved a proposal to release
-Apache Calcite X.Y.Z (incubating).
-
-Proposal:
-http://s.apache.org/calcite-X.Y.Z-vote
-
-Vote result:
-N binding +1 votes
-N non-binding +1 votes
-No -1 votes
-http://s.apache.org/calcite-X.Y.Z-result
-
-The commit to be voted upon:
-http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/NNNNNN
-
-Its hash is XXXX.
-
-The artifacts to be voted on are located here:
-https://dist.apache.org/repos/dist/dev/incubator/calcite/apache-calcite-X.Y.Z-incubating-rcN/
-
-The hashes of the artifacts are as follows:
-src.tar.gz.md5 XXXX
-src.tar.gz.sha1 XXXX
-src.zip.md5 XXXX
-src.zip.sha1 XXXX
-
-A staged Maven repository is available for review at:
-https://repository.apache.org/content/repositories/orgapachecalcite-NNNN
-
-Release artifacts are signed with the following key:
-https://people.apache.org/keys/committer/jhyde.asc
-
-Pursuant to the Releases section of the Incubation Policy and with
-the endorsement of NNN of our mentors we would now like to request
-the permission of the Incubator PMC to publish the release. The vote
-is open for 72 hours, or until the necessary number of votes (3 +1)
-is reached.
-
-[ ] +1 Release this package as Apache Calcite X.Y.Z incubating
-[ ] -1 Do not release this package because...
-
-Julian Hyde, on behalf of Apache Calcite PPMC
-```
-
-After vote finishes, send out the result:
-
-```
-To: general@incubator.apache.org
-Subject: [RESULT] [VOTE] Release Apache Calcite X.Y.Z (incubating)
-
-This vote passes with N +1s and no 0 or -1 votes:
-+1 <name> (mentor)
-
-There was some feedback during voting. I shall open a separate
-thread to discuss.
-
-Thanks everyone. We’ll now roll the release out to the mirrors.
-
-Julian
-```
-
-## Publishing a release (for Calcite committers)
-
-After a successful release vote, we need to push the release
-out to mirrors, and other tasks.
-
-In JIRA, search for all issues resolved in this release,
-and do a bulk update changing their status to "Closed",
-with a change comment
-"Resolved in release X.Y.Z-incubating (YYYY-MM-DD)"
-(fill in release number and date appropriately).
-
-Promote the staged nexus artifacts.
-* Go to https://repository.apache.org/
-* Under "Build Promotion" click "Staging Repositories"
-* In the line with "orgapachecalcite-xxxx", check the box
-* Press "Release" button
-
-Check the artifacts into svn.
-
-```bash
-# Get the release candidate.
-mkdir -p ~/dist/dev
-cd ~/dist/dev
-svn co https://dist.apache.org/repos/dist/dev/incubator/calcite
-
-# Copy the artifacts. Note that the copy does not have '-rcN' suffix.
-mkdir -p ~/dist/release
-cd ~/dist/release
-svn co https://dist.apache.org/repos/dist/release/incubator/calcite
-cd calcite
-cp -rp ../../dev/calcite/apache-calcite-X.Y.Z-incubating-rcN apache-calcite-X.Y.Z-incubating
-svn add apache-calcite-X.Y.Z-incubating
-
-# Check in.
-svn ci
-```
-
-Svnpubsub will publish to
-https://dist.apache.org/repos/dist/release/incubator/calcite and propagate to
-http://www.apache.org/dyn/closer.cgi/incubator/calcite within 24 hours.
-
-## Publishing the web site (for Calcite committers)
-
-Get the code:
-
-```bash
-$ svn co https://svn.apache.org/repos/asf/incubator/calcite/site calcite-site
-```
-
-(Note: `https:`, not `http:`.)
-
-Build the site:
-
-```bash
-$ cd calcite-site
-$ ./build.sh
-```
-
-It will prompt you to install jekyll, redcarpet and pygments, if you
-do not have them installed. It will also check out the git source code
-repo, so that it can generate javadoc.
-
-Check in:
-
-```bash
-svn ci -m"Commit message" file...
-```
-
-The site will automatically be deployed as http://calcite.incubator.apache.org.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/INCUBATOR_PROPOSAL.md
----------------------------------------------------------------------
diff --git a/doc/INCUBATOR_PROPOSAL.md b/doc/INCUBATOR_PROPOSAL.md
deleted file mode 100644
index 2b07205..0000000
--- a/doc/INCUBATOR_PROPOSAL.md
+++ /dev/null
@@ -1,357 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# Optiq
-
-## Abstract
-
-Optiq is a framework that allows efficient translation of queries
-involving heterogeneous and federated data.
-
-## Proposal
-
-Optiq is a highly customizable engine for parsing and planning queries
-on data in a wide variety of formats. It allows database-like access,
-and in particular a SQL interface and advanced query optimization, for
-data not residing in a traditional database.
-
-## Background
-
-Databases were traditionally engineered in a monolithic stack,
-providing a data storage format, data processing algorithms, query
-parser, query planner, built-in functions, metadata repository and
-connectivity layer. They innovate in some areas but rarely in all.
-
-Modern data management systems are decomposing that stack
-into separate components, separating data, processing engine,
-metadata, and query language support. They are highly heterogeneous,
-with data in multiple locations and formats, caching and redundant
-data, different workloads, and processing occurring in different
-engines.
-
-Query planning (sometimes called query optimization) has always been
-a key function of a DBMS, because it allows the implementors to
-introduce new query-processing algorithms, and allows data
-administrators to re-organize the data without affecting applications
-built on that data. In a componentized system, the query planner
-integrates the components (data formats, engines, algorithms) without
-introducing unncessary coupling or performance tradeoffs.
-
-But building a query planner is hard; many systems muddle along
-without a planner, and indeed a SQL interface, until the demand from
-their customers is overwhelming.
-
-There is an opportunity to make this process more efficient by
-creating a re-usable framework.
-
-## Rationale
-
-Optiq allows database-like access, and in particular a SQL interface
-and advanced query optimization, for data not residing in a
-traditional database. It is complementary to many current Hadoop and
-NoSQL systems, which have innovative and performant storage and
-runtime systems but lack a SQL interface and intelligent query
-translation.
-
-Optiq is already in use by several projects, including Apache Drill,
-Apache Hive and Cascading Lingual, and commercial products.
-
-Optiq's architecture consists of:
-* An extensible relational algebra.
-* SPIs (service-provider interfaces) for metadata (schemas and
-  tables), planner rules, statistics, cost-estimates, user-defined
-  functions.
-* Built-in sets of rules for logical transformations and common data-sources.
-* Two query planning engines driven by rules, statistics, etc. One
-  engine is cost-based, the other rule-based.
-* Optional SQL parser, validator and translator to relational algebra.
-* Optional JDBC driver.
-
-## Initial Goals
-
-The initial goals are be to move the existing codebase to Apache and
-integrate with the Apache development process. Once this is
-accomplished, we plan for incremental development and releases that
-follow the Apache guidelines.
-
-As we move the code into the org.apache namespace, we will restructure
-components as necessary to allow clients to use just the components of
-Optiq that they need.
-
-A version 1.0 release, including pre-built binaries, will foster wider
-adoption.
-
-## Current Status
-
-Optiq has had over a dozen minor releases over the last 18 months. Its
-core SQL parser and validator, and its planning engine and core rules,
-are mature and robust and are the basis for several production
-systems; but other components and SPIs are still undergoing rapid
-evolution.
-
-### Meritocracy
-
-We plan to invest in supporting a meritocracy. We will discuss the
-requirements in an open forum. We encourage the companies and projects
-using Optiq to discuss their requirements in an open forum and to
-participate in development.  We will encourage and monitor community
-participation so that privileges can be extended to those that
-contribute.
-
-Optiq's pluggable architecture encourages developers to contribute
-extensions such as adapters for data sources, new planning rules, and
-better statistics and cost-estimation functions.  We look forward to
-fostering a rich ecosystem of extensions.
-
-### Community
-
-Building a data management system requires a high degree of
-technical skill, and correspondingly, the community of developers
-directly using Optiq is potentially fairly small, albeit highly
-technical and engaged. But we also expect engagement from members of
-the communities of projects that use Optiq, such as Drill and
-Hive. And we intend to structure Optiq so that it can be used for
-lighter weight applications, such as providing a SQL and JDBC
-interface to a NoSQL system.
-
-### Core Developers
-
-The developers on the initial committers list are all experienced open
-source developers, and are actively using Optiq in their projects.
-
-* Julian Hyde is lead developer of Mondrian, an open source OLAP
-  engine, and an Apache Drill committer.
-* Chris Wensel is lead developer of Cascading, and of Lingual, the SQL
-  interface to Cascading built using Optiq.
-* Jacques Nadeau is lead developer of Apache Drill, which uses Optiq.
-
-In addition, there are several regular contributors whom we hope will
-graduate to committers during the incubation process.
-
-We realize that additional employer diversity is needed, and we will
-work aggressively to recruit developers from additional companies.
-
-### Alignment
-
-Apache, and in particular the ecosystem surrounding Hadoop, contains
-several projects for building data management systems that leverage
-each other's capabilities. Optiq is a natural fit for that ecosystem,
-and will help foster projects meeting new challenges.
-
-Optiq is already used by Apache Hive and Apache Drill; Optiq embeds
-Apache Spark as an optional engine; we are in discussion with Apache
-Phoenix about integrating JDBC and query planning.
-
-## Known Risks
-
-### Orphaned Products
-
-Optiq is already a key component in three independent projects, each
-backed by a different company, so the risk of being orphaned is
-relatively low. We plan to mitigate this risk by recruiting additional
-committers, and promoting Optiq's adoption as a framework by other
-projects.
-
-### Inexperience with Open Source
-
-The initial committers are all Apache members, some of whom have
-several years in the Apache Hadoop community. The founder of the
-project, Julian Hyde, has been a founder and key developer in open
-source projects for over ten years.
-
-### Homogenous Developers
-
-The initial committers are employed by a number of companies,
-including Concurrent, Hortonworks, MapR Technologies and Salesforce.com.
-We are committed to recruiting additional committers from outside these
-companies.
-
-### Reliance on Salaried Developers
-
-Like most open source projects, Optiq receives substantial support
-from salaried developers. This is to be expected given that it is a
-highly technical framework. However, they are all passionate about the
-project, and we are confident that the project will continue even if
-no salaried developers contribute to the project. As a framework, the
-project encourages the involvement of members of other projects, and
-of academic researchers. We are committed to recruiting additional
-committers including non-salaried developers.
-
-### Relationships with Other Apache Products
-
-As mentioned in the Alignment section, Optiq is being used by
-<a href="http://hive.apache.org">Apache Hive</a> and
-<a href="http://incubator.apache.org/drill">Apache Drill</a>,
-and has adapters for
-<a href="http://phoenix.incubator.apache.org">Apache Phoenix</a>
-and
-<a href="http://spark.apache.org">Apache Spark</a>.
-Optiq often operates on data in a Hadoop environment, so collaboration
-with other Hadoop projects is desirable and highly likely.
-
-Unsurprisingly there is some overlap in capabilities between Optiq and
-other Apache projects. Several projects that are databases or
-database-like have query-planning capabilities. These include Hive,
-Drill, Phoenix, Spark,
-<a href="http://db.apache.org/derby">Apache Derby</a>,
-<a href="http://pig.apache.org">Apache Pig</a>,
-<a href="http://jena.apache.org">Apache Jena</a> and
-<a href="http://tajo.apache.org">Apache Tajo</a>.
-Optiq’s query planner is extensible at run time, and does
-not have a preferred runtime engine on which to execute compiled
-queries. These capabilities, and the large corpus of pre-built rules,
-are what allow Optiq to be embedded in other projects.
-
-Several other Apache projects access third-party data sources,
-including Hive, Pig, Drill, Spark and
-<a href="http://metamodel.incubator.apache.org">Apache MetaModel</a>.
-Optiq allows users to optimize access to third-party data sources by
-writing rules to push processing down to the data source, and provide
-a cost model to choose the optimal location for processing. That said,
-maintaining a library of adapters is burdensome, and so it would make
-sense to collaborate with other projects on adapter libraries, and
-re-use libraries where possible.
-
-Optiq supports several front ends for submitting queries. The most
-popular is SQL, with driver connectivity via JDBC (and ODBC
-planned). Other Apache projects with a SQL parser include Hive, Spark,
-Phoenix, Derby, Tajo. Drill uses Optiq’s parser and JDBC stack; both
-Phoenix and Drill have expressed interest in collaborating on JDBC and
-ODBC. Optiq’s Linq4j API is similar to the fluent query-builder APIs
-in Spark and MetaModel. Use of a front end is not required; for
-instance, Hive integrates with Optiq by directly building a graph of
-`RelNode` objects.
-
-### An Excessive Fascination with the Apache Brand
-
-Optiq solves a real problem, as evidenced by its take-up by other
-projects. This proposal is not for the purpose of generating
-publicity. Rather, the primary benefits to joining Apache are those
-outlined in the Rationale section.
-
-## Documentation
-
-Additional documentation for Optiq may be found on its github site:
-* Overview - https://github.com/julianhyde/optiq/blob/master/README.md
-* Tutorial - https://github.com/julianhyde/optiq-csv/blob/master/TUTORIAL.md
-* HOWTO - https://github.com/julianhyde/optiq/blob/master/HOWTO.md
-* Reference guide -  https://github.com/julianhyde/optiq/blob/master/REFERENCE.md
-
-Presentation:
-* <a href="https://github.com/julianhyde/share/blob/master/slides/optiq-richrelevance-2013.pdf?raw=true">SQL on Big Data using Optiq</a>
-
-## Initial Source
-
-The initial code codebase resides in three projects, all hosted on github:
-* https://github.com/julianhyde/optiq
-* https://github.com/julianhyde/optiq-csv
-* https://github.com/julianhyde/linq4j
-
-### Source and Intellectual Property Submission Plan
-
-The initial codebase is already distributed under the Apache 2.0
-License. The owners of the IP have indicated willingness to sign the
-SGA.
-
-## External Dependencies
-
-Optiq and Linq4j have the following external dependencies.
-
-* Java 1.6, 1.7 or 1.8
-* Apache Maven, Commons
-* JavaCC (BSD license)
-* Sqlline 1.1.6 (BSD license)
-* Junit 4.11 (EPL)
-* Janino (BSD license)
-* Guava (Apache 2.0 license)
-* Eigenbase-resgen, eigenbase-xom, eigenbase-properties (Apache 2.0 license)
-
-Some of Optiq's adapters (optiq-csv, optiq-mongodb, optiq-spark,
-optiq-splunk) are currently developed alongside core Optiq, and have
-the following additional dependencies:
-
-* Open CSV 2.3 (Apache 2.0 license)
-* Apache Incubator Spark
-* Mongo Java driver (Apache 2.0 license)
-
-Upon acceptance to the incubator, we would begin a thorough analysis
-of all transitive dependencies to verify this information and
-introduce license checking into the build and release process by
-integrating with Apache Rat.
-
-## Cryptography
-
-Optiq will eventually support encryption on the wire. This is not one
-of the initial goals, and we do not expect Optiq to be a controlled
-export item due to the use of encryption.
-
-## Required Resources
-
-### Mailing Lists
-
-* private@optiq.incubator.apache.org
-* dev@optiq.incubator.apache.org (will be migrated from optiq-dev@googlegroups.com)
-* commits@optiq.incubator.apache.org
-
-### Source control
-
-The Optiq team would like to use git for source control, due to our
-current use of git/github. We request a writeable git repo
-git://git.apache.org/incubator-optiq, and mirroring to be set up to
-github through INFRA.
-
-### Issue Tracking
-
-Optiq currently uses the github issue tracking system associated with
-its github repo: https://github.com/julianhyde/optiq/issues. We will
-migrate to the Apache JIRA:
-http://issues.apache.org/jira/browse/OPTIQ.
-
-## Initial Committers
-
-Julian Hyde (jhyde at apache dot org)
-Jacques Nadeau (jacques at apache dot org)
-James R. Taylor (jamestaylor at apache dot org)
-Chris Wensel (cwensel at apache dot org)
-
-## Affiliations
-
-The initial committers are employees of Concurrent, Hortonworks, MapR
-and Salesforce.com.
-
-* Julian Hyde (Hortonworks)
-* Jacques Nadeau (MapR Technologies)
-* James R. Taylor (Salesforce.com)
-* Chris Wensel (Concurrent)
-
-## Sponsors
-
-### Champion
-
-* Ashutosh Chauhan (hashutosh at apache dot org)
-
-### Nominated Mentors
-
-* Ted Dunning (tdunning at apache dot org) – Chief Application Architect at
-  MapR Technologies; committer for Lucene, Mahout and ZooKeeper.
-* Alan Gates (gates at apache dot org) - Architect at Hortonworks;
-  committer for Pig, Hive and others.
-* Steven Noels (stevenn at apache dot org) - Chief Technical Officer at NGDATA;
-  committer for Cocoon and Forrest, mentor for Phoenix.
-
-### Sponsoring Entity
-
-The Apache Incubator.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/MODEL.md
----------------------------------------------------------------------
diff --git a/doc/MODEL.md b/doc/MODEL.md
deleted file mode 100644
index 484a302..0000000
--- a/doc/MODEL.md
+++ /dev/null
@@ -1,409 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# Calcite JSON model reference
-
-## Elements
-
-### Root
-
-```json
-{
-  version: '1.0',
-  defaultSchema: 'mongo',
-  schemas: [ Schema... ]
-}
-```
-
-`version` (required string) must have value `1.0`.
-
-`defaultSchema` (optional string). If specified, it is
-the name (case-sensitive) of a schema defined in this model, and will
-become the default schema for connections to Calcite that use this model.
-
-`schemas` (optional list of <a href="#schema">Schema</a> elements).
-
-### Schema
-
-Occurs within `root.schemas`.
-
-```json
-{
-  name: 'foodmart',
-  path: ['lib'],
-  cache: true,
-  materializations: [ Materialization... ]
-}
-```
-
-`name` (required string) is the name of the schema.
-
-`type` (optional string, default `map`) indicates sub-type. Values are:
-* `map` for <a href="#map-schema">Map Schema</a>
-* `custom` for <a href="#custom-schema">Custom Schema</a>
-* `jdbc` for <a href="#jdbc-schema">JDBC Schema</a>
-
-`path` (optional list) is the SQL path that is used to
-resolve functions used in this schema. If specified it must be a list,
-and each element of the list must be either a string or a list of
-strings. For example,
-
-```json
-  path: [ ['usr', 'lib'], 'lib' ]
-```
-
-declares a path with two elements: the schema '/usr/lib' and the
-schema '/lib'. Most schemas are at the top level, so you can use a
-string.
-
-`materializations` (optional list of
-<a href="#materialization">Materialization</a>) defines the tables
-in this schema that are materializations of queries.
-
-`cache` (optional boolean, default true) tells Calcite whether to
-cache metadata (tables, functions and sub-schemas) generated
-by this schema.
-
-* If `false`, Calcite will go back to the schema each time it needs
-  metadata, for example, each time it needs a list of tables in order to
-  validate a query against the schema.
-
-* If `true`, Calcite will cache the metadata the first time it reads
-  it. This can lead to better performance, especially if name-matching is
-  case-insensitive.
-
-However, it also leads to the problem of cache staleness.
-A particular schema implementation can override the
-`Schema.contentsHaveChangedSince` method to tell Calcite
-when it should consider its cache to be out of date.
-
-Tables, functions and sub-schemas explicitly created in a schema are
-not affected by this caching mechanism. They always appear in the schema
-immediately, and are never flushed.
-
-### Map Schema
-
-Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
-
-```json
-{
-  name: 'foodmart',
-  type: 'map',
-  tables: [ Table... ],
-  functions: [ Function... ]
-}
-```
-
-`name`, `type`, `path`, `cache`, `materializations` inherited from
-<a href="#schema">Schema</a>.
-
-`tables` (optional list of <a href="#table">Table</a> elements)
-defines the tables in this schema.
-
-`functions` (optional list of <a href="#function">Function</a> elements)
-defines the functions in this schema.
-
-### Custom Schema
-
-Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
-
-```json
-{
-  name: 'mongo',
-  type: 'custom',
-  factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',
-  operand: {
-    host: 'localhost',
-    database: 'test'
-  }
-}
-```
-
-`name`, `type`, `path`, `cache`, `materializations` inherited from
-<a href="#schema">Schema</a>.
-
-`factory` (required string) is the name of the factory class for this
-schema. Must implement interface `org.apache.calcite.schema.SchemaFactory`
-and have a public default constructor.
-
-`operand` (optional map) contains attributes to be passed to the
-factory.
-
-### JDBC Schema
-
-Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
-
-```json
-{
-  name: 'foodmart',
-  type: 'jdbc',
-  jdbcDriver: TODO,
-  jdbcUrl: TODO,
-  jdbcUser: TODO,
-  jdbcPassword: TODO,
-  jdbcCatalog: TODO,
-  jdbcSchema: TODO
-}
-```
-
-`name`, `type`, `path`, `cache`, `materializations` inherited from
-<a href="#schema">Schema</a>.
-
-`jdbcDriver` (optional string) is the name of the JDBC driver class. It not
-specified, uses whichever class the JDBC DriverManager chooses.
-
-`jdbcUrl` (optional string) is the JDBC connect string, for example
-"jdbc:mysql://localhost/foodmart".
-
-`jdbcUser` (optional string) is the JDBC user name.
-
-`jdbcPassword` (optional string) is the JDBC password.
-
-`jdbcCatalog` (optional string) is the name of the initial catalog in the JDBC
-data source.
-
-`jdbcSchema` (optional string) is the name of the initial schema in the JDBC
-data source.
-
-### Materialization
-
-Occurs within `root.schemas.materializations`.
-
-```json
-{
-  view: 'V',
-  table: 'T',
-  sql: 'select deptno, count(*) as c, sum(sal) as s from emp group by deptno'
-}
-```
-
-`view` (optional string) TODO
-
-`table` (optional string) TODO
-
-`sql` (optional string, or list of strings that will be concatenated as a
- multi-line string) is the SQL definition of the materialization.
-
-### Table
-
-Occurs within `root.schemas.tables`.
-
-```json
-{
-  name: 'sales_fact',
-  columns: [ Column... ]
-}
-```
-
-`name` (required string) is the name of this table. Must be unique within the schema.
-
-`type` (optional string, default `custom`) indicates sub-type. Values are:
-* `custom` for <a href="#custom-table">Custom Table</a>
-* `view` for <a href="#view">View</a>
-
-`columns` (optional list of <a href="#column">Column</a> elements)
-
-### View
-
-Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
-
-```json
-{
-  name: 'female_emps',
-  type: 'view',
-  sql: "select * from emps where gender = 'F'"
-}
-```
-
-`name`, `type`, `columns` inherited from <a href="#table">Table</a>.
-
-`sql` (required string, or list of strings that will be concatenated as a
- multi-line string) is the SQL definition of the view.
-
-`path` (optional list) is the SQL path to resolve the query. If not
-specified, defaults to the current schema.
-
-### Custom Table
-
-Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
-
-```json
-{
-  name: 'female_emps',
-  type: 'custom',
-  factory: 'TODO',
-  operand: {
-    todo: 'TODO'
-  }
-}
-```
-
-`name`, `type`, `columns` inherited from <a href="#table">Table</a>.
-
-`factory` (required string) is the name of the factory class for this
-table. Must implement interface `org.apache.calcite.schema.TableFactory`
-and have a public default constructor.
-
-`operand` (optional map) contains attributes to be passed to the
-factory.
-
-### Column
-
-Occurs within `root.schemas.tables.columns`.
-
-```json
-{
-  name: 'empno'
-}
-```
-
-`name` (required string) is the name of this column.
-
-### Function
-
-Occurs within `root.schemas.functions`.
-
-```json
-{
-  name: 'MY_PLUS',
-  className: 'com.example.functions.MyPlusFunction',
-  methodName: 'apply',
-  path: []
-}
-```
-
-`name` (required string) is the name of this function.
-
-`className` (required string) is the name of the class that implements this
-function.
-
-`methodName` (optional string) is the name of the method that implements this
-function.
-
-`path` (optional list of string) is the path for resolving this function.
-
-### Lattice
-
-Occurs within `root.schemas.lattices`.
-
-```json
-{
-  name: 'star',
-  sql: [
-    'select 1 from "foodmart"."sales_fact_1997" as "s"',
-    'join "foodmart"."product" as "p" using ("product_id")',
-    'join "foodmart"."time_by_day" as "t" using ("time_id")',
-    'join "foodmart"."product_class" as "pc" on "p"."product_class_id" = "pc"."product_class_id"'
-  ],
-  auto: false,
-  algorithm: true,
-  algorithmMaxMillis: 10000,
-  rowCountEstimate: 86837,
-  defaultMeasures: [ {
-    agg: 'count'
-  } ],
-  tiles: [ {
-    dimensions: [ 'the_year', ['t', 'quarter'] ],
-    measures: [ {
-      agg: 'sum',
-      args: 'unit_sales'
-    }, {
-      agg: 'sum',
-      args: 'store_sales'
-    }, {
-      agg: 'count'
-    } ]
-  } ]
-}
-```
-
-`name` (required string) is the name of this lattice.
-
-`sql` (required string, or list of strings that will be concatenated as a
-multi-line string) is the SQL statement that defines the fact table, dimension
-tables, and join paths for this lattice.
-
-`auto` (optional boolean, default true) is whether to materialize tiles on need
-as queries are executed.
-
-`algorithm` (optional boolean, default false) is whether to use an optimization
-algorithm to suggest and populate an initial set of tiles.
-
-`algorithmMaxMillis` (optional long, default -1, meaning no limit) is the
-maximum number of milliseconds for which to run the algorithm. After this point,
-takes the best result the algorithm has come up with so far.
-
-`rowCountEstimate` (optional double, default 1000.0) estimated number of rows in
-the star
-
-`tiles` (optional list of <a href="#tile">Tile</a> elements) is a list of
-materialized aggregates to create up front.
-
-`defaultMeasures`  (optional list of <a href="#measure">Measure</a> elements)
-is a list of measures that a tile should have by default.
-Any tile defined in `tiles` can still define its own measures, including
-measures not on this list. If not specified, the default list of measures is
-just 'count(*)':
-
-```json
-[ { name: 'count' } ]
-```
-
-### Tile
-
-Occurs within `root.schemas.lattices.tiles`.
-
-```json
-{
-  dimensions: [ 'the_year', ['t', 'quarter'] ],
-  measures: [ {
-    agg: 'sum',
-    args: 'unit_sales'
-  }, {
-    agg: 'sum',
-    args: 'store_sales'
-  }, {
-    agg: 'count'
-  } ]
-}
-```
-
-`dimensions` is a list of dimensions (columns from the star), like a `GROUP BY`
-clause. Each element is either a string (the unique label of the column within
-the star) or a string list (a column name qualified by a table name).
-
-`measures` (optional list of <a href="#measure">Measure</a> elements) is a list
-of aggregate functions applied to arguments. If not specified, uses the
-lattice's default measure list.
-
-### Measure
-
-Occurs within `root.schemas.lattices.defaultMeasures`
-and `root.schemas.lattices.tiles.measures`.
-
-```json
-{
-  agg: 'sum',
-  args: [ 'unit_sales' ]
-}
-```
-
-`agg` is the name of an aggregate function (usually 'count', 'sum', 'min',
-'max').
-
-`args` (optional) is a column label (string), or list of zero or more columns.
-If a list, each element is either a string (the unique label of the column
-within the star) or a string list (a column name qualified by a table name).

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/REFERENCE.md
----------------------------------------------------------------------
diff --git a/doc/REFERENCE.md b/doc/REFERENCE.md
deleted file mode 100644
index 03f66e3..0000000
--- a/doc/REFERENCE.md
+++ /dev/null
@@ -1,569 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# Calcite SQL language reference
-
-## SQL constructs
-
-```SQL
-statement:
-      setStatement
-  |   explain
-  |   insert
-  |   update
-  |   merge
-  |   delete
-  |   query
-
-setStatement:
-      ALTER ( SYSTEM | SESSION ) SET identifier = expression
-
-explain:
-      EXPLAIN PLAN
-      [ WITH TYPE | WITH IMPLEMENTATION | WITHOUT IMPLEMENTATION ]
-      [ EXCLUDING ATTRIBUTES | INCLUDING [ ALL ] ATTRIBUTES ]
-      FOR ( insert | update | merge | delete | query )
-
-insert:
-      ( INSERT | UPSERT ) INTO tablePrimary
-      [ '(' column [, column ]* ')' ]
-      query
-
-update:
-      UPDATE tablePrimary
-      SET assign [, assign ]*
-      [ WHERE booleanExpression ]
-
-assign:
-      identifier '=' expression
-
-merge:
-      MERGE INTO tablePrimary [ [ AS ] alias ]
-      USING tablePrimary
-      ON booleanExpression
-      [ WHEN MATCHED THEN UPDATE SET assign [, assign ]* ]
-      [ WHEN NOT MATCHED THEN INSERT VALUES '(' value [ , value ]* ')' ]
-
-delete:
-      DELETE FROM tablePrimary [ [ AS ] alias ]
-      [ WHERE booleanExpression ]
-
-query:
-      [ WITH withItem [ , withItem ]* query ]
-  |   {
-          select
-      |   query UNION [ ALL ] query
-      |   query EXCEPT query
-      |   query INTERSECT query
-      }
-      [ ORDER BY orderItem [, orderItem ]* ]
-      [ LIMIT { count | ALL } ]
-      [ OFFSET start { ROW | ROWS } ]
-      [ FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } ]
-
-withItem:
-      name
-      [ '(' column [, column ]* ')' ]
-      AS '(' query ')'
-
-orderItem:
-      expression [ ASC | DESC ] [ NULLS FIRST | NULLS LAST ]
-
-select:
-      SELECT [ STREAM ] [ ALL | DISTINCT ]
-          { * | projectItem [, projectItem ]* }
-      FROM tableExpression
-      [ WHERE booleanExpression ]
-      [ GROUP BY { groupItem [, groupItem ]* } ]
-      [ HAVING booleanExpression ]
-      [ WINDOW windowName AS windowSpec [, windowName AS windowSpec ]* ]
-
-projectItem:
-      expression [ [ AS ] columnAlias ]
-  |   tableAlias . *
-
-tableExpression:
-      tableReference [, tableReference ]*
-  |   tableExpression [ NATURAL ] [ LEFT | RIGHT | FULL ] JOIN tableExpression [ joinCondition ]
-
-joinCondition:
-      ON booleanExpression
-  |   USING '(' column [, column ]* ')'
-
-tableReference:
-      [ LATERAL ]
-      tablePrimary
-      [ [ AS ] alias [ '(' columnAlias [, columnAlias ]* ')' ] ]
-
-tablePrimary:
-      [ TABLE ] [ [ catalogName . ] schemaName . ] tableName
-  |   '(' query ')'
-  |   values
-  |   UNNEST '(' expression ')'
-  |   '(' TABLE expression ')'
-
-values:
-      VALUES expression [, expression ]*
-
-groupItem:
-      expression
-  |   '(' ')'
-  |   '(' expression [, expression ]* ')'
-  |   CUBE '(' expression [, expression ]* ')'
-  |   ROLLUP '(' expression [, expression ]* ')'
-  |   GROUPING SETS '(' groupItem [, groupItem ]* ')'
-
-windowRef:
-      windowName
-  |   windowSpec
-
-windowSpec:
-      [ windowName ]
-      '('
-      [ ORDER BY orderItem [, orderItem ]* ]
-      [ PARTITION BY expression [, expression ]* ]
-      [
-          RANGE numericOrIntervalExpression { PRECEDING | FOLLOWING }
-      |   ROWS numericExpression { PRECEDING | FOLLOWING }
-      ]
-      ')'
-```
-
-In *merge*, at least one of the WHEN MATCHED and WHEN NOT MATCHED clauses must
-be present.
-
-In *orderItem*, if *expression* is a positive integer *n*, it denotes
-the <em>n</em>th item in the SELECT clause.
-
-An aggregate query is a query that contains a GROUP BY or a HAVING
-clause, or aggregate functions in the SELECT clause. In the SELECT,
-HAVING and ORDER BY clauses of an aggregate query, all expressions
-must be constant within the current group (that is, grouping constants
-as defined by the GROUP BY clause, or constants), or aggregate
-functions, or a combination of constants and aggregate
-functions. Aggregate and grouping functions may only appear in an
-aggregate query, and only in a SELECT, HAVING or ORDER BY clause.
-
-A scalar sub-query is a sub-query used as an expression. It can occur
-in most places where an expression can occur (such as the SELECT
-clause, WHERE clause, or as an argument to an aggregate
-function). If the sub-query returns no rows, the value is NULL; if it
-returns more than one row, it is an error.
-
-A sub-query can occur in the FROM clause of a query and also in IN
-and EXISTS expressions.  A sub-query that occurs in IN and
-EXISTS expressions may be correlated; that is, refer to tables in
-the FROM clause of an enclosing query.
-
-## Identifiers
-
-Identifiers are the names of tables, columns and other metadata
-elements used in a SQL query.
-
-Unquoted identifiers, such as emp, must start with a letter and can
-only contain letters, digits, and underscores. They are implicitly
-converted to upper case.
-
-Quoted identifiers, such as "Employee Name", start and end with
-double quotes.  They may contain virtually any character, including
-spaces and other punctuation.  If you wish to include a double quote
-in an identifier, use another double quote to escape it, like this:
-"An employee called ""Fred""."
-
-In Calcite, matching identifiers to the name of the referenced object is
-case-sensitive.  But remember that unquoted identifiers are implicitly
-converted to upper case before matching, and if the object it refers
-to was created using an unquoted identifier for its name, then its
-name will have been converted to upper case also.
-
-## Data types
-
-### Scalar types
-
-| Data type   | Description               | Range and examples   |
-| ----------- | ------------------------- | ---------------------|
-| BOOLEAN     | Logical values            | Values: TRUE, FALSE, UNKNOWN
-| TINYINT     | 1 byte signed integer     | Range is -255 to 256
-| SMALLINT    | 2 byte signed integer     | Range is -32768 to 32767
-| INTEGER, INT | 4 byte signed integer    | Range is -2147483648 to 2147483647
-| BIGINT      | 8 byte signed integer     | Range is -9223372036854775808 to 9223372036854775807
-| DECIMAL(p, s) | Fixed point             | Example: 123.45 is a DECIMAL(5, 2) value.
-| NUMERIC     | Fixed point               |
-| REAL, FLOAT | 4 byte floating point     | 6 decimal digits precision
-| DOUBLE      | 8 byte floating point     | 15 decimal digits precision
-| CHAR(n), CHARACTER(n) | Fixed-width character string | 'Hello', '' (empty string), _latin1'Hello', n'Hello', _UTF16'Hello', 'Hello' 'there' (literal split into multiple parts)
-| VARCHAR(n), CHARACTER VARYING(n) | Variable-length character string | As CHAR(n)
-| BINARY(n)   | Fixed-width binary string | x'45F0AB', x'' (empty binary string), x'AB' 'CD' (multi-part binary string literal)
-| VARBINARY(n), BINARY VARYING(n) | Variable-length binary string | As BINARY(n)
-| DATE        | Date                      | Example: DATE '1969-07-20'
-| TIME        | Time of day               | Example: TIME '20:17:40'
-| TIMESTAMP [ WITHOUT TIME ZONE ] | Date and time | Example: TIMESTAMP '1969-07-20 20:17:40'
-| TIMESTAMP WITH TIME ZONE | Date and time with time zone | Example: TIMESTAMP '1969-07-20 20:17:40 America/Los Angeles'
-| INTERVAL timeUnit [ TO timeUnit ] | Date time interval | Examples: INTERVAL '1:5' YEAR TO MONTH, INTERVAL '45' DAY
-| Anchored interval | Date time interval  | Example: (DATE '1969-07-20', DATE '1972-08-29')
-
-Where:
-```SQL
-timeUnit:
-  YEAR | MONTH | DAY | HOUR | MINUTE | SECOND
-```
-
-Note:
-* DATE, TIME and TIMESTAMP have no time zone. There is not even an implicit
-  time zone, such as UTC (as in Java) or the local time zone. It is left to
-  the user or application to supply a time zone.
-
-### Non-scalar types
-
-| Type     | Description
-| -------- | -----------------------------------------------------------
-| ANY      | A value of an unknown type
-| ROW      | Row with 1 or more columns
-| MAP      | Collection of keys mapped to values
-| MULTISET | Unordered collection that may contain duplicates
-| ARRAY    | Ordered, contiguous collection that may contain duplicates
-| CURSOR   | Cursor over the result of executing a query
-
-## Operators and functions
-
-### Comparison operators
-
-| Operator syntax                                   | Description
-| ------------------------------------------------- | -----------
-| value1 = value2                                   | Equals
-| value1 <> value2                                  | Not equal
-| value1 > value2                                   | Greater than
-| value1 >= value2                                  | Greater than or equal
-| value1 < value2                                   | Less than
-| value1 <= value2                                  | Less than or equal
-| value IS NULL                                     | Whether *value* is null
-| value IS NOT NULL                                 | Whether *value* is not null
-| value1 IS DISTINCT FROM value2                    | Whether two values are not equal, treating null values as the same
-| value1 IS NOT DISTINCT FROM value2                | Whether two values are equal, treating null values as the same
-| value1 BETWEEN value2 AND value3                  | Whether *value1* is greater than or equal to *value2* and less than or equal to *value3*
-| value1 NOT BETWEEN value2 AND value3              | Whether *value1* is less than *value2* or greater than *value3*
-| string1 LIKE string2 [ ESCAPE string3 ]           | Whether *string1* matches pattern *string2*
-| string1 NOT LIKE string2 [ ESCAPE string3 ]       | Whether *string1* does not match pattern *string2*
-| string1 SIMILAR TO string2 [ ESCAPE string3 ]     | Whether *string1* matches regular expression *string2*
-| string1 NOT SIMILAR TO string2 [ ESCAPE string3 ] | Whether *string1* does not match regular expression *string2*
-| value IN (value [, value]* )                      | Whether *value* is equal to a value in a list
-| value NOT IN (value [, value]* )                  | Whether *value* is not equal to every value in a list
-| value IN (sub-query)                              | Whether *value* is equal to a row returned by *sub-query*
-| value NOT IN (sub-query)                          | Whether *value* is not equal to every row returned by *sub-query*
-| EXISTS (sub-query)                                | Whether *sub-query* returns at least one row
-
-### Logical operators
-
-| Operator syntax        | Description
-| ---------------------- | -----------
-| boolean1 OR boolean2   | Whether *boolean1* is TRUE or *boolean2* is TRUE
-| boolean1 AND boolean2  | Whether *boolean1* and *boolean2* are both TRUE
-| NOT boolean            | Whether *boolean* is not TRUE; returns UNKNOWN if *boolean* is UNKNOWN
-| boolean IS FALSE       | Whether *boolean* is FALSE; returns FALSE if *boolean* is UNKNOWN
-| boolean IS NOT FALSE   | Whether *boolean* is not FALSE; returns TRUE if *boolean* is UNKNOWN
-| boolean IS TRUE        | Whether *boolean* is TRUE; returns FALSE if *boolean* is UNKNOWN
-| boolean IS NOT TRUE    | Whether *boolean* is not TRUE; returns TRUE if *boolean* is UNKNOWN
-| boolean IS UNKNOWN     | Whether *boolean* is UNKNOWN
-| boolean IS NOT UNKNOWN | Whether *boolean* is not UNKNOWN
-
-### Arithmetic operators and functions
-
-| Operator syntax           | Description
-| ------------------------- | -----------
-| + numeric                 | Returns *numeric*
-| - numeric                 | Returns negative *numeric*
-| numeric1 + numeric2       | Returns *numeric1* plus *numeric2*
-| numeric1 - numeric2       | Returns *numeric1* minus *numeric2*
-| numeric1 * numeric2       | Returns *numeric1* multiplied by *numeric2*
-| numeric1 / numeric2       | Returns *numeric1* divided by *numeric2*
-| POWER(numeric1, numeric2) | Returns *numeric1* raised to the power of *numeric2*
-| ABS(numeric)              | Returns the absolute value of *numeric*
-| MOD(numeric, numeric)     | Returns the remainder (modulus) of *numeric1* divided by *numeric2*. The result is negative only if *numeric1* is negative
-| SQRT(numeric)             | Returns the square root of *numeric*
-| LN(numeric)               | Returns the natural logarithm (base *e*) of *numeric*
-| LOG10(numeric)            | Returns the base 10 logarithm of *numeric*
-| EXP(numeric)              | Returns *e* raised to the power of *numeric*
-| CEIL(numeric)             | Rounds *numeric* up, and returns the smallest number that is greater than or equal to *numeric*
-| FLOOR(numeric)            | Rounds *numeric* down, and returns the largest number that is less than or equal to *numeric*
-
-### Character string operators and functions
-
-| Operator syntax            | Description
-| -------------------------- | -----------
-| string &#124;&#124; string | Concatenates two character strings.
-| CHAR_LENGTH(string)        | Returns the number of characters in a character string
-| CHARACTER_LENGTH(string)   | As CHAR_LENGTH(*string*)
-| UPPER(string)              | Returns a character string converted to upper case
-| LOWER(string)              | Returns a character string converted to lower case
-| POSITION(string1 IN string2) | Returns the position of the first occurrence of *string1* in *string2*
-| TRIM( { BOTH ;&#124; LEADING ;&#124; TRAILING } string1 FROM string2) | Removes the longest string containing only the characters in *string1* from the start/end/both ends of *string1*
-| OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ]) | Replaces a substring of *string1* with *string2*
-| SUBSTRING(string FROM integer)  | Returns a substring of a character string starting at a given point.
-| SUBSTRING(string FROM integer FOR integer) | Returns a substring of a character string starting at a given point with a given length.
-| INITCAP(string)            | Returns *string* with the first letter of each word converter to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters.
-
-Not implemented:
-* SUBSTRING(string FROM regexp FOR regexp)
-
-### Binary string operators and functions
-
-| Operator syntax | Description
-| --------------- | -----------
-| binary &#124;&#124; binary | Concatenates two binary strings.
-| POSITION(binary1 IN binary2) | Returns the position of the first occurrence of *binary1* in *binary2*
-| OVERLAY(binary1 PLACING binary2 FROM integer [ FOR integer2 ]) | Replaces a substring of *binary1* with *binary2*
-| SUBSTRING(binary FROM integer) | Returns a substring of *binary* starting at a given point
-| SUBSTRING(binary FROM integer FOR integer) | Returns a substring of *binary* starting at a given point with a given length
-
-### Date/time functions
-
-| Operator syntax           | Description
-| ------------------------- | -----------
-| LOCALTIME                 | Returns the current date and time in the session time zone in a value of datatype TIME
-| LOCALTIME(precision)      | Returns the current date and time in the session time zone in a value of datatype TIME, with *precision* digits of precision
-| LOCALTIMESTAMP            | Returns the current date and time in the session time zone in a value of datatype TIMESTAMP
-| LOCALTIMESTAMP(precision) | Returns the current date and time in the session time zone in a value of datatype TIMESTAMP, with *precision* digits of precision
-| CURRENT_TIME              | Returns the current time in the session time zone, in a value of datatype TIMESTAMP WITH TIME ZONE
-| CURRENT_DATE              | Returns the current date in the session time zone, in a value of datatype DATE
-| CURRENT_TIMESTAMP         | Returns the current date and time in the session time zone, in a value of datatype TIMESTAMP WITH TIME ZONE
-| EXTRACT(timeUnit FROM datetime) | Extracts and returns the value of a specified datetime field from a datetime value expression
-| FLOOR(datetime TO timeUnit) | Rounds *datetime* down to *timeUnit*
-| CEIL(datetime TO timeUnit) | Rounds *datetime* up to *timeUnit*
-
-Not implemented:
-* EXTRACT(timeUnit FROM interval)
-* CEIL(interval)
-* FLOOR(interval)
-* datetime - datetime timeUnit [ TO timeUnit ]
-* interval OVERLAPS interval
-* + interval
-* - interval
-* interval + interval
-* interval - interval
-* interval / interval
-* datetime + interval
-* datetime - interval
-
-### System functions
-
-| Operator syntax | Description
-| --------------- | -----------
-| USER            | Equivalent to CURRENT_USER
-| CURRENT_USER    | User name of current execution context
-| SESSION_USER    | Session user name
-| SYSTEM_USER     | Returns the name of the current data store user as identified by the operating system
-| CURRENT_PATH    | Returns a character string representing the current lookup scope for references to user-defined routines and types
-| CURRENT_ROLE    | Returns the current active role
-
-### Conditional functions and operators
-
-| Operator syntax | Description
-| --------------- | -----------
-| CASE value<br/>WHEN value1 [, value11 ]* THEN result1<br/>[ WHEN valueN [, valueN1 ]* THEN resultN ]*<br/>[ ELSE resultZ ]<br/> END | Simple case
-| CASE<br/>WHEN condition1 THEN result1<br/>[ WHEN conditionN THEN resultN ]*<br/>[ ELSE resultZ ]<br/>END | Searched case
-| NULLIF(value, value) | Returns NULL if the values are the same. For example, <code>NULLIF(5, 5)</code> returns NULL; <code>NULLIF(5, 0)</code> returns 5.
-| COALESCE(value, value [, value]* ) | Provides a value if the first value is null. For example, <code>COALESCE(NULL, 5)</code> returns 5.
-
-### Type conversion
-
-| Operator syntax | Description
-| --------------- | -----------
-| CAST(value AS type) | Converts a value to a given type.
-
-### Value constructors
-
-| Operator syntax | Description
-| --------------- | -----------
-| ROW (value [, value]* ) | Creates a row from a list of values.
-| (value [, value]* )     | Creates a row from a list of values.
-| map [ key ]     | Returns the element of a map with a particular key.
-| array [ index ] | Returns the element at a particular location in an array.
-| ARRAY [ value [, value ]* ] | Creates an array from a list of values.
-| MAP [ key, value [, key, value ]* ] | Creates a map from a list of key-value pairs.
-
-### Collection functions
-
-| Operator syntax | Description
-| --------------- | -----------
-| ELEMENT(value)  | Returns the sole element of a array or multiset; null if the collection is empty; throws if it has more than one element.
-| CARDINALITY(value) | Returns the number of elements in an array or multiset.
-
-See also: UNNEST relational operator converts a collection to a relation.
-
-### JDBC function escape
-
-#### Numeric
-
-| Operator syntax                | Description
-| ------------------------------ | -----------
-| {fn LOG10(numeric)}            | Returns the base-10 logarithm of *numeric*
-| {fn POWER(numeric1, numeric2)} | Returns *numeric1* raised to the power of *numeric2*
-
-Not implemented:
-* {fn ABS(numeric)} - Returns the absolute value of *numeric*
-* {fn ACOS(numeric)} - Returns the arc cosine of *numeric*
-* {fn ASIN(numeric)} - Returns the arc sine of *numeric*
-* {fn ATAN(numeric)} - Returns the arc tangent of *numeric*
-* {fn ATAN2(numeric, numeric)}
-* {fn CEILING(numeric)} - Rounds *numeric* up, and returns the smallest number that is greater than or equal to *numeric*
-* {fn COS(numeric)} - Returns the cosine of *numeric*
-* {fn COT(numeric)}
-* {fn DEGREES(numeric)} - Converts *numeric* from radians to degrees
-* {fn EXP(numeric)} - Returns *e* raised to the power of *numeric*
-* {fn FLOOR(numeric)} - Rounds *numeric* down, and returns the largest number that is less than or equal to *numeric*
-* {fn LOG(numeric)} - Returns the natural logarithm (base *e*) of *numeric*
-* {fn MOD(numeric1, numeric2)} - Returns the remainder (modulus) of *numeric1* divided by *numeric2*. The result is negative only if *numeric1* is negative
-* {fn PI()} - Returns a value that is closer than any other value to *pi*
-* {fn RADIANS(numeric)} - Converts *numeric* from degrees to radians
-* {fn RAND(numeric)}
-* {fn ROUND(numeric, numeric)}
-* {fn SIGN(numeric)}
-* {fn SIN(numeric)} - Returns the sine of *numeric*
-* {fn SQRT(numeric)} - Returns the square root of *numeric*
-* {fn TAN(numeric)} - Returns the tangent of *numeric*
-* {fn TRUNCATE(numeric, numeric)}
-
-#### String
-
-| Operator syntax | Description
-| --------------- | -----------
-| {fn LOCATE(string1, string2)} | Returns the position in *string2* of the first occurrence of *string1*. Searches from the beginning of the second CharacterExpression, unless the startIndex parameter is specified.
-| {fn INSERT(string1, start, length, string2)} | Inserts *string2* into a slot in *string1*
-| {fn LCASE(string)}            | Returns a string in which all alphabetic characters in *string* have been converted to lower case
-
-Not implemented:
-* {fn ASCII(string)} - Convert a single-character string to the corresponding ASCII code, an integer between 0 and 255
-* {fn CHAR(string)}
-* {fn CONCAT(character, character)} - Returns the concatenation of character strings
-* {fn DIFFERENCE(string, string)}
-* {fn LEFT(string, integer)}
-* {fn LENGTH(string)}
-* {fn LOCATE(string1, string2 [, integer])} - Returns the position in *string2* of the first occurrence of *string1*. Searches from the beginning of *string2*, unless *integer* is specified.
-* {fn LTRIM(string)}
-* {fn REPEAT(string, integer)}
-* {fn REPLACE(string, string, string)}
-* {fn RIGHT(string, integer)}
-* {fn RTRIM(string)}
-* {fn SOUNDEX(string)}
-* {fn SPACE(integer)}
-* {fn SUBSTRING(string, integer, integer)}
-* {fn UCASE(string)} - Returns a string in which all alphabetic characters in *string* have been converted to upper case
-
-#### Date/time
-
-Not implemented:
-* {fn CURDATE()}
-* {fn CURTIME()}
-* {fn DAYNAME(date)}
-* {fn DAYOFMONTH(date)}
-* {fn DAYOFWEEK(date)}
-* {fn DAYOFYEAR(date)}
-* {fn HOUR(time)}
-* {fn MINUTE(time)}
-* {fn MONTH(date)}
-* {fn MONTHNAME(date)}
-* {fn NOW()}
-* {fn QUARTER(date)}
-* {fn SECOND(time)}
-* {fn TIMESTAMPADD(interval, count, timestamp)}
-* {fn TIMESTAMPDIFF(interval, timestamp, timestamp)}
-* {fn WEEK(date)}
-* {fn YEAR(date)}
-
-#### System
-
-Not implemented:
-* {fn DATABASE()}
-* {fn IFNULL(value, value)}
-* {fn USER(value, value)}
-* {fn CONVERT(value, type)}
-
-### Aggregate functions
-
-Syntax:
-
-```SQL
-aggregateCall:
-        agg( [ DISTINCT ] value [, value]* ) [ FILTER ( WHERE condition ) ]
-    |   agg(*) [ FILTER ( WHERE condition ) ]
-```
-
-If `FILTER` is present, the aggregate function only considers rows for which
-*condition* evaluates to TRUE.
-
-If `DISTINCT` is present, duplicate argument values are eliminated before being
-passed to the aggregate function.
-
-| Operator syntax                    | Description
-| ---------------------------------- | -----------
-| COUNT( [ DISTINCT ] value [, value]* ) | Returns the number of input rows for which *value* is not null (wholly not null if *value* is composite)
-| COUNT(*)                           | Returns the number of input rows
-| AVG( [ DISTINCT ] numeric)         | Returns the average (arithmetic mean) of *numeric* across all input values
-| SUM( [ DISTINCT ] numeric)         | Returns the sum of *numeric* across all input values
-| MAX( [ DISTINCT ] value)           | Returns the maximum value of *value* across all input values
-| MIN( [ DISTINCT ] value)           | Returns the minimum value of *value* across all input values
-| STDDEV_POP( [ DISTINCT ] numeric)  | Returns the population standard deviation of *numeric* across all input values
-| STDDEV_SAMP( [ DISTINCT ] numeric) | Returns the sample standard deviation of *numeric* across all input values
-| VAR_POP( [ DISTINCT ] value)       | Returns the population variance (square of the population standard deviation) of *numeric* across all input values
-| VAR_SAMP( [ DISTINCT ] numeric)    | Returns the sample variance (square of the sample standard deviation) of *numeric* across all input values
-| COVAR_POP(numeric1, numeric2)      | Returns the population covariance of the pair (*numeric1*, *numeric2*) across all input values
-| COVAR_SAMP(numeric1, numeric2)     | Returns the sample covariance of the pair (*numeric1*, *numeric2*) across all input values
-| REGR_SXX(numeric1, numeric2)       | Returns the sum of squares of the dependent expression in a linear regression model
-| REGR_SYY(numeric1, numeric2)       | Returns the sum of squares of the independent expression in a linear regression model
-
-Not implemented:
-* REGR_AVGX(numeric1, numeric2)
-* REGR_AVGY(numeric1, numeric2)
-* REGR_COUNT(numeric1, numeric2)
-* REGR_INTERCEPT(numeric1, numeric2)
-* REGR_R2(numeric1, numeric2)
-* REGR_SLOPE(numeric1, numeric2)
-* REGR_SXY(numeric1, numeric2)
-
-### Window functions
-
-| Operator syntax                           | Description
-| ----------------------------------------- | -----------
-| COUNT(value [, value ]* ) OVER window     | Returns the number of rows in *window* for which *value* is not null (wholly not null if *value* is composite)
-| COUNT(*) OVER window                      | Returns the number of rows in *window*
-| AVG(numeric) OVER window                  | Returns the average (arithmetic mean) of *numeric* across all values in *window*
-| SUM(numeric) OVER window                  | Returns the sum of *numeric* across all values in *window*
-| MAX(value) OVER window                    | Returns the maximum value of *value* across all values in *window*
-| MIN(value) OVER window                    | Returns the minimum value of *value* across all values in *window*
-| RANK() OVER window                        | Returns the rank of the current row with gaps; same as ROW_NUMBER of its first peer
-| DENSE_RANK() OVER window                  | Returns the rank of the current row without gaps; this function counts peer groups
-| ROW_NUMBER() OVER window                  | Returns the number of the current row within its partition, counting from 1
-| FIRST_VALUE(value) OVER window            | Returns *value* evaluated at the row that is the first row of the window frame
-| LAST_VALUE(value) OVER window             | Returns *value* evaluated at the row that is the last row of the window frame
-| LEAD(value, offset, default) OVER window  | Returns *value* evaluated at the row that is *offset* rows after the current row within the partition; if there is no such row, instead returns *default*. Both *offset* and *default* are evaluated with respect to the current row. If omitted, *offset* defaults to 1 and *default* to NULL
-| LAG(value, offset, default) OVER window   | Returns *value* evaluated at the row that is *offset* rows before the current row within the partition; if there is no such row, instead returns *default*. Both *offset* and *default* are evaluated with respect to the current row. If omitted, *offset* defaults to 1 and *default* to NULL
-| NTILE(value) OVER window                  | Returns an integer ranging from 1 to *value*, dividing the partition as equally as possible
-
-Not implemented:
-* COUNT(DISTINCT value) OVER window
-* FIRST_VALUE(value) IGNORE NULLS OVER window
-* LAST_VALUE(value) IGNORE NULLS OVER window
-* PERCENT_RANK(value) OVER window
-* CUME_DIST(value) OVER window
-* NTH_VALUE(value, nth) OVER window
-
-### Grouping functions
-
-| Operator syntax      | Description
-| -------------------- | -----------
-| GROUPING(expression) | Returns 1 if expression is rolled up in the current row's grouping set, 0 otherwise
-| GROUP_ID()           | Returns an integer that uniquely identifies the combination of grouping keys
-| GROUPING_ID(expression [, expression ] * ) | Returns a bit vector of the given grouping expressions


[02/13] incubator-calcite git commit: Upgrade quidem

Posted by jh...@apache.org.
Upgrade quidem


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

Branch: refs/heads/master
Commit: 88599ef5c9dbce95d3155474f2286d4087724a2f
Parents: f98d567
Author: Julian Hyde <jh...@apache.org>
Authored: Fri May 8 15:50:06 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri May 8 17:06:02 2015 -0700

----------------------------------------------------------------------
 core/src/test/resources/sql/misc.oq     | 24 ++++++++++++------------
 core/src/test/resources/sql/subquery.oq |  8 ++++----
 pom.xml                                 |  2 +-
 3 files changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/88599ef5/core/src/test/resources/sql/misc.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.oq b/core/src/test/resources/sql/misc.oq
index 4d4bc80..75af0bc 100644
--- a/core/src/test/resources/sql/misc.oq
+++ b/core/src/test/resources/sql/misc.oq
@@ -805,18 +805,18 @@ select distinct gender, sum(deptno) as s from emp group by gender;
 (2 rows)
 
 !ok
-# The following queries should give error. Uncomment when
-#   [QUIDEM-1] Add '!error' command
-# is fixed.
-#select distinct gender, deptno from emp group by gender;
-#Expression 'DEPTNO' is not being grouped
-#!error
-#select distinct gender, deptno from emp group by gender, ename;
-#Expression 'DEPTNO' is not being grouped
-#!error
-#select distinct gender, deptno, sum(deptno) as s from emp group by gender;
-#Expression 'DEPTNO' is not being grouped
-#!error
+
+select distinct gender, deptno from emp group by gender;
+Expression 'DEPTNO' is not being grouped
+!error
+
+select distinct gender, deptno from emp group by gender, ename;
+Expression 'DEPTNO' is not being grouped
+!error
+
+select distinct gender, deptno, sum(deptno) as s from emp group by gender;
+Expression 'DEPTNO' is not being grouped
+!error
 
 !use scott
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/88599ef5/core/src/test/resources/sql/subquery.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.oq b/core/src/test/resources/sql/subquery.oq
index 72349a6..9db95d3 100644
--- a/core/src/test/resources/sql/subquery.oq
+++ b/core/src/test/resources/sql/subquery.oq
@@ -131,7 +131,7 @@ select deptno, deptno not in (select deptno from emp where deptno is null) from
 
 # RHS has only NOT NULL keys
 select * from dept where deptno not in (select deptno from emp where deptno is not null);
- DEPTNO | DNAME      
+ DEPTNO | DNAME
 --------+-------------
      40 | Empty      
 (1 row)
@@ -161,7 +161,7 @@ select deptno, deptno not in (select deptno from emp where deptno is not null) f
 # RHS has no rows
 # Even 'NULL NOT IN ...' is TRUE.
 select * from dept where deptno not in (select deptno from emp where false);
- DEPTNO | DNAME      
+ DEPTNO | DNAME
 --------+-------------
      10 | Sales      
      20 | Marketing  
@@ -195,7 +195,7 @@ select deptno, deptno not in (select deptno from emp where false) from dept;
 select * from dept
 where deptno in (select deptno from emp where gender = 'F')
 or deptno in (select deptno from emp where gender = 'M');
- DEPTNO | DNAME      
+ DEPTNO | DNAME
 --------+-------------
      30 | Engineering
      10 | Sales      
@@ -208,7 +208,7 @@ or deptno in (select deptno from emp where gender = 'M');
 select * from dept
 where deptno in (select deptno from emp where gender = 'F')
 or exists (select 99, 101 from emp where gender = 'X');
- DEPTNO | DNAME      
+ DEPTNO | DNAME
 --------+-------------
      30 | Engineering
      10 | Sales      

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/88599ef5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 37efe84..1572a57 100644
--- a/pom.xml
+++ b/pom.xml
@@ -200,7 +200,7 @@ limitations under the License.
       <dependency>
         <groupId>net.hydromatic</groupId>
         <artifactId>quidem</artifactId>
-        <version>0.3</version>
+        <version>0.4</version>
       </dependency>
       <dependency>
         <groupId>net.hydromatic</groupId>


[09/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/HISTORY.md
----------------------------------------------------------------------
diff --git a/doc/HISTORY.md b/doc/HISTORY.md
deleted file mode 100644
index 417cb20..0000000
--- a/doc/HISTORY.md
+++ /dev/null
@@ -1,1475 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-# Apache Calcite release history
-
-For a full list of releases, see
-<a href="https://github.com/apache/incubator-calcite/releases">github</a>.
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-1.2.0-incubating">1.2.0-incubating</a> / 2015-04-07
-
-A short release, less than a month after 1.1.
-
-There have been many changes to Avatica, hugely improving its coverage of the
-JDBC API and overall robustness. A new provider, `JdbcMeta`, allows
-you to remote an existing JDBC driver.
-
-[<a href="https://issues.apache.org/jira/browse/CALCITE-606">CALCITE-606</a>]
-improves how the planner propagates traits such as collation and
-distribution among relational expressions.
-
-[<a href="https://issues.apache.org/jira/browse/CALCITE-613">CALCITE-613</a>]
-and [<a href="https://issues.apache.org/jira/browse/CALCITE-307">CALCITE-307</a>]
-improve implicit and explicit conversions in SQL.
-
-New features
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-366">CALCITE-366</a>]
-  Support Aggregate push down in bushy joins (Jesus Camacho Rodriguez)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-613">CALCITE-613</a>]
-  Implicitly convert character values in comparisons
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-307">CALCITE-307</a>]
-  Implement `CAST` between date-time types
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-634">CALCITE-634</a>]
-  Allow `ORDER BY` aggregate function in `SELECT DISTINCT`, provided that it
-  occurs in `SELECT` clause (Sean Hsuan-Yi Chu)
-* In linq4j, implement `firstOrDefault`, `single`, and `singleOrDefault` methods
-  (Daniel Cooper)
-* JDBC adapter
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-631">CALCITE-631</a>]
-    Push theta joins down to JDBC adapter (Ng Jiunn Jye)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-657">CALCITE-657</a>]
-    `NullPointerException` when executing `JdbcAggregate.implement`
-    method (Yuri Au Yong)
-* Metadata
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-659">CALCITE-659</a>]
-    Missing types in `averageTypeValueSize` method in `RelMdSize`
-    (Jesus Camacho Rodriguez)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-650">CALCITE-650</a>]
-    Add metadata for average size of a tuple in `SemiJoin` (Jesus
-    Camacho Rodriguez)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-649">CALCITE-649</a>]
-    Extend `splitCondition` method in `RelOptUtil` to handle multiple
-    joins on the same key (Jesus Camacho Rodriguez)
-
-Avatica features and bug fixes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-670">CALCITE-670</a>]
-  `AvaticaPreparedStatement` should support `execute()` and
-  `executeUpdate()` (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-641">CALCITE-641</a>]
-  Implement logging throughout Avatica server (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-646">CALCITE-646</a>]
-  `AvaticaStatement.execute` method broken over remote JDBC (Yeong Wei
-  and Julian Hyde)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-660">CALCITE-660</a>]
-  Improve Avatica date support
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-655">CALCITE-655</a>]
-  Implement `ConnectionSync` RPC (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-654">CALCITE-654</a>]
-  Tighten up `AvaticaStatement.execute` semantics (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-658">CALCITE-658</a>]
-  Cleanup dependency usage (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-652">CALCITE-652</a>]
-  Move server pieces of `avatica` into `avatica-server` (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-651">CALCITE-651</a>]
-  In `JdbcMeta`, convert property definitions to an enum (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-640">CALCITE-640</a>]
-  Avatica server should expire stale connections/statements (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-639">CALCITE-639</a>]
-  Open up permissions on avatica server components (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-637">CALCITE-637</a>]
-  Implement Avatica `CloseConnection` RPC (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-636">CALCITE-636</a>]
-  Connection isolation for Avatica clients (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-626">CALCITE-626</a>]
-  Implement `CloseStatement` RPC (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-630">CALCITE-630</a>]
-  Flesh out `AvaticaParameter.setObject` (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-627">CALCITE-627</a>]
-  Add Avatica support for `getTableTypes`, `getColumns` (Xavier FH Leong)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-618">CALCITE-618</a>]
-  Add Avatica support for `getTables` (Julian Hyde and Nick Dimiduk)
-
-API changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-617">CALCITE-617</a>]
-  Check at initialization time in `CachingInvocationHandler` that MD provider
-  is not null (Jesus Camacho Rodriguez)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-638">CALCITE-638</a>]
-  SQL standard `REAL` is 4 bytes, `FLOAT` is 8 bytes
-
-Bug-fixes and internal changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-672">CALCITE-672</a>]
-  SQL `ANY` type should be nullable (Jinfeng Ni)
-* Disable tests, pending
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-673">CALCITE-673</a>]
-  Timeout executing joins against MySQL
-* Fix traits in MongoDB adapter, and `NullPointerException` in `JdbcTest`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-662">CALCITE-662</a>]
-  Query validation fails when an `ORDER BY` clause is used with `WITH CLAUSE`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-606">CALCITE-606</a>]
-  Fix trait propagation and add test case
-* Remove checkstyle Eclipse properties from git tracking
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-644">CALCITE-644</a>]
-  Increase check style line limit to 100 chars (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-648">CALCITE-648</a>]
-  Update `ProjectMergeRule` description for new naming convention (Jinfeng Ni)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-625">CALCITE-625</a>]
-  `README.md` linking to the wrong page of `optiq-csv` (hongbin ma)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-632">CALCITE-632</a>]
-  Sort order returned by `SUPERCLASS_COMPARATOR` in
-  `ReflectiveRelMetadataProvider` is inconsistent (Jesus Camacho
-  Rodriguez)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-335">CALCITE-335</a>]
-  Remove uses of linq4j `Functions.adapt`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-592">CALCITE-592</a>]
-  Upgrade to Guava 14.0.1
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-596">CALCITE-596</a>]
-  JDBC adapter incorrectly reads null values as 0 (Ng Jiunn Jye)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-633">CALCITE-633</a>]
-  `WITH ... ORDER BY` cannot find table
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-614">CALCITE-614</a>]
-  `IN` clause in `CASE` in `GROUP BY` gives `AssertionError`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-619">CALCITE-619</a>]
-  Slim down dependencies in parent POM
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-1.1.0-incubating">1.1.0-incubating</a> / 2015-03-13
-
-This Calcite release makes it possible to exploit physical properties
-of relational expressions to produce more efficient plans, introducing
-collation and distribution as traits, `Exchange` relational operator,
-and several new forms of metadata.
-
-We add experimental support for streaming SQL.
-
-This release drops support for JDK 1.6; Calcite now requires 1.7 or
-later.
-
-We have introduced static `create` methods for many sub-classes of
-`RelNode`. We strongly suggest that you use these rather than
-calling constructors directly.
-
-New features
-* SQL
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-602">CALCITE-602</a>]
-    Streaming queries (experimental)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-588">CALCITE-588</a>]
-    Allow `TableMacro` to consume maps and collections
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-583">CALCITE-583</a>]
-    Operator `||` mishandles `ANY` type (Sean Hsuan-Yi Chu)
-* Planner rule improvements
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-445">CALCITE-445</a>]
-    Pull up filters rejected by a `ProjectableFilterableTable`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-600">CALCITE-600</a>]
-    Use `SetOpFactory` in rules containing `Union` operator (Jesus
-    Camacho Rodriguez)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-603">CALCITE-603</a>]
-    Metadata providers for size, memory, parallelism
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-607">CALCITE-607</a>]
-      Change visibility of constructor in metadata providers for size,
-      memory, parallelism (Jesus Camacho Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-608">CALCITE-608</a>]
-      Exception is thrown when `RelMdDistribution` for `Project`
-      operator is called (Jesus Camacho Rodriguez)
-* Collation and distribution as traits
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-88">CALCITE-88</a>]
-    Add collation as a trait and a kind of `RelNode` metadata
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-569">CALCITE-569</a>]
-    `ArrayIndexOutOfBoundsException` when deducing collation (Aman Sinha)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-581">CALCITE-581</a>]
-    Add `LogicalSort` relational expression, and make `Sort` abstract
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-526">CALCITE-526</a>]
-    Add `EnumerableMergeJoin`, which exploits sorted inputs
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-71">CALCITE-71</a>]
-    Provide a way to declare that tables are sorted
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-576">CALCITE-576</a>]
-    Make `RelCollation` trait and `AbstractRelNode.getCollationList` consistent
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-254">CALCITE-254</a>]
-    Propagate `RelCollation` on aliased columns in `JoinRule`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-569">CALCITE-569</a>]
-    `ArrayIndexOutOfBoundsException` when deducing collation
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-594">CALCITE-594</a>]
-    Add `RelDistribution` trait and `Exchange` relational expression
-
-API changes
-* Many sub-classes of `RelNode` now have a static `create` method
-  which automatically sets up traits such as collation and
-  distribution. The constructors are not marked deprecated, but we
-  strongly suggest that you use the `create` method if it exists.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-591">CALCITE-591</a>]
-  Drop support for Java 1.6 (and JDBC 4.0)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-587">CALCITE-587</a>]
-  Upgrade `jetty-server` to 9.2.7.v20150116 and port avatica-server `HttpServer`
-  (Trevor Hartman)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-577">CALCITE-577</a>]
-  Revert temporary API changes introduced in
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-575">CALCITE-575</a>]
-* Add means to create `Context` instances by wrapping objects and by chaining
-  contexts
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-599">CALCITE-599</a>]
-  `EquiJoin` in wrong package (Jesus Camacho Rodriguez)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-573">CALCITE-573</a>]
-  Use user-given names in `RelOptUtil.createProject` and `createRename`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-572">CALCITE-572</a>]
-  Remove `Project.flags` (methods are deprecated, to be removed before 2.0)
-
-Bug-fixes and internal changes
-* Remove the `LICENSE` file of calcite-example-csv (the former
-  optiq-csv) and move its history into main history
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-615">CALCITE-615</a>]
-  AvaticaParameter should be Jackson serializable (Nick Dimiduk)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-612">CALCITE-612</a>]
-  Update AvaticaStatement to handle cancelled queries (Parth Chandra)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-605">CALCITE-605</a>]
-  Reduce dependency on third-party maven repositories
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-611">CALCITE-611</a>]
-  Method `setAggChildKeys` should take into account indicator columns of
-  `Aggregate` operator (Jesus Camacho Rodriguez)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-566">CALCITE-566</a>]
-  `ReduceExpressionsRule` requires planner to have an `Executor`
-* Refactor `TableScanNode.create` method
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-593">CALCITE-593</a>]
-  Validator in `Frameworks` should expand identifiers (Jinfeng Ni)
-* Australian time-zones changed in `tzdata2014f`, Java 1.8.0_31
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-580">CALCITE-580</a>]
-  Average aggregation on an `Integer` column throws `ClassCastException`
-* In Travis, ask Surefire to print results to screen
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-586">CALCITE-586</a>]
-  Prevent JSON serialization of `Signature.internalParameters`
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-1.0.0-incubating">1.0.0-incubating</a> / 2015-01-31
-
-Calcite's first major release.
-
-Since the previous release we have re-organized the into the `org.apache.calcite`
-namespace. To make migration of your code easier, we have described the
-<a href="https://issues.apache.org/jira/secure/attachment/12681620/mapping.txt">mapping from old to new class names</a>
-as an attachment to
-[<a href="https://issues.apache.org/jira/browse/CALCITE-296">CALCITE-296</a>].
-
-The release adds SQL support for `GROUPING SETS`, `EXTEND`, `UPSERT` and sequences;
-a remote JDBC driver;
-improvements to the planner engine and built-in planner rules;
-improvements to the algorithms that implement the relational algebra,
-including an interpreter that can evaluate queries without compilation;
-and fixes about 30 bugs.
-
-New features
-* SQL
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-494">CALCITE-494</a>]
-    Support `NEXT`/`CURRENT VALUE FOR` syntax for using sequences
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-492">CALCITE-492</a>]
-    Support `UPSERT` statement in parser
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-493">CALCITE-493</a>]
-    Add `EXTEND` clause, for defining columns and their types at query/DML time
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-497">CALCITE-497</a>]
-    Support optional qualifier for column name references
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-356">CALCITE-356</a>]
-    Allow column references of the form `schema.table.column`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-462">CALCITE-462</a>]
-    Allow table functions in `LATERAL` expression
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-282">CALCITE-282</a>]
-    Add `{fn QUARTER(date)}` function (Benoy Antony)
-  * Grouping sets
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-370">CALCITE-370</a>]
-      Support `GROUPING SETS`, `CUBE`, `ROLLUP` in SQL and algebra
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-512">CALCITE-512</a>]
-      Add `GROUP_ID`,`GROUPING_ID`, `GROUPING` functions
-* Planner rule improvements
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-92">CALCITE-92</a>]
-    Optimize away `Project` that merely renames fields
-  * Detect and merge duplicate predicates `AND(x, y, x)` to `AND(x, y)` in more
-    circumstances
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-557">CALCITE-557</a>]
-    Speed up planning by never creating `AbstractConverter`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-545">CALCITE-545</a>]
-    When a projected expression can only have one value, replace with that
-    constant
-  * Grouping sets
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-542">CALCITE-542</a>]
-      Support for `Aggregate` with grouping sets in `RelMdColumnOrigins` (Jesus
-      Camacho Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-533">CALCITE-533</a>]
-      Support for grouping sets in `FilterAggregateTransposeRule` (Jesus Camacho
-      Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-532">CALCITE-532</a>]
-      Support for grouping sets in `AggregateFilterTransposeRule` (Jesus Camacho
-      Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-513">CALCITE-513</a>]
-      Support for grouping sets in `AggregateProjectMergeRule` (Jesus Camacho
-      Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-510">CALCITE-510</a>]
-      Support for grouping sets in `AggregateExpandDistinctAggregatesRule` (Jesus
-      Camacho Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-502">CALCITE-502</a>]
-      Support for grouping sets in `AggregateUnionTransposeRule` (Jesus Camacho
-      Rodriguez)
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-503">CALCITE-503</a>]
-      Tests to check rules on `Aggregate` operator without grouping sets (Jesus
-      Camacho Rodriguez)
-* Algorithms
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-451">CALCITE-451</a>]
-    Implement theta join, inner and outer, in enumerable convention
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-489">CALCITE-489</a>]
-    Update `Correlate` mechanics and implement `EnumerableCorrelate` (aka nested
-    loops join)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-544">CALCITE-544</a>]
-    Implement `Union` in interpreter
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-562">CALCITE-562</a>]
-    Implement inner `Join` in interpreter and improve handling of scalar expressions
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-543">CALCITE-543</a>]
-    Implement `Aggregate` (including `GROUPING SETS`) in interpreter (Jacques
-    Nadeau)
-  * In progress towards
-    [<a href="https://issues.apache.org/jira/browse/CALCITE-558">CALCITE-558</a>]
-    add `BINDABLE` convention (but `ENUMERABLE` is still the default), and add
-    `ArrayBindable` and `Scalar` interfaces
-* Remote driver
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-93">CALCITE-93</a>]
-    Calcite RPC server
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-94">CALCITE-94</a>]
-    Remote JDBC driver
-  * Make `JsonHandler` and `JsonService` thread-safe
-
-API changes
-* The great code re-org
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-296">CALCITE-296</a>]
-    Re-organize package structure
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-419">CALCITE-419</a>]
-    Naming convention for planner rules
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-306">CALCITE-306</a>]
-    Standardize code style for "import package.*;"
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-474">CALCITE-474</a>]
-    Clean up rule naming in order to support enabling/disabling rules
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-460">CALCITE-460</a>]
-    Add `ImmutableBitSet` and replace uses of `BitSet`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-479">CALCITE-479</a>]
-    Migrate `RelNode.getChildExps` to `RelNode.accept(RexShuttle)`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-527">CALCITE-527</a>]
-    Drop `rowType` field and constructor/copy argument of `Calc`
-* Add linq4j and example-csv modules
-  * Remove unused packages in linq4j, and fix checkstyle issues in linq4j and csv
-  * Add calcite-linq4j and calcite-example-csv as POM sub-modules
-  * Import 'optiq-csv' project as 'example/csv/', and add Apache headers
-  * Import 'linq4j' project, and add Apache headers
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-478">CALCITE-478</a>]
-    Move CSV tutorial (Siva Narayanan)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-464">CALCITE-464</a>]
-  Make parser accept configurable max length for SQL identifier
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-465">CALCITE-465</a>]
-  Remove `OneRow` and `Empty` relational expressions; `Values` will suffice
-
-Bug-fixes and internal changes
-* Build improvements
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-541">CALCITE-541</a>]
-    Update maven-source-plugin to 2.4 to get speedup in jdk 1.8
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-537">CALCITE-537</a>]
-    Skip overwrite of `NOTICE`, `DEPENDENCIES`, and `LICENSE` files
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-538">CALCITE-538</a>]
-    Generate `Parser.jj` only at first build
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-539">CALCITE-539</a>]
-    Avoid rewrite of `org-apache-calcite-jdbc.properties`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-540">CALCITE-540</a>]
-    Create git.properties file only at first build. This saves time in
-    development at a cost of stale `git.properties`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-536">CALCITE-536</a>]
-    Add `@PackageMarker` to `package-info.java` so maven-compiler skips
-    compilation when the sources are unchanged
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-535">CALCITE-535</a>]
-    Support skip overwrite in hydromatic-resource
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-582">CALCITE-582</a>]
-  `EnumerableTableScan` broken when table has single column
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-575">CALCITE-575</a>]
-  Variant of `ProjectRemoveRule` that considers a project trivial only if its
-  field names are identical (John Pullokkaran)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-571">CALCITE-571</a>]
-  `ReduceExpressionsRule` tries to reduce `SemiJoin` condition to non-equi
-  condition
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-568">CALCITE-568</a>]
-  Upgrade to a version of `pentaho-aggdesigner` that does not pull in
-  `servlet-api`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-567">CALCITE-567</a>]
-  Make `quidem` dependency have scope "test"
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-570">CALCITE-570</a>]
-  `ReduceExpressionsRule` throws "duplicate key" exception
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-561">CALCITE-561</a>]
-  Upgrade parent POM
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-458">CALCITE-458</a>]
-  ArrayIndexOutOfBoundsException when using just a single column in interpreter
-* Fix spurious extra row from `FULL JOIN`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-554">CALCITE-554</a>]
-  Outer join over NULL keys generates wrong result
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-489">CALCITE-489</a>]
-  Teach `CalciteAssert` to respect multiple settings
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-516">CALCITE-516</a>]
-  `GROUP BY` on a `CASE` expression containing `IN` predicate fails (Aman Sinha)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-552">CALCITE-552</a>]
-  Upgrade tpcds (which depends on an old version of guava)
-* Copy identifier when fully-qualifying, so column aliases have the right case
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-548">CALCITE-548</a>]
-  Extend `induce` method to return `CUBE` and `ROLLUP` (Jesus Camacho Rodriguez)
-  * Simplify `Group.induce` by assuming that group sets are sorted
-* Test case for
-  [<a  href="https://issues.apache.org/jira/browse/CALCITE-212">CALCITE-212</a>]
-  Join condition with `OR`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-550">CALCITE-550</a>]
-  Case-insensitive matching of sub-query columns fails
-  * Add more unit tests (Jinfeng Ni)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-448">CALCITE-448</a>]
-  `FilterIntoJoinRule` creates filters containing invalid `RexInputRef`
-* When registering a `RelNode`, be tolerant if it is equivalent to a `RelNode`
-  with different traits
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-547">CALCITE-547</a>]
-  Set nullability while inferring return type of `item(any,...)` operator
-* In Travis CI, enable containers, and cache `.m2` directory
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-534">CALCITE-534</a>]
-  Missing implementation of `ResultSetMetaData.getColumnClassName` (Knut
-  Forkalsrud)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-506">CALCITE-506</a>]
-  Update `EnumerableRelImplementor.stash` so it is suitable for all kinds of
-  classes
-* Merge join algorithm for `Enumerable`s
-* Efficient `Enumerable` over random-access list
-* Add a test that calls all functions with arguments of all types that they
-  claim to accept
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-511">CALCITE-511</a>]
-  `copy` method in `LogicalAggregate` not copying the indicator value properly
-* Add a model that has lattices and works against HSQLDB
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-509">CALCITE-509</a>]
-  `RelMdColumnUniqueness` uses `ImmutableBitSet.Builder` twice, gets
-  `NullPointerException`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-488">CALCITE-488</a>]
-  `Enumerable<Holder>` does not work if where `Holder` is a custom class
-  with a single field; Calcite tries to treat it as `SCALAR` due to premature
-  `JavaRowFormat.optimize`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-352">CALCITE-352</a>]
-  Throw exception if `ResultSet.next()` is called after `close()`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-403">CALCITE-403</a>]
-  `Enumerable` gives `NullPointerException` with `NOT` on nullable expression
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-469">CALCITE-469</a>]
-  Update example/csv README.md instructions
-* Document `WITH`, `LATERAL`, `GROUPING SETS`, `CUBE`, `ROLLUP`;
-  add descriptions for all built-in functions and operators
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-470">CALCITE-470</a>]
-  Print warning when column type hint is not understood;
-  Update `EMPS.deptno` column Integer &rarr; int
-* Fix `Linq4j.product`; the cartesian product of 0 attributes is one row of 0
-  attributes
-* Update link optiq-mat-plugin &rarr; mat-calcite-plugin
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-467">CALCITE-467</a>]
-  Incorrect namespace in `package-info.java`
-* Add headers, to appease the RAT
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-446">CALCITE-446</a>]
-  CSV adapter should read from directory relative to the model file
-* Add examples of scannable and filterable tables, matching
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-436">CALCITE-436</a>]
-  Simpler SPI to query Table
-* Fix `JdbcTest.testVersion` now that version is 1.0
-* Update release HOWTO
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-0.9.2-incubating">0.9.2-incubating</a> / 2014-11-05
-
-A fairly minor release, and last release before we rename all of the
-packages and lots of classes, in what we expect to call 1.0. If you
-have an existing application, it's worth upgrading to this first,
-before you move on to 1.0.
-
-New features
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-436">CALCITE-436</a>]
-  Simpler SPI to query `Table`
-
-API changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-447">CALCITE-447</a>]
-  Change semi-join rules to make use of factories
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-442">CALCITE-442</a>
-  Add `RelOptRuleOperand` constructor that takes a predicate
-
-Bug-fixes and internal changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-397">CALCITE-397</a>]
-  `SELECT DISTINCT *` on reflective schema gives `ClassCastException` at runtime
-* Various lattice improvements.
-* sqlline: Looking for class-path in inconsistent locations.
-* Re-order test suite, so that fast tests are run first.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-444">CALCITE-444</a>]
-  Filters wrongly pushed into full outer join
-* Make it more convenient to unit test `RelMetadataQuery`, and add some more
-  tests for
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-443">CALCITE-443</a>]
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-443">CALCITE-443</a>]
-  `getPredicates` from a Union is not correct
-* Update references to web sites, git repositories, jira, mailing lists,
-  travis CI now that [INFRA-8413] is fixed
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-434">CALCITE-435</a>]
-  `FilterAggregateTransposeRule` loses conditions that cannot be pushed
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-435">CALCITE-435</a>]
-  `LoptOptimizeJoinRule` incorrectly re-orders outer joins
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-439">CALCITE-439</a>]
-  `SqlValidatorUtil.uniquify()` may not terminate under some conditions
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-438">CALCITE-438</a>]
-  Push predicates through `SemiJoinRel`
-* Add test case for `LIKE ... ESCAPE`.
-* HOWTO: Modify release instructions.
-* Update `DiffRepository` documentation.
-* Add tests for windowed aggregates without `ORDER BY`. (Works already.)
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/calcite-0.9.1-incubating">0.9.1-incubating</a> / 2014-10-02
-
-This is the first release as Calcite. (The project was previously called Optiq.)
-
-New features
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-430">CALCITE-430</a>]
-  Rename project from Optiq to Calcite
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-426">CALCITE-426</a>]
-  Pool JDBC data sources, to make it easier to pool connections
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-416">CALCITE-416</a>]
-  Execute logical `RelNode`s using an interpreter
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-376">CALCITE-376</a>]
-  Move `SqlRun` into its own artifact,
-  <a href="https://github.com/julianhyde/quidem">Quidem</a>.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-269">CALCITE-269</a>]
-  MongoDB result sets larger than 16MB
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-373">CALCITE-373</a>]
-  `NULL` values in `NOT IN` sub-queries
-* SQL functions:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-422">CALCITE-422</a>]
-    Add `REGR_SXX` and `REGR_SYY` regression functions
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-421">CALCITE-421</a>]
-    Add `COVAR_POP` and `COVAR_SAMP` aggregate functions
-* Planner rules:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-425">CALCITE-425</a>]
-    Add `FilterAggregateTransposeRule`, that pushes a filter through an
-    aggregate
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-399">CALCITE-399</a>]
-    Factorize common `AND` factors out of `OR` predicates
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-404">CALCITE-404</a>]
-    `MergeProjectRule` should not construct `RexProgram`s for simple mappings
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-394">CALCITE-394</a>]
-    Add `RexUtil.toCnf()`, to convert expressions to conjunctive normal form
-    (CNF)
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-389">CALCITE-389</a>]
-    `MergeFilterRule` should flatten `AND` condition
-* Lattices:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-428">CALCITE-428</a>]
-    Use optimization algorithm to suggest which tiles of a lattice to
-    materialize
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-410">CALCITE-410</a>]
-    Allow lattice tiles to satisfy a query by rolling up
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-406">CALCITE-406</a>]
-    Add tile and measure elements to lattice model element
-  * Now, a lattice can materialize an aggregate-join and use it in a subsequent
-    query.
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-402">CALCITE-402</a>]
-    Lattice should create materializations on demand
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-344">CALCITE-344</a>]
-    Lattice data structure
-* Field trimmer:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-408">CALCITE-408</a>]
-    Make `FieldTrimmer` work with `RelNode` base classes
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-388">CALCITE-388</a>]
-    Handle semi-joins in field trimmer
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-395">CALCITE-395</a>]
-    Make `FieldTrimmer.trimFields(SetOp)` generate `ProjectRel` instead of
-    `CalcRel`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-393">CALCITE-393</a>]
-    If no fields are projected from a table, field trimmer should project a
-    dummy expression
-
-API changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-413">CALCITE-413</a>]
-  Add `RelDataTypeSystem` plugin, allowing different max precision of a
-  `DECIMAL`
-* In `Planner`, query de-correlation no longer requires state in a
-  `SqlToRelConverter`.
-* Factories:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-392">CALCITE-392</a>]
-    `RelFieldTrimmer` should use factory to create new rel nodes
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-382">CALCITE-382</a>]
-    Refactoring rules to use factories
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-398">CALCITE-398</a>]
-    Move `CalcRel.createProject` methods to `RelOptUtil`
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-396">CALCITE-396</a>]
-    Change return type of `JoinFactory.createJoin()`; add `SemiJoinFactory`
-
-Bug-fixes and internal changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-386">CALCITE-386</a>]
-  Fix NOTICE
-* Add tests inspired by Derby bugs.
-* Add recent presentation to README.md.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-427">CALCITE-427</a>]
-  Off-by-one issues in `RemoveDistinctAggregateRule`,
-  `AggregateFilterTransposeRule`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-414">CALCITE-414</a>]
-  Bad class name in `sqlline` shell script
-* Bad package name in `package-info.java` was causing errors in Eclipse.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-412">CALCITE-412</a>]
-  `RelFieldTrimmer`: when trimming `SortRel`, the collation and trait set don't
-  match
-* Add test case for
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-411">CALCITE-411</a>]
-  Duplicate column aliases
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-407">CALCITE-407</a>]
-  `RemoveTrivialProjectRule` drops child node's traits
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-409">CALCITE-409</a>]
-  `PushFilterPastProjectRule` should not push filters past windowed aggregates
-* Fix tests on Windows.
-* Don't load `FoodMartQuerySet` unless we have to. It's big.
-* Enable connection pooling in test suite.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-384">CALCITE-384</a>]
-  Add `apache-` prefix to tarball and directory within tarball
-* Freeze hive fmpp > freemarker plugin dependency.
-* Upgrade Janino
-* Removed hardcoded foodmart schema information
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-387">CALCITE-387</a>]
-  CompileException when cast TRUE to nullable boolean
-* Temporary fix for
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-390">CALCITE-390</a>]
-  Transitive inference (`RelMdPredicates`) doesn't handle semi-join
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-385">CALCITE-385</a>]
-  Change comment style for Java headers
-* Disable test that is inconistent between JDK 1.7 and 1.8.
-* Fix `git-commit-id-plugin` error when running in Travis-CI.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-381">CALCITE-381</a>]
-  Remove plugin versions from the `&lt;plugins&gt;` tag in root pom
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-383">CALCITE-383</a>]
-  Each jar should have a `git.properties` file describing its exact version
-* Fix `mvn site` on JDK 1.8 and enable in Travis-CI.
-* Status icon based on master branch, not whichever branch happened to build
-  most recently.
-* HOWTO:
-  * Document how to build from git, and how to get jars from maven repo.
-  * Optiq web site
-  * Template emails for Apache votes
-  * Update JIRA cases following release
-  * Instructions for making and verifying a release
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-0.9.0-incubating">0.9.0-incubating</a> / 2014-08-19
-
-This is the first release under the Apache incubator process.
-
-New features
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-371">CALCITE-371</a>]
-  Implement `JOIN` whose `ON` clause contains mixed equi and theta
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-369">CALCITE-369</a>]
-  Add `EnumerableSemiJoinRel`, implementation of semi-join in enumerable
-  convention
-* Add class `Strong`, for detecting null-rejecting predicates.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-368">CALCITE-368</a>]
-  Add SemiJoinRule, planner rule to convert project-join-aggregate into semi-join
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-367">CALCITE-367</a>]
-  `PushFilterPastJoinRule` should strengthen join type
-* Add `EquiJoinRel`, base class for joins known to be equi-joins.
-* Implement `CAST(&lt;string&gt; AS &lt;datetime&gt;)` and
-  `&lt;datetime&gt; + &lt;interval&gt;`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-360">CALCITE-360</a>]
-  Introduce a rule to infer predicates from equi-join conditions
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-349">CALCITE-349</a>]
-  Add heuristic join-optimizer that can generate bushy joins
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-346">CALCITE-346</a>]
-  Add commutative join rule
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-347">CALCITE-347</a>]
-  In `SqlRun`, add `!plan` command
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-314">CALCITE-314</a>]
-  Allow simple UDFs based on methods
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-327">CALCITE-327</a>]
-  Rules should use base class to find rule match & use factory for object
-  creation
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-316">CALCITE-316</a>]
-  In `SqlRun`, match output regardless of order if `ORDER BY` not present
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-300">CALCITE-300</a>]
-  Support multiple parameters in `COUNT(DISTINCT x, y, ...)`
-
-API changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-343">CALCITE-343</a>]
-  RelDecorrelator should build its own mappings, not inherit from SqlToRelConverter
-* Remove deprecated methods.
-* Convert `Hook` to use Guava `Function` (was linq4j `Function1`).
-* Add fluent method `withHook`, to more easily add hooks in tests.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-321">CALCITE-321</a>]
-  Add support for overriding implementation of `CompoundIdentifier` in
-  `SqlParser`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-322">CALCITE-322</a>]
-  Add support for `SqlExplain`, `SqlOrderBy` and `SqlWith` to support
-  `SqlShuttle` use.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-323">CALCITE-323</a>]
-  Override `SqlUnresolvedFunction.inferReturnType()` to return `ANY` type
-  so framework implementors can support late bound function implementations.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-324">CALCITE-324</a>]
-  Add `ViewExpander` for `Planner` in `Frameworks`. Expose additional
-  properties of `ViewTable` to allow subclassing.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-247">CALCITE-247</a>]
-  Add `Context` and `FrameworkConfig`
-
-Bug-fixes and internal changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-380">CALCITE-380</a>]
-  Downgrade to Guava 11.0.2
-* Move several .md files into new 'doc' directory, to keep the root directory simple.
-* Add DISCLAIMER
-* Update history and HOWTO
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-377">CALCITE-377</a>]
-  UnregisteredDriver should catch, log and re-throw NoClassDefFoundError
-* Inherit maven-release-plugin from Apache POM.
-* Test case for
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-373">CALCITE-373</a>]
-  NOT IN and NULL values
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-372">CALCITE-372</a>]
-  Change `LoptOptimizeJoinRule` &amp; `PushFilterPast`* rules to use factory
-* Upgrade `maven-checkstyle-plugin`.
-* Add class `Holder`, a mutable slot that can contain one object.
-* Remove the 2-minute wait at the top of the hour for tests of
-  `CURRENT_TIME`, etc.
-* Tune `ImmutableIntList`'s iterators.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-364">CALCITE-364</a>]
-  Validator rejects valid `WITH ... ORDER BY` query
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-363">CALCITE-363</a>]
-  Use `dependencyManagement` and `pluginManagement` in POM files
-* Add `FilterFactory`.
-* Add `README` file, incubation disclaimers, and how-to build and running tests.
-* Add `KEYS` and start how-to for making snapshots and releases.
-* Capital case component names; inherit license info from Apache parent POM.
-* Only run `apache-rat` and `git-commit-id-plugin` in "release" maven profile.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-348">CALCITE-348</a>]
-  Add Apache RAT as maven plugin
-* Change license headers from "Julian Hyde" to "ASF"; add headers where missing.
-* Fix build breakage on JDK 1.6 due to missing method `BitSet.previousClearBit`.
-* Refactor test infrastructure to allow testing against heuristic bushy-join
-  optimizer.
-* Add methods and tests for BitSets, and re-organize tests.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-354">CALCITE-354</a>]
-  Change maven groupId to "org.apache.optiq"
-* Specify return type when calling `RexBuilder.makeCall`, if possible.
-* Eliminate duplicate conditions in `RexProgramBuilder.addCondition`, not
-  `RexBuilder.makeCall` as previously.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-345">CALCITE-345</a>]
-  `AssertionError` in `RexToLixTranslator` comparing to date literal
-* Restore `PushFilterPastJoinRule` to `RelDecorrelator`; interim pending
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-343">CALCITE-343</a>]
-  fix.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-340">CALCITE-340</a>]
-  Fix bug in `SqlToRelConverter` when push expressions in join conditions into
-  `ProjectRel`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-313">CALCITE-313</a>]
-  Query decorrelation fails
-* While unifying a `RelNode` tree with a materialized view expression,
-  switch representation to `MutableRel`s.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-305">CALCITE-305</a>]
-  Unit test failure on release candidates
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-325">CALCITE-325</a>]
-  Use Java list instead of Guava list to avoid null checks in case of
-  `SqlTypeExplicitPrecedenceList`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-326">CALCITE-326</a>]
-  Fix `RelOptUtil` `ANY` type check.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-303">CALCITE-303</a>]
-  Migrate issue URLs
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-331">CALCITE-331</a>]
-  Precision/scale compatibility checks should always succeed for `ANY` type
-* In `SqlRun`, allow `!plan` after `!ok` for same SQL statement.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-318">CALCITE-318</a>]
-  Add unit test for `SqlRun`
-* Fix a bug where composite `SELECT DISTINCT` would return duplicate rows.
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.8">0.8</a> / 2014-06-27
-
-New features
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-310">CALCITE-310</a>]
-   Implement LEAD, LAG and NTILE windowed aggregates
-* Reduce `COUNT(not-null-expression)` to `COUNT()`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-292">CALCITE-292</a>]
-   Improve windowed aggregate return types
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-302">CALCITE-302</a>]
-   Use heuristic rule to plan queries with large numbers of joins
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-283">CALCITE-283</a>]
-  Add TPC-DS data generator
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-294">CALCITE-294</a>]
-  Implement DENSE_RANK windowed aggregate function
-* SqlRun utility
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-290">CALCITE-290</a>]
-    Add `SqlRun`, an idempotent utility for running SQL test scripts
-  * Add "!skip" command to SqlRun.
-  * Add MySQL formatting mode to SqlRun.
-
-API changes
-* Re-organize planner initialization,
-  to make it easier to use heuristic join order.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-301">CALCITE-301</a>]
-  Add `Program` interface, a planner phase more general than current `RuleSet`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-263">CALCITE-263</a>]
-  Add operand type that will cause a rule to fire when a new subset is created
-* Clean up and document SqlKind.
-  * Add `IS_NOT_TRUE` and `IS_NOT_FALSE` `SqlKind` enums.
-  * Add `SqlKind.IS_NOT_NULL` enum value, and use where possible,
-    including for `IS_NOT_UNKNOWN` operator.
-
-Bug-fixes and internal changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-312">CALCITE-312</a>]
-  Trim non-required fields before `WindowRel`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-311">CALCITE-311</a>]
-  Wrong results when filtering the results of windowed aggregation
-* More tests for `WITH ... ORDER BY`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-309">CALCITE-309</a>]
-  `WITH ... ORDER BY` query gives `AssertionError`
-* Enable `MultiJoinRel` and some other planner rule tests.
-* Add `ImmutableNullableList` and `UnmodifiableArrayList`,
-  and remove calls to `Arrays.asList`.
-* Add method `IntPair.zip`.
-* Reimplement regular and windowed aggregates
-* Switch from github to Apache JIRA for issues tracking.
-  * In release history, update issue URLs from github to Apache JIRA
-* The Apache mailing list is now the official mailing list. Add presentations.
-* Add test for overloaded UDF.
-* Add tests for `NOT IN` where sub-query returns NULL values.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-288">CALCITE-288</a>]
-  Add tests for windowed aggregation based on Postgres reference queries
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-286">CALCITE-286</a>]
-  Error casting MongoDB date
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-284">CALCITE-284</a>]
-  Window functions range defaults to `CURRENT ROW`
-* [<a href=https://issues.apache.org/jira/browse/CALCITE-285">CALCITE-285</a>]
-  Window functions throw exception without `ORDER BY`
-* Test case for
-  [<a href=““https://issues.apache.org/jira/browse/CALCITE-285”>CALCITE-285</a>].
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-281">CALCITE-281</a>]
-  `EXTRACT` function's SQL return type is `BIGINT` but implemented as Java `int`
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.7">0.7</a> / 2014-05-13
-
-New features
-* Implement table functions.
-* Arrays and multi-sets:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-267">CALCITE-267</a>]
-    Improve support for ARRAY data type
-  * Better type information for JDBC Array; nested array now possible.
-  * Implement `JOIN LATERAL` and `JOIN UNNEST`.
-  * Implement the `UNNEST` relational operator, and various improvements
-    to `ARRAY` and `MULTISET` data types.
-  * Represent `ARRAY` columns as Java lists.
-  * Implement `CARDINALITY(ARRAY)` SQL operator.
-* Implement scalar sub-query in `SELECT` clause.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-273">CALCITE-273</a>]
-  Support column alias in WITH queries (common table expressions)
-* Windowed aggregates:
-  * Aggregate over constants, e.g. `SUM(1) OVER (ROWS 10 PRECEDING)`;
-  * `UNBOUNDED PRECEDING` window range;
-  * Windowed aggregates computed over primitive scalars.
-* Fix return type inference for aggregate calls. If the `GROUP BY` clause is
-  empty, `SUM` may return null.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-37">CALCITE-37</a>]
-  Document JSON model file format (as <a href="MODEL.md">MODEL.md</a>).
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-238">CALCITE-238</a>]
-  Add adapter that generates TPC-H data
-* Improve exception message in `AvaticaConnection`; add
-  `ExceptionMessageTest`.
-* Implement micro-benchmarks via
-  <a href="http://openjdk.java.net/projects/code-tools/jmh/">JMH</a>.
-
-API changes
-* Provide an option to create root schema without the "metadata" schema.
-* Schema SPI:
-  * [<a href="https://issues.apache.org/jira/browse/CALCITE-175">CALCITE-175</a>]
-    Modify Schema SPI to allow caching
-  * Get sub-schemas defined by a Schema SPI, and cache their `OptiqSchema`
-    wrappers. (Tobi Vollebregt and Julian Hyde)
-* SqlAdvisor callable from client via JDBC.
-
-Bug-fixes and internal changes
-* Add Apache incubator proposal.
-* Rename RELEASE.md to HISTORY.md.
-* Upgrade maven-release-plugin.
-* Upgrade to linq4j-0.3.
-* Code generation improvements:
- * Move code-generation optimizer to linq4j;
- * Improve translation of strict functions;
- * Mark most methods in `SqlFunctions` as `@Deterministic`;
- * Support `static final` constants generated by linq4j.
- * Avoid excessive box and unbox of primitives when using `Object[]` storage.
- * In JDBC result set, avoid row computation on each accessor call.
-* Test composite join conditions in various flavors of outer join.
-* Use `fromTrait` of the just previously converted `RelNode` instead
-  of the original `RelNode`.
-* Disable a MongoDB test, pending
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-270">CALCITE-270</a>].
-* Hush warnings from `SplunkAdapterTest` if Splunk is not available.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-252">CALCITE-252</a>]
-  Scalar sub-query that returns 0 rows should become NULL value
-* `SplunkAdapterTest` now uses the same Foodmart database as `JdbcTest`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-242">CALCITE-242</a>]
-  SplunkAdapterTest fails
-* Remove some obsolete classes.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-205">CALCITE-205</a>]
-  Suspicious map.get in VolcanoPlanner.reregister
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.6">0.6</a> / 2014-04-11
-
-New features
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-214">CALCITE-214</a>]
-  Modify Frameworks to allow Schema to be re-used
-  Obsoletes `name` field of `ReflectiveSchema`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-237">CALCITE-237</a>]
-  Allow user-defined aggregate functions (UDAs) to be defined in a model
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-227">CALCITE-227</a>]
-  Extend `EXTRACT` function to support `DATE`, `TIME` and `TIMESTAMP` values
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-222">CALCITE-222</a>]
-  User-defined table macros
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-179">CALCITE-179</a>]
-  Optiq on Windows
-  * Add `sqlline.bat` and fix issues running `sqlline` under Cygwin.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-195">CALCITE-195</a>]
-  Push aggregation into MongoDB adapter
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-193">CALCITE-193</a>]
-  Implement OFFSET and LIMIT in MongoDB adapter
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-164">CALCITE-164</a>]
-  Improve query performance of optiq over MongoDB
-* Add Phoenix (HBase) SQL dialect (Bruno Dumon)
-
-API changes
-* Obsolete `RexImpTable.AggregateImplementor` and rename `AggImplementor2`.
-  (**This is a breaking change**.)
-* Convert `CombinedParser.jj` into freemarker template to allow
-  custom parser implementations. (Venki Korukanti)
-* Extend `Planner` to pass a custom `ConvertletTable` and custom SQL parser.
-* In `Frameworks`, add a way to specify list of `TraitDef`s that will be used
-  by planner. (Jinfeng Ni)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-198">CALCITE-198</a>]
-  Use `RexExecutor` to evaluate projections and filters
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-219">CALCITE-219</a>]
-  Parse `ALTER scope SET option = value` statement
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-215">CALCITE-215</a>]
-  A Schema should not have to remember its name and parent
-  (**This is a breaking change**.)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-180">CALCITE-180</a>]
-  Common base class for TableFunction, ScalarFunction
-  (**This is a breaking change**.)
-* Add methods for dealing with symbols; deprecate
-  `SqlLiteral.booleanValue(SqlNode)`, `SqlLiteral.symbolValue(SqlNode)`.
-* Add `RelOptPlanner.clear()`; now it is safe to call `transform` twice.
-  (Jinfeng Ni)
-* Remove APIs deprecated for 0.5.
-* Move around some operator classes and singletons.
-
-Bug fixes and internal changes
-* Upgrade to linq4j-0.2.
-* `FETCH` and `LIMIT` are ignored during SQL-to-RelNode translation.
-  (Venki Korukanti)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-245">CALCITE-245</a>]
-  Off-by-one translation of ON clause of JOIN
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-191">CALCITE-191</a>]
-  Rotate time/date/timestamp vals to local timezone
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-244">CALCITE-244</a>]
-  `RelOptTableImpl.create` always expects `QueryableTable` type in
-  `OptiqSchema.TableEntry`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-225">CALCITE-225</a>]
-  Optiq doesn't correctly decorrelate queries
-* Clean up package-info.  Remove duplicates in test packages so they
-  don't conflict with those in non-test packages.
-* Add `Pair.adjacents(Iterable)`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-199">CALCITE-199</a>]
-  Various `ANY` type conditions aren't correctly being considered
-  (Jacques Nadeau)
-* Add files to `.gitignore` that shouldn't be checked in when using
-  Eclipse. (Jacques Nadeau)
-* Add class `ControlFlowException`, and make it base class of
-  existing control-flow exception classes.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-232">CALCITE-232</a>]
-  Sum and avg of empty set should be null as per SQL specification
-* Add `SqlUnresolvedFunction`, to improve how return type of
-  user-defined functions is resolved. (Vladimir Sitnikov)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-228">CALCITE-228</a>]
-  Error while compiling generated Java code when using UDF in expression
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-226">CALCITE-226</a>]
-  User-defined functions should work without explicit schema prefix
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-229">CALCITE-229</a>]
-  Join between different JDBC schemas not implementable
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-230">CALCITE-230</a>]
-  RemoveSortRule derives trait set from sort, should derive it from sort's child
-* Test view and sub-query with `ORDER BY` and `LIMIT`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-223">CALCITE-223</a>]
-  Add `NOTICE` and `LICENSE` files in all generated JAR files
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-209">CALCITE-209</a>]
-  Consistent strategy for line-endings in tests
-  Convert uses of `NL` in tests to Linux newline "\n".
-  This makes string constants simpler.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-218">CALCITE-218</a>]
-  Functions case sensitive when using `Lex.MYSQL`
-* Add tests that a query with aggregate expressions in the `SELECT`
-  clause is considered an aggregate query, even if there is no `GROUP BY`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-216">CALCITE-216</a>]
-  Inconsistent use of provided operator table causes inability to
-  add aggregate functions
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-200">CALCITE-200</a>]
-  Javadoc generation fails under JDK 1.8
-* Add class `XmlOutput` (based on `org.eigenbase.xom.XMLOutput`) and remove
-  dependency on eigenbase-xom.
-* Performance: Don't create stack-trace for exceptions used for control-flow.
-  (Vladimir Sitnikov)
-* Performance: Tune `RexProgramBuilder` by using `Pair` rather than `String` as
-  expression key. (Vladimir Sitnikov)
-* Fix NPE using TRIM function with JDBC. (Bruno Dumon)
-* Add dependency on
-  <a href="https://github.com/julianhyde/hydromatic-resource">hydromatic-resource-maven-plugin</a>
-  and obsolete our copy of the resource framework.
-* Fix race condition in `SpaceList`.
-* In planner, use `RelTrait.subsumes` rather than `equals` in an assert.
-  (Jinfeng Ni)
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.5">0.5</a> / 2014-03-14
-
-New features
-* Allow `quoting`, `quotedCasing`, `unquotedCasing`, and `caseSensitive`
-  properties to be specified explicitly (Vladimir Sitnikov)
-* Recognize more kinds of materializations, including filter-on-project (where
-  project contains expressions) and some kinds of aggregation.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-128">CALCITE-128</a>]
-  Support `WITH` queries (common table expressions)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-53">CALCITE-53</a>]
-  Allow `WHEN` clause in simple `CASE` expression to have multiple values
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-156">CALCITE-156</a>]
-  Optiq should recognize 'SYSTEM TABLE', 'JOIN', 'INDEX' as table types
-* Support querying ARRAY columns from JDBC source. (Gabriel Reid)
-
-API changes
-* Add `ProjectRelBase.copy(RelTraitSet, RelNode, List&lt;RexNode&gt;,
-  RelDataType)` and make `ProjectRelBase.copy(RelTraitSet, RelNode)` final.
-  (**This is a breaking change** for sub-classes of `ProjectRelBase`.)
-* Change `RexBuilder.makeRangeReference` parameter type.
-* `RexBuilder.makeInputRef` replaces `RelOptUtil.createInputRef`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-160">CALCITE-160</a>]
-  Allow comments in schema definitions
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-147">CALCITE-147</a>]
-  Create a new kind of `SqlCall` that keeps operands in fields, not an operands
-  array
-  * Very widely used parse tree nodes with complex operands, including
-    `SqlSelect`, `SqlJoin`, `SqlInsert`, and a new node type `SqlOrderBy`, are
-    now sub-classes of `SqlCall` but not `SqlBasicCall`.
-  * (**This is a breaking change** to code that assumes that, say,
-    `SqlSelect` has an `operands` field.)
-* Convert all enum constants to upper-case.
-  (**This is a breaking change**.)
-
-Bug-fixes and internal changes
-* Generate optiq-core-VERSION-tests.jar not parent-VERSION-tests.jar.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-176">CALCITE-176</a>]
-  ORDER BY expression doesn't work with SELECT \*
-* Fix VARCHAR casts sent to hsqldb source (Bruno Dumon)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-143">CALCITE-143</a>]
-  Remove dependency on eigenbase-resgen
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-173">CALCITE-173</a>]
-  Case-insensitive table names are not supported for `Casing.UNCHANGED`
-* `DATE.getLimit` now returns `Calendar` in GMT time zone (Vladimir Sitnikov)
-* Set `en_US` locale in tests that match against error numbers, dates
-  (Vladimir Sitnikov)
-* Use 1 test thread per CPU to avoid thread starvation on dual core CPUs
-  (Vladimir Sitnikov)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-174">CALCITE-174</a>]
-  Move hsqldb to test scope
-* Add unit tests for `RexExecutorImpl`.
-* Correct JSON model examples in Javadoc comments. (Karel Vervaeke)
-* Move test reference logs from `src/test/java` to `src/test/resources`
-  (reduces the number of 'untracked files' reported by git)
-* Tune `Util.SpaceList`, fix race condition, and move into new utility class
-  `Spaces`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-163">CALCITE-163</a>]
-  Equi-join warning
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-157">CALCITE-157</a>]
-  Handle `SQLFeatureNotSupported` when calling `setQueryTimeout`
-  (Karel Vervaeke)
-* Fix Optiq on Windows. (All tests and checkstyle checks pass.)
-* In checkstyle, support Windows-style file separator, otherwise build fails in
-  Windows due to suppressions not used. (Vladimir Sitnikov)
-* Enable MongoDB tests when `-Dcalcite.test.mongodb=true`.
-* Cleanup cache exception-handling and an assert.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-153">CALCITE-153</a>]
-  Error using MongoDB adapter: Failed to set setXIncludeAware(true)
-* Disable spark engine unless Spark libraries are on the class path and
-  `spark=true` is specified in the connect string.
-* Fix path to `mongo-zips-model.json` in HOWTO. (Mariano Luna)
-* Fix bug deriving the type of a join-key.
-* Fix the value of `ONE_MINUS_EPSILON`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-158">CALCITE-158</a>]
-  Optiq fails when call `Planner.transform()` multiple times, each with
-  different ruleset
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-148">CALCITE-148</a>]
- Less verbose description of collation. Also, optimize `RelTraitSet` creation
- and amortize `RelTraitSet.toString()`.
-* Add generics to SQL parser.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-145">CALCITE-145</a>]
-  Unexpected upper-casing of keywords when using java lexer
-* Remove duplicate `maven-source-plugin`.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-141">CALCITE-141</a>]
-  Downgrade to guava-11.0.2. (This is necessary for Hadoop compatibility.
-  Later versions of Guava can also be used.)
-* Upgrade to spark-0.9.0. (Because this version of spark is available from
-  maven-central, we can make optiq-spark part of the regular build, and remove
-  the spark profile.)
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.18">0.4.18</a> / 2014-02-14
-
-API and functionality changes
-* Configurable lexical policy
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-33">CALCITE-33</a>]
-      SQL parser should allow different identifier quoting
-    * [<a href="https://issues.apache.org/jira/browse/CALCITE-34">CALCITE-34</a>]
-      Policy for case-sensitivity of identifiers should be configurable
-    * New connect-string parameter "lex", with allowable values
-      "ORACLE", "MYSQL", "SQL_SERVER", "JAVA" sets policy to be like those
-      databases, in terms of quote string, whether quoted and unquoted
-      identifiers are converted to upper/lower case, and whether
-      identifiers are matched case-sensitively. "JAVA" is case-sensitive,
-      even for unquoted identifiers. It should be possible
-      for each connection to have its own settings for these. Objects
-      shared between sessions (views, materialized views) might
-      require more work.
-    * Added various internals to make it easy for developers to do the
-      right thing. When you need to look up a schema, table or
-      column/field name, you should use a catalog reader, and it will
-      apply the right case-sensitivity policy.
-    * Enable optiq consumer to utilize different lexical settings in
-      Frameworks/Planner. (Jacques Nadeau)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-135">CALCITE-115</a>]
-  Add a PARSE_TREE hook point with SqlNode parameter
-* Change planner rules to use `ProjectFactory` for creating
-  projects. (John Pullokkaran)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-131">CALCITE-131</a>]
-  Add interfaces for metadata (statistics)
-  (**This is a breaking change**.)
-* Update Avatica to allow `Cursor` & `Accessor` implementations to throw
-  `SQLException`. (Jacques Nadeau)
-* Separate cost model (`RelOptCostFactory`) from planner. Allow
-  `VolcanoPlanner` to be sub-classed with different cost factory.
-    * Remove references to VolcanoCost from RelSubset, so clients can
-      use a different `RelOptCost`. (Harish Butani)
-    * Make `VolcanoCost` immutable.
-* Break `SqlTypeStrategies` into `OperandTypes`, `ReturnTypes` and
-  `InferTypes`, and rename its static members to upper-case, per
-  checkstyle. (**This is a breaking change**.)
-* Add a mechanism for defining configuration parameters and have them
-  appear in the responses to `AvaticaDatabaseMetaData` methods.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-113">CALCITE-113</a>]
-  User-defined scalar functions
-* Add rules to short-cut a query if `LIMIT 0` is present. Also remove
-  sort, aggregation, join if their inputs are known to be empty, and
-  propagate the fact that the relational expressions are known to be
-  empty up the tree. (We already do this for union, filter, project.)
-* `RexNode` and its sub-classes are now immutable.
-
-Bug fixes and internal changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-61">CALCITE-16</a>]
-  Upgrade to janino-2.7
-* Upgrade to guava-15.0 (guava-14.0.1 still allowed), sqlline-1.1.7,
-  maven-surefire-plugin-2.16, linq4j-0.1.13.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-136">CALCITE-136</a>]
-  Support Hive dialect
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-138">CALCITE-138</a>]
-  SqlDataTypeSpec.clone handles collection types wrong
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-137">CALCITE-137</a>]
-  If a subset is created that is subsumed by an existing subset, its
-  'best' is not assigned
-    * If best rel in a Volcano subset doesn't have metadata, see if
-      other rels have metadata.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-127">CALCITE-127</a>]
-  EnumerableCalcRel can't support 3+ AND conditions (Harish Butani)
-* Fix push-down of datetime literals to JDBC data sources.
-* Add `Util.startsWith(List, List)` and `Util.hashCode(double)`.
-* Add maven-checkstyle-plugin, enable in "verify" phase, and fix exceptions.
-* Fix `SqlValidator` to rely on `RelDataType` to do field name matching.  Fix
-  `RelDataTypeImpl` to correctly use the case sensitive flag rather than
-  ignoring it.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-119">CALCITE-119</a>]
-  Comparing Java type long with SQL type INTEGER gives wrong answer
-* Enable multi-threaded testing, and fix race conditions.
-    * Two of the race conditions involved involving trait caches. The
-      other was indeterminacy in type system when precision was not
-      specified but had a default; now we canonize TIME to TIME(0), for
-      instance.
-* Convert files to `us-ascii`.
-* Work around
-  [<a href="http://jira.codehaus.org/browse/JANINO-169">JANINO-169</a>].
-* Refactor SQL validator testing infrastructure so SQL parser is
-  configurable.
-* Add `optiq-mat-plugin` to README.
-* Fix the check for duplicate subsets in a rule match.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-112">CALCITE-112</a>]
-  Java boolean column should be treated as SQL boolean
-* Fix escaped unicode characters above 0x8000. Add tests for unicode
-  strings.
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.17">0.4.17</a> / 2014-01-13
-
-API changes
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-106">CALCITE-106</a>]
-  Make `Schema` and `Table` SPIs simpler to implement, and make them
-  re-usable across connections
-  (**This is a breaking change**.)
-* Make it easier to define sub-classes of rule operands. The new class
-  `RelOptRuleOperandChildren` contains the children of an operand and
-  the policy for dealing with them. Existing rules now use the new
-  methods to construct operands: `operand()`, `leaf()`, `any()`, `none()`,
-  `unordered()`. The previous methods are now deprecated and will be
-  removed before 0.4.18. (**This is a breaking change**.)
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-101">CALCITE-101</a>]
-  Enable phased access to the Optiq engine
-* List-handling methods in `Util`: add methods `skipLast`, `last`, `skip`;
-  remove `subList`, `butLast`.
-* Convert `SqlIdentifier.names` from `String[]` to `ImmutableList<String>`.
-* Rename `OptiqAssert.assertThat()` to `that()`, to avoid clash with junit's
-  `Assert.assertThat()`.
-* Usability improvements for `RelDataTypeFactory.FieldInfoBuilder`. It
-  now has a type-factory, so you can just call `build()`.
-* Rework `HepProgramBuilder` into a fluent API.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-105">CALCITE-105</a>]
-  Externalize RelNode to and from JSON
-
-Tuning
-* If `EnumerableAggregateRel` has no aggregate functions, generate a
-   call to `Enumerable.distinct()`, thereby saving the effort of
-   building trivial accumulators.
-* Default rule set now does not introduce `CalcRel` until a later phase
-  of planning. This reduces the number of trivial projects and calcs
-  created, merged, and elimated.
-* Reduce the amount of time spent creating record types that
-  already exist.
-* More efficient implementation of `Util.isDistinct` for small lists.
-* When an internal record has 0 fields, rather than generating a
-  synthetic class and lots of instances that are all the same, use the
-  new `Unit` class, which is a singleton.
-* To take advantage of asymmetric hash join added recently in linq4j,
-  tweak cost of `EnumerableJoinRel` so that join is cheaper if the
-  larger input is on the left, and more expensive if it is a cartesian
-  product.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-70">CALCITE-70</a>]
-  Joins seem to be very expensive in memory
-* Make planning process more efficient by not sorting the list of
-  matched rules each cycle. It is sorted if tracing is enabled;
-  otherwise we scan to find the most important element. For this list,
-  replace `LinkedList` with `ChunkList`, which has an O(1) remove and add,
-  a fast O(n) get, and fast scan.
-
-Other
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-87">CALCITE-87</a>]
-  Constant folding
-  * Rules for constant-expression reduction, and to simplify/eliminate
-    `VALUES` operator.
-* Graph algorithms: Implement breadth-first iterator and cycle-detector.
-* Fix bug in planner which occurred when two `RelNode`s have identical
-  digest but different row-type.
-* Fix link to optiq-csv tutorial.
-* Fix bugs in `RemoveTrivialProjectRule.strip`, `JdbcProjectRel.implement`
-  and `SortRel.computeSelfCost`.
-* Reformat code, and remove `@author` tags.
-* Upgrade to eigenbase-xom-1.3.4, eigenbase-properties-1.1.4,
-  eigenbase-resgen-1.3.6.
-* Upgrade to linq4j-0.1.12.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-97">CALCITE-97</a>]
-  Correlated EXISTS
-* Fix a bug in `VolcanoCost`.
-* Add class `FoodMartQuerySet`, that contains the 6,700 foodmart queries.
-* Fix factory class names in `UnregisteredDriver`
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-96">CALCITE-96</a>]
-  LIMIT against a table in a clone schema causes UnsupportedOperationException
-* Disable spark module by default.
-* Allow `CloneSchema` to be specified in terms of url, driver, user,
-  password; not just dataSource.
-* Wrap internal error in `SQLException`.
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.16">0.4.16</a> / 2013-11-24
-
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-69">CALCITE-69</a>]
-  Can't join on string columns and other problems with expressions in the join
-  condition
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-74">CALCITE-74</a>]
-  JOIN ... USING fails in 3-way join with UnsupportedOperationException.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-65">CALCITE-65</a>]
-  Fix issues in the JDBC driver, and in particular to DatabaseMetaData methods,
-  to make Squirrel-SQL run better.
-* Fix JDBC column, table, schema names for when the table is not in a schema of
-  depth 1.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-85">CALCITE-85</a>]
-  Adding a table to the root schema causes breakage in OptiqPrepareImpl
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-84">CALCITE-84</a>]
-  Extract Optiq's JDBC driver as a new JDBC driver framework, Avatica.
-  Other projects can use this to implement a JDBC driver by implementing
-  just a few methods. If you wish to use Optiq's JDBC driver, you will
-  now need to include optiq-avatica.jar in addition to optiq-core.jar.
-  Avatica does not depend on anything besides the standard Java library.
-* Support for parameters in PreparedStatement.
-* First steps in recognizing complex materializations. Internally we introduce a
-  concept called a "star table", virtual table composed of real tables joined
-  together via many-to-one relationships. The queries that define
-  materializations and end-user queries are canonized in terms of star tables.
-  Matching (not done yet) will then be a matter of looking for sort, groupBy,
-  project. It is not yet possible to define a star in an Optiq model file.
-* Add section to <a href="HOWTO.md">HOWTO</a> on implementing adapters.
-* Fix data type conversions when creating a clone table in memory.
-* Fix how strings are escaped in JsonBuilder.
-* Test suite now depends on an embedded hsqldb database, so you can run
-  <code>mvn test</code> right after pulling from git. You can instead use a
-  MySQL database if you specify '-Dcalcite.test.db=mysql', but you need to
-  manually populate it.
-* Fix a planner issue which occurs when the left and right children of join are
-  the same relational expression, caused by a self-join query.
-* [<a href="https://issues.apache.org/jira/browse/CALCITE-76">CALCITE-76</a>]
-  Precedence of the item operator, <code>map[index]</code>; remove the space
-  before '[' when converting parse tree to string.
-* Allow <code>CAST(expression AS ANY)</code>, and fix an issue with the ANY type
-  and NULL values.
-* Handle null timestamps and dates coming out of JDBC adapter.
-* Add <code>jdbcDriver</code> attribute to JDBC schema in model, for drivers
-  that do not auto-register.
-* Allow join rules to match any subclass of JoinRelBase.
-* Push projects, filters and sorts down to MongoDB. (Fixes
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-57">CALCITE-57</a>],
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-60">CALCITE-60</a>] and
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-72">CALCITE-72</a>].)
-* Add instructions for loading FoodMart data set into MongoDB, and how to enable
-  tracing.
-* Now runs on JDK 1.8 (still runs on JDK 1.6 and JDK 1.7).
-* Upgrade to junit-4.11 (avoiding the dodgy junit-4.1.12).
-* Upgrade to linq4j-0.1.11.
-
-## <a href="https://github.com/apache/incubator-calcite/releases/tag/optiq-parent-0.4.15">0.4.15</a> / 2013-10-14
-
-* Lots of good stuff that this margin is too small to contain. See
-  <a href="REFERENCE.md">SQL language reference</a> and
-  <a href="MODEL.md">JSON model reference</a>.
-
-# Optiq-csv release history
-
-Optiq-csv-0.3 was the last independent release of optiq-csv. From
-calcite-0.9.2 onwards, the code was included as the
-calcite-example-csv module.
-
-* Upgrade to calcite-0.9.1
-* Support gzip-compressed CSV and JSON files (recognized by '.gz' suffix)
-* Cleanup, and fix minor timezone issue in a test
-* Support for date types (date, time, timestamp) (Martijn van den Broek)
-* Upgrade to optiq-0.8, optiq-avatica-0.8, linq4j-0.4
-* Add support for JSON files (recognized by '.json' suffix)
-* Upgrade maven-release-plugin to version 2.4.2
-* Upgrade to optiq-0.6, linq4j-0.2
-* Add NOTICE and LICENSE files in generated JAR file
-
-## <a href="https://github.com/julianhyde/optiq-csv/releases/tag/optiq-csv-0.3">0.3</a> / 2014-03-21
-
-* Upgrade to optiq-0.5
-* Add workaround to
-  [<a href="https://github.com/jline/jline2/issues/62">JLINE2-62</a>]
-  to `sqlline.bat` (windows) and `sqlline` (windows using cygwin)
-* Fix classpath construction: `sqlline.bat` copies dependencies to
-  `target/dependencies`; `sqlline` constructs `target/classpath.txt`
-* Build, checkstyle and tests now succeed on windows (both native and cygwin)
-* Models can now contain comments
-* [<a href="https://github.com/julianhyde/optiq-csv/issues/2">OPTIQ-CSV-2</a>]
-  Update tutorial to reflect changes to Optiq's JDBC adapter
-
-## <a href="https://github.com/julianhyde/optiq-csv/releases/tag/optiq-csv-0.2">0.2</a> / 2014-02-18
-
-* Add test case for
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-112">CALCITE-112</a>]
-* Add `sqlline.bat`, Windows SQL shell (based on fix for
-  [<a href="https://issues.apache.org/jira/browse/DRILL-338">DRILL-338</a>])
-* Upgrade to optiq-0.4.18, sqlline-1.1.7
-* Return a single object for single-col enumerator (Gabriel Reid)
-* Enable maven-checkstyle-plugin; fix checkstyle exceptions
-
-## <a href="https://github.com/julianhyde/optiq-csv/releases/tag/optiq-csv-0.1">0.1</a> / 2014-01-13
-
-* Add release notes and history
-* Enable maven-release-plugin
-* Upgrade to optiq-0.4.17, linq4j-0.1.12, sqlline-1.1.6
-* Upgrade tutorial for new Schema and Table SPIs
-* Fixes for optiq SPI changes in
-  [<a href="https://issues.apache.org/jira/browse/CALCITE-106">CALCITE-106</a>]
-* Enable oraclejdk8 in Travis CI
-* Fix bug where non-existent directory would give NPE; instead print warning
-* Add an example of a planner rule
-* Add `CsvTableFactory`, an example of a custom table
-* Add a view to tutorial
-* Split into scenario with a "simple" schema that generates tables
-  (`CsvTable`) that just execute and a "smart" schema that generates
-  tables (`CsvSmartTable`) that undergo optimization
-* Make `CsvEnumerator` a top-level class
-* Implement the algorithms to sniff names and types from the first
-  row, and to return an enumerator of all rows
-* Read column types from header of CSV file
-
-# Linq4j release history
-
-Linq4j-0.4 was the last independent release of linq4j. From
-calcite-0.9.2 onwards, the code was included as calcite-linq4j, and
-features added to linq4j in a particular calcite release are described
-with the other changes in that release.
-
-## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.4">0.4</a> / 2014-05-28
-
-* Fix <a href="https://github.com/julianhyde/linq4j/issues/27">#27</a>,
-  "Incorrectly inlines non-final variable".
-* Maven build process now deploys web site.
-* Implement `Enumerable` methods: `any`, `all`,
-  `contains` with `EqualityComparer`, `first`, `first` with predicate.
-
-## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.3">0.3</a> / 2014-04-21
-
-* Move optimizer visitor from optiq to linq4j; add
-  `ExpressionType.modifiesLvalue` to avoid invalid inlining.
-* Fix <a href="https://github.com/julianhyde/linq4j/issues/17">#17</a>,
-  "Assign constant expressions to 'static final' members";
-  add `@Deterministic` annotation to help deduce which expressions are
-  constant.
-* Multi-pass optimization: some of the variables might be avoided and
-  inlined after the first pass.
-* Various other peephole optimizations: `Boolean.valueOf(const)`,
-  'not' expressions (`!const`, `!!a`, `!(a==b)`, `!(a!=b)`, `!(a>b)`,
-  etc.),
-  '?' expressions coming from `CASE` (`a ? booleanConstant : b` and `a
-  ? b : booleanConstant`).
-* Implement left, right and full outer join.
-* Clean build on cygwin/Windows.
-
-## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.2">0.2</a> / 2014-04-11
-
-* Fix <a href="https://github.com/julianhyde/linq4j/issues/8">#8</a>,
-  "Javadoc generation fails under JDK 1.8".
-* Fix <a href="https://github.com/julianhyde/linq4j/issues/15">#15</a>,
-  "`Expressions.ifThenElse` does not work".
-* Use `HashMap` for searching of declarations to reuse; consider both
-  `optimizing` and `optimize` flags when reusing.
-* Implement `equals` and `hashCode` for expressions. Hash codes for
-  complex expressions are cached into a field of the expression.
-* Add example, `com.example.Linq4jExample`.
-* Fix optimizing away parameter declarations in assignment target.
-* Support Windows path names in checkstyle-suppresions.
-* Support `Statement.toString` via `ExpressionWriter`.
-* Use `AtomicInteger` for naming of `ParameterExpression`s to avoid
-  conflicts in multithreaded usage
-* Cleanup: use `Functions.adapt` rather than `new AbstractList`
-* Add `NOTICE` and `LICENSE` files in generated JAR file.
-* Optimize `select()` if selector is identity.
-* Enable checkstyle.
-
-## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.1.13">0.1.13</a> / 2014-01-20
-
-* Remove spurious "null" generated when converting expression to string.
-* Allow a field declaration to not have an initializer.
-* Add `Primitive.defaultValue`.
-* Enable `oraclejdk8` in <a href="https://travis-ci.org/julianhyde/linq4j">Travis CI</a>.
-
-## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.1.12">0.1.12</a> / 2013-12-07
-
-* Add release notes.
-* Fix implementation of `Enumerable.asEnumerable` in
-  `DefaultQueryable` (inherited by most classes that implement
-  `Queryable`).
-
-## <a href="https://github.com/julianhyde/linq4j/releases/tag/linq4j-0.1.11">0.1.11</a> / 2013-11-06
-
-* Initial commit
-


[10/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
[CALCITE-722] Rename markdown files to lower-case


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

Branch: refs/heads/master
Commit: 06a192a0afaf8de758d55aaf3e4858733df6c55e
Parents: 85887fe
Author: Julian Hyde <jh...@apache.org>
Authored: Mon May 11 17:41:24 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon May 11 17:46:24 2015 -0700

----------------------------------------------------------------------
 README                                          |    4 +-
 README.md                                       |   23 +-
 .../org/apache/calcite/model/package-info.java  |    3 +-
 doc/HISTORY.md                                  | 1475 ------------------
 doc/HOWTO.md                                    |  801 ----------
 doc/INCUBATOR_PROPOSAL.md                       |  357 -----
 doc/MODEL.md                                    |  409 -----
 doc/REFERENCE.md                                |  569 -------
 doc/STREAM.md                                   |  631 --------
 doc/TUTORIAL.md                                 |  753 ---------
 doc/history.md                                  | 1475 ++++++++++++++++++
 doc/howto.md                                    |  801 ++++++++++
 doc/model.md                                    |  409 +++++
 doc/reference.md                                |  569 +++++++
 doc/stream.md                                   |  631 ++++++++
 doc/tutorial.md                                 |  753 +++++++++
 .../org/apache/calcite/test/MongoAdapterIT.java |    2 +-
 17 files changed, 4654 insertions(+), 5011 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/README
----------------------------------------------------------------------
diff --git a/README b/README
index 600b76f..37e52b4 100644
--- a/README
+++ b/README
@@ -2,14 +2,14 @@ Apache Calcite release 1.2.0 (incubating)
 
 This is a source or binary distribution of Apache Calcite.
 
-Changes since the previous release are described in the doc/HISTORY.md
+Changes since the previous release are described in the doc/history.md
 file.
 
 The LICENSE and NOTICE files contain license information.
 
 If this is a source distribution, you can find instructions how to
 build the release in the "Building from a source release" section in
-doc/HOWTO.md.
+doc/howto.md.
 
 README.md contains examples of running Calcite.
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index bf00288..61e210d 100644
--- a/README.md
+++ b/README.md
@@ -25,9 +25,9 @@ It was formerly called Optiq.
 ## Getting Calcite
 
 To run Apache Calcite, you can either
-[download and build from github](doc/HOWTO.md#building-from-git),
+[download and build from github](doc/howto.md#building-from-git),
 or [download a release](http://www.apache.org/dyn/closer.cgi/incubator/calcite)
-then [build the source code](doc/HOWTO.md#building-from-a-source-distribution).
+then [build the source code](doc/howto.md#building-from-a-source-distribution).
 
 Pre-built jars are in
 [the Apache maven repository](https://repository.apache.org/content/repositories/releases)
@@ -164,10 +164,10 @@ The subproject under example/csv provides a CSV adapter, which is fully function
 but is also simple enough to serve as a good template if you are writing
 your own adapter.
 
-See the <a href="https://github.com/apache/incubator-calcite/blob/master/doc/TUTORIAL.md">csv tutorial</a>
-for information on using csv adapter and writing other adapters.
+See the <a href="https://github.com/apache/incubator-calcite/blob/master/doc/tutorial.md">CSV tutorial</a>
+for information on using the CSV adapter and writing other adapters.
 
-See the <a href="doc/HOWTO.md">HOWTO</a> for more information about using other
+See the <a href="doc/howto.md">HOWTO</a> for more information about using other
 adapters, and about using Calcite in general.
 
 ## Status
@@ -181,7 +181,7 @@ The following features are complete.
 * <a href="https://github.com/julianhyde/linq4j">Linq4j</a> front-end
 * SQL features: SELECT, FROM (including JOIN syntax), WHERE, GROUP BY (and aggregate functions including COUNT(DISTINCT ...)), HAVING, ORDER BY (including NULLS FIRST/LAST), set operations (UNION, INTERSECT, MINUS), sub-queries (including correlated sub-queries), windowed aggregates, LIMIT (syntax as <a href="http://www.postgresql.org/docs/8.4/static/sql-select.html#SQL-LIMIT">Postgres</a>)
 
-For more details, see the <a href="doc/REFERENCE.md">Reference guide</a>.
+For more details, see the <a href="doc/reference.md">Reference guide</a>.
 
 ### Drivers
 
@@ -210,11 +210,12 @@ For more details, see the <a href="doc/REFERENCE.md">Reference guide</a>.
   (<a href="http://mail-archives.apache.org/mod_mbox/incubator-calcite-dev/">archive</a>,
   <a href="mailto:dev-subscribe@calcite.incubator.apache.org">subscribe</a>)
 * Twitter: <a href="https://twitter.com/ApacheCalcite">@ApacheCalcite</a>
-* <a href="doc/HOWTO.md">HOWTO</a>
-* <a href="doc/MODEL.md">JSON model</a>
-* <a href="doc/REFERENCE.md">Reference guide</a>
-* <a href="doc/STREAM.md">Streaming SQL</a>
-* <a href="doc/HISTORY.md">Release notes and history</a>
+* <a href="doc/howto.md">HOWTO</a>
+* <a href="doc/model.md">JSON model</a>
+* <a href="doc/reference.md">Reference guide</a>
+* <a href="doc/stream.md">Streaming SQL</a>
+* <a href="doc/avatica.md">Avatica JDBC framework</a>
+* <a href="doc/history.md">Release notes and history</a>
 
 ### Pre-Apache resources
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/core/src/main/java/org/apache/calcite/model/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/package-info.java b/core/src/main/java/org/apache/calcite/model/package-info.java
index ce16b52..b792f65 100644
--- a/core/src/main/java/org/apache/calcite/model/package-info.java
+++ b/core/src/main/java/org/apache/calcite/model/package-info.java
@@ -31,8 +31,7 @@
  * interfaces and including a custom schema in the model.</p>
  *
  * <p>There are several examples of schemas in the
- * <a href="https://github.com/julianhyde/blog/master/optiq-csv/TUTORIAL.md">optiq-csv
- * tutorial</a>.
+ * <a href="https://github.com/apache/incubator-calcite/blob/master/doc/tutorial.md">tutorial</a>.
  */
 @PackageMarker
 package org.apache.calcite.model;


[05/13] incubator-calcite git commit: [CALCITE-722] Rename markdown files to lower-case

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/howto.md
----------------------------------------------------------------------
diff --git a/doc/howto.md b/doc/howto.md
new file mode 100644
index 0000000..60a4cae
--- /dev/null
+++ b/doc/howto.md
@@ -0,0 +1,801 @@
+<!--
+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.
+-->
+# Calcite HOWTO
+
+Here's some miscellaneous documentation about using Calcite and its various
+adapters.
+
+## Building from a source distribution
+
+Prerequisites are maven (3.2.1 or later)
+and Java (JDK 1.7 or later, 1.8 preferred) on your path.
+
+Unpack the source distribution `.tar.gz` or `.zip` file,
+`cd` to the root directory of the unpacked source,
+then build using maven:
+
+```bash
+$ tar xvfz calcite-1.2.0-incubating-source.tar.gz
+$ cd calcite-1.2.0-incubating
+$ mvn install
+```
+
+[Running tests](howto.md#running-tests) describes how to run more or fewer
+tests.
+
+## Building from git
+
+Prerequisites are git, maven (3.2.1 or later)
+and Java (JDK 1.7 or later, 1.8 preferred) on your path.
+
+Create a local copy of the github repository,
+`cd` to its root directory,
+then build using maven:
+
+```bash
+$ git clone git://github.com/apache/incubator-calcite.git
+$ cd incubator-calcite
+$ mvn install
+```
+
+[Running tests](howto.md#running-tests) describes how to run more or fewer
+tests.
+
+## Running tests
+
+The test suite will run by default when you build, unless you specify
+`-DskipTests`:
+
+```bash
+$ mvn clean # Note: mvn clean install does not work, use mvn clean && mvn install
+$ mvn -DskipTests install
+```
+
+There are other options that control which tests are run, and in what
+environment, as follows.
+
+* `-Dcalcite.test.db=DB` (where db is `h2`, `hsqldb`, `mysql`, or `postgresql`) allows you
+  to change the JDBC data source for the test suite. Calcite's test
+  suite requires a JDBC data source populated with the foodmart data
+  set.
+   * `hsqldb`, the default, uses an in-memory hsqldb database.
+   * all others access test virtual machine (see [integration tests](howto.md#running-integration-tests) below)
+     `mysql` and `postgresql` might be somewhat faster than hsqldb, but you need to populate it (i.e. provision a VM).
+* `-Dcalcite.debug` prints extra debugging information to stdout.
+* `-Dcalcite.test.slow` enables tests that take longer to execute. For
+  example, there are tests that create virtual TPC-H and TPC-DS schemas
+  in-memory and run tests from those benchmarks.
+* `-Dcalcite.test.splunk=true` enables tests that run against Splunk.
+  Splunk must be installed and running.
+
+## Running integration tests
+
+For testing Calcite's external adapters, a test virtual machine should be used.
+The VM includes H2, HSQLDB, MySQL, MongoDB, and PostgreSQL.
+
+Test VM requires 5GiB of disk space and it takes 30 minutes to build.
+
+Note: you can use [calcite-test-dataset](https://github.com/vlsi/calcite-test-dataset)
+ to populate your own database, however it is recommended to use test VM so the test environment can be reproduced.
+
+### VM preparation
+
+0) Install dependencies: [Vagrant](https://www.vagrantup.com/) and [VirtualBox](https://www.virtualbox.org/)
+
+1) Clone https://github.com/vlsi/calcite-test-dataset.git at the same level as calcite repository.
+For instance:
+```bash
+code
+  +-- calcite
+  +-- calcite-test-dataset
+```
+
+Note: integration tests search for ../calcite-test-dataset or ../../calcite-test-dataset.
+ You can specify full path via calcite.test.dataset system property.
+
+2) Build and start the VM:
+```bash
+cd calcite-test-dataset && mvn install
+```
+
+### VM management
+
+Test VM is provisioned by Vagrant, so regular Vagrant `vagrant up` and `vagrant halt` should be used to start and stop the VM.
+The connection strings for different databases are listed in [calcite-test-dataset](https://github.com/vlsi/calcite-test-dataset) readme.
+
+### Suggested test flow
+
+Note: test VM should be started before you launch integration tests. Calcite itself does not start/stop the VM.
+
+Command line:
+* Executing regular unit tests (does not require external data): no change. `mvn test` or `mvn install`.
+* Executing all tests, for all the DBs: `mvn verify -Pit`. `it` stands for "integration-test". `mvn install -Pit` works as well.
+* Executing just tests for external DBs, excluding unit tests: `mvn -Dtest=foo -DfailIfNoTests=false -Pit verify`
+* Executing just MongoDB tests: `cd mongo; mvn verify -Pit`
+
+From within IDE:
+* Executing regular unit tests: no change.
+* Executing MongoDB tests: run `MongoAdapterIT.java` as usual (no additional properties are required)
+* Executing MySQL tests: run `JdbcTest` and `JdbcAdapterTest` with setting `-Dcalcite.test.db=mysql`
+* Executing PostgreSQL tests: run `JdbcTest` and `JdbcAdapterTest` with setting `-Dcalcite.test.db=postgresql`
+
+### Integration tests technical details
+
+Tests with external data are executed at maven's integration-test phase.
+We do not currently use pre-integration-test/post-integration-test, however we could use that in future.
+The verification of build pass/failure is performed at verify phase.
+Integration tests should be named `...IT.java`, so they are not picked up on unit test execution.
+
+## Contributing
+
+We welcome contributions.
+
+If you are planning to make a large contribution, talk to us first! It
+helps to agree on the general approach. Log a
+[JIRA case](https://issues.apache.org/jira/browse/CALCITE) for your
+proposed feature or start a discussion on the dev list.
+
+Fork the github repository, and create a branch for your feature.
+
+Develop your feature and test cases, and make sure that `mvn
+install` succeeds. (Run extra tests if your change warrants it.)
+
+Commit your change to your branch, and use a comment that starts with
+the JIRA case number, like this:
+
+```
+[CALCITE-345] AssertionError in RexToLixTranslator comparing to date literal
+```
+
+If your change had multiple commits, use `git rebase -i master` to
+combine them into a single commit, and to bring your code up to date
+with the latest on the main line.
+
+Then push your commit(s) to github, and create a pull request from
+your branch to the incubator-calcite master branch. Update the JIRA case
+to reference your pull request, and a committer will review your
+changes.
+
+## Getting started
+
+Calcite is a community, so the first step to joining the project is to introduce yourself.
+Join the [developers list](http://mail-archives.apache.org/mod_mbox/incubator-calcite-dev/)
+and send an email.
+
+If you have the chance to attend a [meetup](http://www.meetup.com/Apache-Calcite/),
+or meet [members of the community](http://calcite.incubator.apache.org/team-list.html)
+at a conference, that's also great.
+
+Choose an initial task to work on. It should be something really simple,
+such as a bug fix or a [Jira task that we have labeled
+"newbie"](https://issues.apache.org/jira/issues/?jql=labels%20%3D%20newbie%20%26%20project%20%3D%20Calcite%20%26%20status%20%3D%20Open).
+Follow the [contributing guidelines](#contributing) to get your change committed.
+
+After you have made several useful contributions we may
+[invite you to become a committer](https://community.apache.org/contributors/).
+We value all contributions that help to build a vibrant community, not just code.
+You can contribute by testing the code, helping verify a release,
+writing documentation or the web site,
+or just by answering questions on the list.
+
+## Tracing
+
+To enable tracing, add the following flags to the java command line:
+
+```
+-Dcalcite.debug=true -Djava.util.logging.config.file=core/src/test/resources/logging.properties
+```
+
+The first flag causes Calcite to print the Java code it generates
+(to execute queries) to stdout. It is especially useful if you are debugging
+mysterious problems like this:
+
+```
+Exception in thread "main" java.lang.ClassCastException: Integer cannot be cast to Long
+  at Baz$1$1.current(Unknown Source)
+```
+
+The second flag specifies a config file for
+the <a href="http://docs.oracle.com/javase/7/docs/api/java/util/logging/package-summary.html">java.util.logging</a>
+framework. Put the following into core/src/test/resources/logging.properties:
+
+```properties
+handlers= java.util.logging.ConsoleHandler
+.level= INFO
+org.apache.calcite.plan.RelOptPlanner.level=FINER
+java.util.logging.ConsoleHandler.level=ALL
+```
+
+The line `org.apache.calcite.plan.RelOptPlanner.level=FINER` tells the planner to produce
+fairly verbose output. You can modify the file to enable other loggers, or to change levels.
+For instance, if you change `FINER` to `FINEST` the planner will give you an account of the
+planning process so detailed that it might fill up your hard drive.
+
+## CSV adapter
+
+See the <a href="tutorial.md">tutorial</a>.
+
+## MongoDB adapter
+
+First, download and install Calcite,
+and <a href="http://www.mongodb.org/downloads">install MongoDB</a>.
+
+Note: you can use MongoDB from integration test virtual machine above.
+
+Import MongoDB's zipcode data set into MongoDB:
+
+```bash
+$ curl -o /tmp/zips.json http://media.mongodb.org/zips.json
+$ mongoimport --db test --collection zips --file /tmp/zips.json
+Tue Jun  4 16:24:14.190 check 9 29470
+Tue Jun  4 16:24:14.469 imported 29470 objects
+```
+
+Log into MongoDB to check it's there:
+
+```bash
+$ mongo
+MongoDB shell version: 2.4.3
+connecting to: test
+> db.zips.find().limit(3)
+{ "city" : "ACMAR", "loc" : [ -86.51557, 33.584132 ], "pop" : 6055, "state" : "AL", "_id" : "35004" }
+{ "city" : "ADAMSVILLE", "loc" : [ -86.959727, 33.588437 ], "pop" : 10616, "state" : "AL", "_id" : "35005" }
+{ "city" : "ADGER", "loc" : [ -87.167455, 33.434277 ], "pop" : 3205, "state" : "AL", "_id" : "35006" }
+> exit
+bye
+```
+
+Connect using the
+<a href="https://github.com/apache/incubator-calcite/blob/master/mongodb/src/test/resources/mongo-zips-model.json">mongo-zips-model.json</a>
+Calcite model:
+```bash
+$ ./sqlline
+sqlline> !connect jdbc:calcite:model=mongodb/target/test-classes/mongo-zips-model.json admin admin
+Connecting to jdbc:calcite:model=mongodb/target/test-classes/mongo-zips-model.json
+Connected to: Calcite (version 1.x.x)
+Driver: Calcite JDBC Driver (version 1.x.x)
+Autocommit status: true
+Transaction isolation: TRANSACTION_REPEATABLE_READ
+sqlline> !tables
++------------+--------------+-----------------+---------------+
+| TABLE_CAT  | TABLE_SCHEM  |   TABLE_NAME    |  TABLE_TYPE   |
++------------+--------------+-----------------+---------------+
+| null       | mongo_raw    | zips            | TABLE         |
+| null       | mongo_raw    | system.indexes  | TABLE         |
+| null       | mongo        | ZIPS            | VIEW          |
+| null       | metadata     | COLUMNS         | SYSTEM_TABLE  |
+| null       | metadata     | TABLES          | SYSTEM_TABLE  |
++------------+--------------+-----------------+---------------+
+sqlline> select count(*) from zips;
++---------+
+| EXPR$0  |
++---------+
+| 29467   |
++---------+
+1 row selected (0.746 seconds)
+sqlline> !quit
+Closing: org.apache.calcite.jdbc.FactoryJdbc41$CalciteConnectionJdbc41
+$
+```
+
+## Splunk adapter
+
+To run the test suite and sample queries against Splunk,
+load Splunk's `tutorialdata.zip` data set as described in
+<a href="http://docs.splunk.com/Documentation/Splunk/6.0.2/PivotTutorial/GetthetutorialdataintoSplunk">the Splunk tutorial</a>.
+
+(This step is optional, but it provides some interesting data for the sample
+queries. It is also necessary if you intend to run the test suite, using
+`-Dcalcite.test.splunk=true`.)
+
+## Implementing an adapter
+
+New adapters can be created by implementing `CalcitePrepare.Context`:
+
+```java
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteRootSchema;
+
+public class AdapterContext implements CalcitePrepare.Context {
+  @Override
+  public JavaTypeFactory getTypeFactory() {
+    // adapter implementation
+    return typeFactory;
+  }
+
+  @Override
+  public CalciteRootSchema getRootSchema() {
+    // adapter implementation
+    return rootSchema;
+  }
+}
+```
+
+### Testing adapter in Java
+
+The example below shows how SQL query can be submitted to
+`CalcitePrepare` with a custom context (`AdapterContext` in this
+case). Calcite prepares and implements the query execution, using the
+resources provided by the `Context`. `CalcitePrepare.PrepareResult`
+provides access to the underlying enumerable and methods for
+enumeration. The enumerable itself can naturally be some adapter
+specific implementation.
+
+```java
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.junit.Test;
+
+public class AdapterContextTest {
+  @Test
+  public void testSelectAllFromTable() {
+    AdapterContext ctx = new AdapterContext();
+    String sql = "SELECT * FROM TABLENAME";
+    Class elementType = Object[].class;
+    CalcitePrepare.PrepareResult<Object> prepared =
+        new CalcitePrepareImpl().prepareSql(ctx, sql, null, elementType, -1);
+    Object enumerable = prepared.getExecutable();
+    // etc.
+  }
+}
+```
+
+### JavaTypeFactory
+
+When Calcite compares types (instances of `RelDataType`), it requires them to be the same
+object. If there are two distinct type instances that refer to the
+same Java type, Calcite may fail to recognize that they match.  It is
+recommended to:
+* Use a single instance of `JavaTypeFactory` within the calcite context;
+* Store the types so that the same object is always returned for the same type.
+
+## Set up PGP signing keys (for Calcite committers)
+
+Follow instructions at http://www.apache.org/dev/release-signing to
+create a key pair. (On Mac OS X, I did `brew install gpg` and `gpg
+--gen-key`.)
+
+Add your public key to the `KEYS` file by following instructions in
+the `KEYS` file.
+
+## Making a snapshot (for Calcite committers)
+
+Before you start:
+* Set up signing keys as described above.
+* Make sure you are using JDK 1.7 (not 1.8).
+* Make sure build and tests succeed with `-Dcalcite.test.db=hsqldb` (the default)
+
+```bash
+# Set passphrase variable without putting it into shell history
+read -s GPG_PASSPHRASE
+
+# Make sure that there are no junk files in the sandbox
+git clean -xn
+mvn clean
+
+mvn -Papache-release -Dgpg.passphrase=${GPG_PASSPHRASE} install
+```
+
+When the dry-run has succeeded, change `install` to `deploy`.
+
+## Making a release (for Calcite committers)
+
+Before you start:
+* Set up signing keys as described above.
+* Make sure you are using JDK 1.7 (not 1.8).
+* Check that `README`, `README.md` and `doc/howto.md` have the correct version number.
+* Make sure build and tests succeed, including with
+  -Dcalcite.test.db={mysql,hsqldb}, -Dcalcite.test.slow=true,
+  -Dcalcite.test.mongodb=true, -Dcalcite.test.splunk=true.
+* Trigger a
+  <a href="https://scan.coverity.com/projects/2966">Coverity scan</a>
+  by merging the latest code into the `julianhyde/coverity_scan` branch,
+  and when it completes, make sure that there are no important issues.
+* Make sure that
+  <a href="https://issues.apache.org/jira/issues/?jql=project%20%3D%20CALCITE%20AND%20status%20%3D%20Resolved%20and%20fixVersion%20is%20null">
+  every "resolved" JIRA case</a> (including duplicates) has
+  a fix version assigned (most likely the version we are
+  just about to release)
+
+Create a release branch named after the release, e.g. `branch-1.1`, and push it to Apache.
+
+```bash
+$ git checkout -b branch-X.Y
+$ git push -u origin branch-X.Y
+
+We will use the branch for the entire the release process. Meanwhile,
+we do not allow commits to the master branch. After the release is
+final, we can use `git merge --ff-only` to append the changes on the
+release branch onto the master branch. (Apache does not allow reverts
+to the master branch, which makes it difficult to clean up the kind of
+messy commits that inevitably happen while you are trying to finalize
+a release.)
+
+Now, set up your environment and do a dry run. The dry run will not
+commit any changes back to git and gives you the opportunity to verify
+that the release process will complete as expected.
+
+If any of the steps fail, clean up (see below), fix the problem, and
+start again from the top.
+
+```bash
+# Set passphrase variable without putting it into shell history
+read -s GPG_PASSPHRASE
+
+# Make sure that there are no junk files in the sandbox
+git clean -xn
+mvn clean
+
+# Do a dry run of the release:prepare step, which sets version numbers.
+mvn -DdryRun=true -DskipTests -DreleaseVersion=X.Y.Z-incubating -DdevelopmentVersion=X.Y.Z+1-incubating-SNAPSHOT -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:prepare 2>&1 | tee /tmp/prepare-dry.log
+```
+
+Check the artifacts:
+* In the `target` directory should be these 8 files, among others:
+  * apache-calcite-X.Y.Z-incubating-src.tar.gz
+  * apache-calcite-X.Y.Z-incubating-src.tar.gz.asc
+  * apache-calcite-X.Y.Z-incubating-src.tar.gz.md5
+  * apache-calcite-X.Y.Z-incubating-src.tar.gz.sha1
+  * apache-calcite-X.Y.Z-incubating-src.zip
+  * apache-calcite-X.Y.Z-incubating-src.zip.asc
+  * apache-calcite-X.Y.Z-incubating-src.zip.md5
+  * apache-calcite-X.Y.Z-incubating-src.zip.sha1
+* Note that the file names start `apache-calcite-` and include
+  `incubating` in the version.
+* In the two source distros `.tar.gz` and `.zip` (currently there is
+  no binary distro), check that all files belong to a directory called
+  `apache-calcite-X.Y.Z-incubating-src`.
+* That directory must contain files `DISCLAIMER`, `NOTICE`, `LICENSE`,
+  `README`, `README.md`
+  * Check that the version in `README` is correct
+* In each .jar (for example
+  `core/target/calcite-core-X.Y.Z-incubating.jar` and
+  `mongodb/target/calcite-mongodb-X.Y.Z-incubating-sources.jar`), check
+  that the `META-INF` directory contains `DEPENDENCIES`, `LICENSE`,
+  `NOTICE` and `git.properties`
+* In each .jar, check that `org-apache-calcite-jdbc.properties` is
+  present and does not contain un-substituted `${...}` variables
+* Check PGP, per https://httpd.apache.org/dev/verification.html
+
+Now, remove the `-DdryRun` flag and run the release for real.
+
+```bash
+# Prepare sets the version numbers, creates a tag, and pushes it to git.
+mvn -DdryRun=false -DskipTests -DreleaseVersion=X.Y.Z-incubating -DdevelopmentVersion=X.Y.Z+1-incubating-SNAPSHOT -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:prepare 2>&1 | tee /tmp/prepare.log
+
+# Perform checks out the tagged version, builds, and deploys to the staging repository
+mvn -DskipTests -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:perform 2>&1 | tee /tmp/perform.log
+```
+
+Verify the staged artifacts in the Nexus repository:
+* Go to https://repository.apache.org/
+* Under `Build Promotion`, click `Staging Repositories`
+* In the `Staging Repositories` tab there should be a line with profile `org.apache.calcite`
+* Navigate through the artifact tree and make sure the .jar, .pom, .asc files are present
+* Check the box on in the first column of the row,
+  and press the 'Close' button to publish the repository at
+  https://repository.apache.org/content/repositories/orgapachecalcite-1000
+  (or a similar URL)
+
+Upload the artifacts via subversion to a staging area,
+https://dist.apache.org/repos/dist/dev/incubator/calcite/apache-calcite-X.Y.Z-incubating-rcN:
+
+```bash
+# Create a subversion workspace, if you haven't already
+mkdir -p ~/dist/dev
+pushd ~/dist/dev
+svn co https://dist.apache.org/repos/dist/dev/incubator/calcite
+popd
+
+# Move the files into a directory
+cd target
+mkdir ~/dist/dev/calcite/apache-calcite-X.Y.Z-incubating-rcN
+mv apache-calcite-* ~/dist/dev/calcite/apache-calcite-X.Y.Z-incubating-rcN
+
+# Check in
+cd ~/dist/dev/calcite
+svn add apache-calcite-X.Y.Z-incubating-rcN
+svn ci
+```
+
+## Cleaning up after a failed release attempt (for Calcite committers)
+
+```
+# Make sure that the tag you are about to generate does not already
+# exist (due to a failed release attempt)
+git tag
+
+# If the tag exists, delete it locally and remotely
+git tag -d apache-calcite-X.Y.Z-incubating
+git push origin :refs/tags/apache-calcite-X.Y.Z-incubating
+
+# Remove modified files
+mvn release:clean
+
+# Check whether there are modified files and if so, go back to the
+# original git commit
+git status
+git reset --hard HEAD
+```
+
+## Validate a release
+
+```bash
+# Check that the signing key (e.g. 2AD3FAE3) is pushed
+gpg --recv-keys key
+
+# Check keys
+curl -O https://dist.apache.org/repos/dist/release/incubator/calcite/KEYS
+
+# Sign/check md5 and sha1 hashes
+# (Assumes your O/S has 'md5' and 'sha1' commands.)
+function checkHash() {
+  cd "$1"
+  for i in *.{zip,pom,gz}; do
+    if [ ! -f $i ]; then
+      continue
+    fi
+    if [ -f $i.md5 ]; then
+      if [ "$(cat $i.md5)" = "$(md5 -q $i)" ]; then
+        echo $i.md5 present and correct
+      else
+        echo $i.md5 does not match
+      fi
+    else
+      md5 -q $i > $i.md5
+      echo $i.md5 created
+    fi
+    if [ -f $i.sha1 ]; then
+      if [ "$(cat $i.sha1)" = "$(sha1 -q $i)" ]; then
+        echo $i.sha1 present and correct
+      else
+        echo $i.sha1 does not match
+      fi
+    else
+      sha1 -q $i > $i.sha1
+      echo $i.sha1 created
+    fi
+  done
+}
+checkHash apache-calcite-X.Y.Z-incubating-rcN
+```
+
+## Get approval for a release via Apache voting process (for Calcite committers)
+
+Release vote on dev list
+
+```
+To: dev@calcite.incubator.apache.org
+Subject: [VOTE] Release apache-calcite-X.Y.Z-incubating (release candidate N)
+
+Hi all,
+
+I have created a build for Apache Calcite X.Y.Z-incubating, release candidate N.
+
+Thanks to everyone who has contributed to this release.
+<Further details about release.> You can read the release notes here:
+https://github.com/apache/incubator-calcite/blob/XXXX/doc/history.md
+
+The commit to be voted upon:
+http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/NNNNNN
+
+Its hash is XXXX.
+
+The artifacts to be voted on are located here:
+https://dist.apache.org/repos/dist/dev/incubator/calcite/apache-calcite-X.Y.Z-incubating-rcN/
+
+The hashes of the artifacts are as follows:
+src.tar.gz.md5 XXXX
+src.tar.gz.sha1 XXXX
+src.zip.md5 XXXX
+src.zip.sha1 XXXX
+
+A staged Maven repository is available for review at:
+https://repository.apache.org/content/repositories/orgapachecalcite-NNNN
+
+Release artifacts are signed with the following key:
+https://people.apache.org/keys/committer/jhyde.asc
+
+Please vote on releasing this package as Apache Calcite X.Y.Z-incubating.
+
+The vote is open for the next 72 hours and passes if a majority of
+at least three +1 PPMC votes are cast.
+
+[ ] +1 Release this package as Apache Calcite X.Y.Z-incubating
+[ ]  0 I don't feel strongly about it, but I'm okay with the release
+[ ] -1 Do not release this package because...
+
+
+Here is my vote:
+
++1 (binding)
+
+Julian
+```
+
+After vote finishes, send out the result:
+
+```
+Subject: [RESULT] [VOTE] Release apache-calcite-X.Y.Z-incubating (release candidate N)
+To: dev@calcite.incubator.apache.org
+
+Thanks to everyone who has tested the release candidate and given
+their comments and votes.
+
+The tally is as follows.
+
+N binding +1s:
+<names>
+
+N non-binding +1s:
+<names>
+
+No 0s or -1s.
+
+Therefore I am delighted to announce that the proposal to release
+Apache Calcite X.Y.Z-incubating has passed.
+
+I'll now start a vote on the general list. Those of you in the IPMC,
+please recast your vote on the new thread.
+
+Julian
+```
+
+Use the [Apache URL shortener](http://s.apache.org) to generate
+shortened URLs for the vote proposal and result emails. Examples:
+[s.apache.org/calcite-1.2-vote](http://s.apache.org/calcite-1.2-vote) and
+[s.apache.org/calcite-1.2-result](http://s.apache.org/calcite-1.2-result).
+
+Propose a vote on the incubator list.
+
+```
+To: general@incubator.apache.org
+Subject: [VOTE] Release Apache Calcite X.Y.Z (incubating)
+
+Hi all,
+
+The Calcite community has voted on and approved a proposal to release
+Apache Calcite X.Y.Z (incubating).
+
+Proposal:
+http://s.apache.org/calcite-X.Y.Z-vote
+
+Vote result:
+N binding +1 votes
+N non-binding +1 votes
+No -1 votes
+http://s.apache.org/calcite-X.Y.Z-result
+
+The commit to be voted upon:
+http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/NNNNNN
+
+Its hash is XXXX.
+
+The artifacts to be voted on are located here:
+https://dist.apache.org/repos/dist/dev/incubator/calcite/apache-calcite-X.Y.Z-incubating-rcN/
+
+The hashes of the artifacts are as follows:
+src.tar.gz.md5 XXXX
+src.tar.gz.sha1 XXXX
+src.zip.md5 XXXX
+src.zip.sha1 XXXX
+
+A staged Maven repository is available for review at:
+https://repository.apache.org/content/repositories/orgapachecalcite-NNNN
+
+Release artifacts are signed with the following key:
+https://people.apache.org/keys/committer/jhyde.asc
+
+Pursuant to the Releases section of the Incubation Policy and with
+the endorsement of NNN of our mentors we would now like to request
+the permission of the Incubator PMC to publish the release. The vote
+is open for 72 hours, or until the necessary number of votes (3 +1)
+is reached.
+
+[ ] +1 Release this package as Apache Calcite X.Y.Z incubating
+[ ] -1 Do not release this package because...
+
+Julian Hyde, on behalf of Apache Calcite PPMC
+```
+
+After vote finishes, send out the result:
+
+```
+To: general@incubator.apache.org
+Subject: [RESULT] [VOTE] Release Apache Calcite X.Y.Z (incubating)
+
+This vote passes with N +1s and no 0 or -1 votes:
++1 <name> (mentor)
+
+There was some feedback during voting. I shall open a separate
+thread to discuss.
+
+Thanks everyone. We’ll now roll the release out to the mirrors.
+
+Julian
+```
+
+## Publishing a release (for Calcite committers)
+
+After a successful release vote, we need to push the release
+out to mirrors, and other tasks.
+
+In JIRA, search for all issues resolved in this release,
+and do a bulk update changing their status to "Closed",
+with a change comment
+"Resolved in release X.Y.Z-incubating (YYYY-MM-DD)"
+(fill in release number and date appropriately).
+
+Promote the staged nexus artifacts.
+* Go to https://repository.apache.org/
+* Under "Build Promotion" click "Staging Repositories"
+* In the line with "orgapachecalcite-xxxx", check the box
+* Press "Release" button
+
+Check the artifacts into svn.
+
+```bash
+# Get the release candidate.
+mkdir -p ~/dist/dev
+cd ~/dist/dev
+svn co https://dist.apache.org/repos/dist/dev/incubator/calcite
+
+# Copy the artifacts. Note that the copy does not have '-rcN' suffix.
+mkdir -p ~/dist/release
+cd ~/dist/release
+svn co https://dist.apache.org/repos/dist/release/incubator/calcite
+cd calcite
+cp -rp ../../dev/calcite/apache-calcite-X.Y.Z-incubating-rcN apache-calcite-X.Y.Z-incubating
+svn add apache-calcite-X.Y.Z-incubating
+
+# Check in.
+svn ci
+```
+
+Svnpubsub will publish to
+https://dist.apache.org/repos/dist/release/incubator/calcite and propagate to
+http://www.apache.org/dyn/closer.cgi/incubator/calcite within 24 hours.
+
+## Publishing the web site (for Calcite committers)
+
+Get the code:
+
+```bash
+$ svn co https://svn.apache.org/repos/asf/incubator/calcite/site calcite-site
+```
+
+(Note: `https:`, not `http:`.)
+
+Build the site:
+
+```bash
+$ cd calcite-site
+$ ./build.sh
+```
+
+It will prompt you to install jekyll, redcarpet and pygments, if you
+do not have them installed. It will also check out the git source code
+repo, so that it can generate javadoc.
+
+Check in:
+
+```bash
+svn ci -m"Commit message" file...
+```
+
+The site will automatically be deployed as http://calcite.incubator.apache.org.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/model.md
----------------------------------------------------------------------
diff --git a/doc/model.md b/doc/model.md
new file mode 100644
index 0000000..484a302
--- /dev/null
+++ b/doc/model.md
@@ -0,0 +1,409 @@
+<!--
+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.
+-->
+# Calcite JSON model reference
+
+## Elements
+
+### Root
+
+```json
+{
+  version: '1.0',
+  defaultSchema: 'mongo',
+  schemas: [ Schema... ]
+}
+```
+
+`version` (required string) must have value `1.0`.
+
+`defaultSchema` (optional string). If specified, it is
+the name (case-sensitive) of a schema defined in this model, and will
+become the default schema for connections to Calcite that use this model.
+
+`schemas` (optional list of <a href="#schema">Schema</a> elements).
+
+### Schema
+
+Occurs within `root.schemas`.
+
+```json
+{
+  name: 'foodmart',
+  path: ['lib'],
+  cache: true,
+  materializations: [ Materialization... ]
+}
+```
+
+`name` (required string) is the name of the schema.
+
+`type` (optional string, default `map`) indicates sub-type. Values are:
+* `map` for <a href="#map-schema">Map Schema</a>
+* `custom` for <a href="#custom-schema">Custom Schema</a>
+* `jdbc` for <a href="#jdbc-schema">JDBC Schema</a>
+
+`path` (optional list) is the SQL path that is used to
+resolve functions used in this schema. If specified it must be a list,
+and each element of the list must be either a string or a list of
+strings. For example,
+
+```json
+  path: [ ['usr', 'lib'], 'lib' ]
+```
+
+declares a path with two elements: the schema '/usr/lib' and the
+schema '/lib'. Most schemas are at the top level, so you can use a
+string.
+
+`materializations` (optional list of
+<a href="#materialization">Materialization</a>) defines the tables
+in this schema that are materializations of queries.
+
+`cache` (optional boolean, default true) tells Calcite whether to
+cache metadata (tables, functions and sub-schemas) generated
+by this schema.
+
+* If `false`, Calcite will go back to the schema each time it needs
+  metadata, for example, each time it needs a list of tables in order to
+  validate a query against the schema.
+
+* If `true`, Calcite will cache the metadata the first time it reads
+  it. This can lead to better performance, especially if name-matching is
+  case-insensitive.
+
+However, it also leads to the problem of cache staleness.
+A particular schema implementation can override the
+`Schema.contentsHaveChangedSince` method to tell Calcite
+when it should consider its cache to be out of date.
+
+Tables, functions and sub-schemas explicitly created in a schema are
+not affected by this caching mechanism. They always appear in the schema
+immediately, and are never flushed.
+
+### Map Schema
+
+Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
+
+```json
+{
+  name: 'foodmart',
+  type: 'map',
+  tables: [ Table... ],
+  functions: [ Function... ]
+}
+```
+
+`name`, `type`, `path`, `cache`, `materializations` inherited from
+<a href="#schema">Schema</a>.
+
+`tables` (optional list of <a href="#table">Table</a> elements)
+defines the tables in this schema.
+
+`functions` (optional list of <a href="#function">Function</a> elements)
+defines the functions in this schema.
+
+### Custom Schema
+
+Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
+
+```json
+{
+  name: 'mongo',
+  type: 'custom',
+  factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',
+  operand: {
+    host: 'localhost',
+    database: 'test'
+  }
+}
+```
+
+`name`, `type`, `path`, `cache`, `materializations` inherited from
+<a href="#schema">Schema</a>.
+
+`factory` (required string) is the name of the factory class for this
+schema. Must implement interface `org.apache.calcite.schema.SchemaFactory`
+and have a public default constructor.
+
+`operand` (optional map) contains attributes to be passed to the
+factory.
+
+### JDBC Schema
+
+Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
+
+```json
+{
+  name: 'foodmart',
+  type: 'jdbc',
+  jdbcDriver: TODO,
+  jdbcUrl: TODO,
+  jdbcUser: TODO,
+  jdbcPassword: TODO,
+  jdbcCatalog: TODO,
+  jdbcSchema: TODO
+}
+```
+
+`name`, `type`, `path`, `cache`, `materializations` inherited from
+<a href="#schema">Schema</a>.
+
+`jdbcDriver` (optional string) is the name of the JDBC driver class. It not
+specified, uses whichever class the JDBC DriverManager chooses.
+
+`jdbcUrl` (optional string) is the JDBC connect string, for example
+"jdbc:mysql://localhost/foodmart".
+
+`jdbcUser` (optional string) is the JDBC user name.
+
+`jdbcPassword` (optional string) is the JDBC password.
+
+`jdbcCatalog` (optional string) is the name of the initial catalog in the JDBC
+data source.
+
+`jdbcSchema` (optional string) is the name of the initial schema in the JDBC
+data source.
+
+### Materialization
+
+Occurs within `root.schemas.materializations`.
+
+```json
+{
+  view: 'V',
+  table: 'T',
+  sql: 'select deptno, count(*) as c, sum(sal) as s from emp group by deptno'
+}
+```
+
+`view` (optional string) TODO
+
+`table` (optional string) TODO
+
+`sql` (optional string, or list of strings that will be concatenated as a
+ multi-line string) is the SQL definition of the materialization.
+
+### Table
+
+Occurs within `root.schemas.tables`.
+
+```json
+{
+  name: 'sales_fact',
+  columns: [ Column... ]
+}
+```
+
+`name` (required string) is the name of this table. Must be unique within the schema.
+
+`type` (optional string, default `custom`) indicates sub-type. Values are:
+* `custom` for <a href="#custom-table">Custom Table</a>
+* `view` for <a href="#view">View</a>
+
+`columns` (optional list of <a href="#column">Column</a> elements)
+
+### View
+
+Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
+
+```json
+{
+  name: 'female_emps',
+  type: 'view',
+  sql: "select * from emps where gender = 'F'"
+}
+```
+
+`name`, `type`, `columns` inherited from <a href="#table">Table</a>.
+
+`sql` (required string, or list of strings that will be concatenated as a
+ multi-line string) is the SQL definition of the view.
+
+`path` (optional list) is the SQL path to resolve the query. If not
+specified, defaults to the current schema.
+
+### Custom Table
+
+Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
+
+```json
+{
+  name: 'female_emps',
+  type: 'custom',
+  factory: 'TODO',
+  operand: {
+    todo: 'TODO'
+  }
+}
+```
+
+`name`, `type`, `columns` inherited from <a href="#table">Table</a>.
+
+`factory` (required string) is the name of the factory class for this
+table. Must implement interface `org.apache.calcite.schema.TableFactory`
+and have a public default constructor.
+
+`operand` (optional map) contains attributes to be passed to the
+factory.
+
+### Column
+
+Occurs within `root.schemas.tables.columns`.
+
+```json
+{
+  name: 'empno'
+}
+```
+
+`name` (required string) is the name of this column.
+
+### Function
+
+Occurs within `root.schemas.functions`.
+
+```json
+{
+  name: 'MY_PLUS',
+  className: 'com.example.functions.MyPlusFunction',
+  methodName: 'apply',
+  path: []
+}
+```
+
+`name` (required string) is the name of this function.
+
+`className` (required string) is the name of the class that implements this
+function.
+
+`methodName` (optional string) is the name of the method that implements this
+function.
+
+`path` (optional list of string) is the path for resolving this function.
+
+### Lattice
+
+Occurs within `root.schemas.lattices`.
+
+```json
+{
+  name: 'star',
+  sql: [
+    'select 1 from "foodmart"."sales_fact_1997" as "s"',
+    'join "foodmart"."product" as "p" using ("product_id")',
+    'join "foodmart"."time_by_day" as "t" using ("time_id")',
+    'join "foodmart"."product_class" as "pc" on "p"."product_class_id" = "pc"."product_class_id"'
+  ],
+  auto: false,
+  algorithm: true,
+  algorithmMaxMillis: 10000,
+  rowCountEstimate: 86837,
+  defaultMeasures: [ {
+    agg: 'count'
+  } ],
+  tiles: [ {
+    dimensions: [ 'the_year', ['t', 'quarter'] ],
+    measures: [ {
+      agg: 'sum',
+      args: 'unit_sales'
+    }, {
+      agg: 'sum',
+      args: 'store_sales'
+    }, {
+      agg: 'count'
+    } ]
+  } ]
+}
+```
+
+`name` (required string) is the name of this lattice.
+
+`sql` (required string, or list of strings that will be concatenated as a
+multi-line string) is the SQL statement that defines the fact table, dimension
+tables, and join paths for this lattice.
+
+`auto` (optional boolean, default true) is whether to materialize tiles on need
+as queries are executed.
+
+`algorithm` (optional boolean, default false) is whether to use an optimization
+algorithm to suggest and populate an initial set of tiles.
+
+`algorithmMaxMillis` (optional long, default -1, meaning no limit) is the
+maximum number of milliseconds for which to run the algorithm. After this point,
+takes the best result the algorithm has come up with so far.
+
+`rowCountEstimate` (optional double, default 1000.0) estimated number of rows in
+the star
+
+`tiles` (optional list of <a href="#tile">Tile</a> elements) is a list of
+materialized aggregates to create up front.
+
+`defaultMeasures`  (optional list of <a href="#measure">Measure</a> elements)
+is a list of measures that a tile should have by default.
+Any tile defined in `tiles` can still define its own measures, including
+measures not on this list. If not specified, the default list of measures is
+just 'count(*)':
+
+```json
+[ { name: 'count' } ]
+```
+
+### Tile
+
+Occurs within `root.schemas.lattices.tiles`.
+
+```json
+{
+  dimensions: [ 'the_year', ['t', 'quarter'] ],
+  measures: [ {
+    agg: 'sum',
+    args: 'unit_sales'
+  }, {
+    agg: 'sum',
+    args: 'store_sales'
+  }, {
+    agg: 'count'
+  } ]
+}
+```
+
+`dimensions` is a list of dimensions (columns from the star), like a `GROUP BY`
+clause. Each element is either a string (the unique label of the column within
+the star) or a string list (a column name qualified by a table name).
+
+`measures` (optional list of <a href="#measure">Measure</a> elements) is a list
+of aggregate functions applied to arguments. If not specified, uses the
+lattice's default measure list.
+
+### Measure
+
+Occurs within `root.schemas.lattices.defaultMeasures`
+and `root.schemas.lattices.tiles.measures`.
+
+```json
+{
+  agg: 'sum',
+  args: [ 'unit_sales' ]
+}
+```
+
+`agg` is the name of an aggregate function (usually 'count', 'sum', 'min',
+'max').
+
+`args` (optional) is a column label (string), or list of zero or more columns.
+If a list, each element is either a string (the unique label of the column
+within the star) or a string list (a column name qualified by a table name).

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/06a192a0/doc/reference.md
----------------------------------------------------------------------
diff --git a/doc/reference.md b/doc/reference.md
new file mode 100644
index 0000000..03f66e3
--- /dev/null
+++ b/doc/reference.md
@@ -0,0 +1,569 @@
+<!--
+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.
+-->
+# Calcite SQL language reference
+
+## SQL constructs
+
+```SQL
+statement:
+      setStatement
+  |   explain
+  |   insert
+  |   update
+  |   merge
+  |   delete
+  |   query
+
+setStatement:
+      ALTER ( SYSTEM | SESSION ) SET identifier = expression
+
+explain:
+      EXPLAIN PLAN
+      [ WITH TYPE | WITH IMPLEMENTATION | WITHOUT IMPLEMENTATION ]
+      [ EXCLUDING ATTRIBUTES | INCLUDING [ ALL ] ATTRIBUTES ]
+      FOR ( insert | update | merge | delete | query )
+
+insert:
+      ( INSERT | UPSERT ) INTO tablePrimary
+      [ '(' column [, column ]* ')' ]
+      query
+
+update:
+      UPDATE tablePrimary
+      SET assign [, assign ]*
+      [ WHERE booleanExpression ]
+
+assign:
+      identifier '=' expression
+
+merge:
+      MERGE INTO tablePrimary [ [ AS ] alias ]
+      USING tablePrimary
+      ON booleanExpression
+      [ WHEN MATCHED THEN UPDATE SET assign [, assign ]* ]
+      [ WHEN NOT MATCHED THEN INSERT VALUES '(' value [ , value ]* ')' ]
+
+delete:
+      DELETE FROM tablePrimary [ [ AS ] alias ]
+      [ WHERE booleanExpression ]
+
+query:
+      [ WITH withItem [ , withItem ]* query ]
+  |   {
+          select
+      |   query UNION [ ALL ] query
+      |   query EXCEPT query
+      |   query INTERSECT query
+      }
+      [ ORDER BY orderItem [, orderItem ]* ]
+      [ LIMIT { count | ALL } ]
+      [ OFFSET start { ROW | ROWS } ]
+      [ FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } ]
+
+withItem:
+      name
+      [ '(' column [, column ]* ')' ]
+      AS '(' query ')'
+
+orderItem:
+      expression [ ASC | DESC ] [ NULLS FIRST | NULLS LAST ]
+
+select:
+      SELECT [ STREAM ] [ ALL | DISTINCT ]
+          { * | projectItem [, projectItem ]* }
+      FROM tableExpression
+      [ WHERE booleanExpression ]
+      [ GROUP BY { groupItem [, groupItem ]* } ]
+      [ HAVING booleanExpression ]
+      [ WINDOW windowName AS windowSpec [, windowName AS windowSpec ]* ]
+
+projectItem:
+      expression [ [ AS ] columnAlias ]
+  |   tableAlias . *
+
+tableExpression:
+      tableReference [, tableReference ]*
+  |   tableExpression [ NATURAL ] [ LEFT | RIGHT | FULL ] JOIN tableExpression [ joinCondition ]
+
+joinCondition:
+      ON booleanExpression
+  |   USING '(' column [, column ]* ')'
+
+tableReference:
+      [ LATERAL ]
+      tablePrimary
+      [ [ AS ] alias [ '(' columnAlias [, columnAlias ]* ')' ] ]
+
+tablePrimary:
+      [ TABLE ] [ [ catalogName . ] schemaName . ] tableName
+  |   '(' query ')'
+  |   values
+  |   UNNEST '(' expression ')'
+  |   '(' TABLE expression ')'
+
+values:
+      VALUES expression [, expression ]*
+
+groupItem:
+      expression
+  |   '(' ')'
+  |   '(' expression [, expression ]* ')'
+  |   CUBE '(' expression [, expression ]* ')'
+  |   ROLLUP '(' expression [, expression ]* ')'
+  |   GROUPING SETS '(' groupItem [, groupItem ]* ')'
+
+windowRef:
+      windowName
+  |   windowSpec
+
+windowSpec:
+      [ windowName ]
+      '('
+      [ ORDER BY orderItem [, orderItem ]* ]
+      [ PARTITION BY expression [, expression ]* ]
+      [
+          RANGE numericOrIntervalExpression { PRECEDING | FOLLOWING }
+      |   ROWS numericExpression { PRECEDING | FOLLOWING }
+      ]
+      ')'
+```
+
+In *merge*, at least one of the WHEN MATCHED and WHEN NOT MATCHED clauses must
+be present.
+
+In *orderItem*, if *expression* is a positive integer *n*, it denotes
+the <em>n</em>th item in the SELECT clause.
+
+An aggregate query is a query that contains a GROUP BY or a HAVING
+clause, or aggregate functions in the SELECT clause. In the SELECT,
+HAVING and ORDER BY clauses of an aggregate query, all expressions
+must be constant within the current group (that is, grouping constants
+as defined by the GROUP BY clause, or constants), or aggregate
+functions, or a combination of constants and aggregate
+functions. Aggregate and grouping functions may only appear in an
+aggregate query, and only in a SELECT, HAVING or ORDER BY clause.
+
+A scalar sub-query is a sub-query used as an expression. It can occur
+in most places where an expression can occur (such as the SELECT
+clause, WHERE clause, or as an argument to an aggregate
+function). If the sub-query returns no rows, the value is NULL; if it
+returns more than one row, it is an error.
+
+A sub-query can occur in the FROM clause of a query and also in IN
+and EXISTS expressions.  A sub-query that occurs in IN and
+EXISTS expressions may be correlated; that is, refer to tables in
+the FROM clause of an enclosing query.
+
+## Identifiers
+
+Identifiers are the names of tables, columns and other metadata
+elements used in a SQL query.
+
+Unquoted identifiers, such as emp, must start with a letter and can
+only contain letters, digits, and underscores. They are implicitly
+converted to upper case.
+
+Quoted identifiers, such as "Employee Name", start and end with
+double quotes.  They may contain virtually any character, including
+spaces and other punctuation.  If you wish to include a double quote
+in an identifier, use another double quote to escape it, like this:
+"An employee called ""Fred""."
+
+In Calcite, matching identifiers to the name of the referenced object is
+case-sensitive.  But remember that unquoted identifiers are implicitly
+converted to upper case before matching, and if the object it refers
+to was created using an unquoted identifier for its name, then its
+name will have been converted to upper case also.
+
+## Data types
+
+### Scalar types
+
+| Data type   | Description               | Range and examples   |
+| ----------- | ------------------------- | ---------------------|
+| BOOLEAN     | Logical values            | Values: TRUE, FALSE, UNKNOWN
+| TINYINT     | 1 byte signed integer     | Range is -255 to 256
+| SMALLINT    | 2 byte signed integer     | Range is -32768 to 32767
+| INTEGER, INT | 4 byte signed integer    | Range is -2147483648 to 2147483647
+| BIGINT      | 8 byte signed integer     | Range is -9223372036854775808 to 9223372036854775807
+| DECIMAL(p, s) | Fixed point             | Example: 123.45 is a DECIMAL(5, 2) value.
+| NUMERIC     | Fixed point               |
+| REAL, FLOAT | 4 byte floating point     | 6 decimal digits precision
+| DOUBLE      | 8 byte floating point     | 15 decimal digits precision
+| CHAR(n), CHARACTER(n) | Fixed-width character string | 'Hello', '' (empty string), _latin1'Hello', n'Hello', _UTF16'Hello', 'Hello' 'there' (literal split into multiple parts)
+| VARCHAR(n), CHARACTER VARYING(n) | Variable-length character string | As CHAR(n)
+| BINARY(n)   | Fixed-width binary string | x'45F0AB', x'' (empty binary string), x'AB' 'CD' (multi-part binary string literal)
+| VARBINARY(n), BINARY VARYING(n) | Variable-length binary string | As BINARY(n)
+| DATE        | Date                      | Example: DATE '1969-07-20'
+| TIME        | Time of day               | Example: TIME '20:17:40'
+| TIMESTAMP [ WITHOUT TIME ZONE ] | Date and time | Example: TIMESTAMP '1969-07-20 20:17:40'
+| TIMESTAMP WITH TIME ZONE | Date and time with time zone | Example: TIMESTAMP '1969-07-20 20:17:40 America/Los Angeles'
+| INTERVAL timeUnit [ TO timeUnit ] | Date time interval | Examples: INTERVAL '1:5' YEAR TO MONTH, INTERVAL '45' DAY
+| Anchored interval | Date time interval  | Example: (DATE '1969-07-20', DATE '1972-08-29')
+
+Where:
+```SQL
+timeUnit:
+  YEAR | MONTH | DAY | HOUR | MINUTE | SECOND
+```
+
+Note:
+* DATE, TIME and TIMESTAMP have no time zone. There is not even an implicit
+  time zone, such as UTC (as in Java) or the local time zone. It is left to
+  the user or application to supply a time zone.
+
+### Non-scalar types
+
+| Type     | Description
+| -------- | -----------------------------------------------------------
+| ANY      | A value of an unknown type
+| ROW      | Row with 1 or more columns
+| MAP      | Collection of keys mapped to values
+| MULTISET | Unordered collection that may contain duplicates
+| ARRAY    | Ordered, contiguous collection that may contain duplicates
+| CURSOR   | Cursor over the result of executing a query
+
+## Operators and functions
+
+### Comparison operators
+
+| Operator syntax                                   | Description
+| ------------------------------------------------- | -----------
+| value1 = value2                                   | Equals
+| value1 <> value2                                  | Not equal
+| value1 > value2                                   | Greater than
+| value1 >= value2                                  | Greater than or equal
+| value1 < value2                                   | Less than
+| value1 <= value2                                  | Less than or equal
+| value IS NULL                                     | Whether *value* is null
+| value IS NOT NULL                                 | Whether *value* is not null
+| value1 IS DISTINCT FROM value2                    | Whether two values are not equal, treating null values as the same
+| value1 IS NOT DISTINCT FROM value2                | Whether two values are equal, treating null values as the same
+| value1 BETWEEN value2 AND value3                  | Whether *value1* is greater than or equal to *value2* and less than or equal to *value3*
+| value1 NOT BETWEEN value2 AND value3              | Whether *value1* is less than *value2* or greater than *value3*
+| string1 LIKE string2 [ ESCAPE string3 ]           | Whether *string1* matches pattern *string2*
+| string1 NOT LIKE string2 [ ESCAPE string3 ]       | Whether *string1* does not match pattern *string2*
+| string1 SIMILAR TO string2 [ ESCAPE string3 ]     | Whether *string1* matches regular expression *string2*
+| string1 NOT SIMILAR TO string2 [ ESCAPE string3 ] | Whether *string1* does not match regular expression *string2*
+| value IN (value [, value]* )                      | Whether *value* is equal to a value in a list
+| value NOT IN (value [, value]* )                  | Whether *value* is not equal to every value in a list
+| value IN (sub-query)                              | Whether *value* is equal to a row returned by *sub-query*
+| value NOT IN (sub-query)                          | Whether *value* is not equal to every row returned by *sub-query*
+| EXISTS (sub-query)                                | Whether *sub-query* returns at least one row
+
+### Logical operators
+
+| Operator syntax        | Description
+| ---------------------- | -----------
+| boolean1 OR boolean2   | Whether *boolean1* is TRUE or *boolean2* is TRUE
+| boolean1 AND boolean2  | Whether *boolean1* and *boolean2* are both TRUE
+| NOT boolean            | Whether *boolean* is not TRUE; returns UNKNOWN if *boolean* is UNKNOWN
+| boolean IS FALSE       | Whether *boolean* is FALSE; returns FALSE if *boolean* is UNKNOWN
+| boolean IS NOT FALSE   | Whether *boolean* is not FALSE; returns TRUE if *boolean* is UNKNOWN
+| boolean IS TRUE        | Whether *boolean* is TRUE; returns FALSE if *boolean* is UNKNOWN
+| boolean IS NOT TRUE    | Whether *boolean* is not TRUE; returns TRUE if *boolean* is UNKNOWN
+| boolean IS UNKNOWN     | Whether *boolean* is UNKNOWN
+| boolean IS NOT UNKNOWN | Whether *boolean* is not UNKNOWN
+
+### Arithmetic operators and functions
+
+| Operator syntax           | Description
+| ------------------------- | -----------
+| + numeric                 | Returns *numeric*
+| - numeric                 | Returns negative *numeric*
+| numeric1 + numeric2       | Returns *numeric1* plus *numeric2*
+| numeric1 - numeric2       | Returns *numeric1* minus *numeric2*
+| numeric1 * numeric2       | Returns *numeric1* multiplied by *numeric2*
+| numeric1 / numeric2       | Returns *numeric1* divided by *numeric2*
+| POWER(numeric1, numeric2) | Returns *numeric1* raised to the power of *numeric2*
+| ABS(numeric)              | Returns the absolute value of *numeric*
+| MOD(numeric, numeric)     | Returns the remainder (modulus) of *numeric1* divided by *numeric2*. The result is negative only if *numeric1* is negative
+| SQRT(numeric)             | Returns the square root of *numeric*
+| LN(numeric)               | Returns the natural logarithm (base *e*) of *numeric*
+| LOG10(numeric)            | Returns the base 10 logarithm of *numeric*
+| EXP(numeric)              | Returns *e* raised to the power of *numeric*
+| CEIL(numeric)             | Rounds *numeric* up, and returns the smallest number that is greater than or equal to *numeric*
+| FLOOR(numeric)            | Rounds *numeric* down, and returns the largest number that is less than or equal to *numeric*
+
+### Character string operators and functions
+
+| Operator syntax            | Description
+| -------------------------- | -----------
+| string &#124;&#124; string | Concatenates two character strings.
+| CHAR_LENGTH(string)        | Returns the number of characters in a character string
+| CHARACTER_LENGTH(string)   | As CHAR_LENGTH(*string*)
+| UPPER(string)              | Returns a character string converted to upper case
+| LOWER(string)              | Returns a character string converted to lower case
+| POSITION(string1 IN string2) | Returns the position of the first occurrence of *string1* in *string2*
+| TRIM( { BOTH ;&#124; LEADING ;&#124; TRAILING } string1 FROM string2) | Removes the longest string containing only the characters in *string1* from the start/end/both ends of *string1*
+| OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ]) | Replaces a substring of *string1* with *string2*
+| SUBSTRING(string FROM integer)  | Returns a substring of a character string starting at a given point.
+| SUBSTRING(string FROM integer FOR integer) | Returns a substring of a character string starting at a given point with a given length.
+| INITCAP(string)            | Returns *string* with the first letter of each word converter to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters.
+
+Not implemented:
+* SUBSTRING(string FROM regexp FOR regexp)
+
+### Binary string operators and functions
+
+| Operator syntax | Description
+| --------------- | -----------
+| binary &#124;&#124; binary | Concatenates two binary strings.
+| POSITION(binary1 IN binary2) | Returns the position of the first occurrence of *binary1* in *binary2*
+| OVERLAY(binary1 PLACING binary2 FROM integer [ FOR integer2 ]) | Replaces a substring of *binary1* with *binary2*
+| SUBSTRING(binary FROM integer) | Returns a substring of *binary* starting at a given point
+| SUBSTRING(binary FROM integer FOR integer) | Returns a substring of *binary* starting at a given point with a given length
+
+### Date/time functions
+
+| Operator syntax           | Description
+| ------------------------- | -----------
+| LOCALTIME                 | Returns the current date and time in the session time zone in a value of datatype TIME
+| LOCALTIME(precision)      | Returns the current date and time in the session time zone in a value of datatype TIME, with *precision* digits of precision
+| LOCALTIMESTAMP            | Returns the current date and time in the session time zone in a value of datatype TIMESTAMP
+| LOCALTIMESTAMP(precision) | Returns the current date and time in the session time zone in a value of datatype TIMESTAMP, with *precision* digits of precision
+| CURRENT_TIME              | Returns the current time in the session time zone, in a value of datatype TIMESTAMP WITH TIME ZONE
+| CURRENT_DATE              | Returns the current date in the session time zone, in a value of datatype DATE
+| CURRENT_TIMESTAMP         | Returns the current date and time in the session time zone, in a value of datatype TIMESTAMP WITH TIME ZONE
+| EXTRACT(timeUnit FROM datetime) | Extracts and returns the value of a specified datetime field from a datetime value expression
+| FLOOR(datetime TO timeUnit) | Rounds *datetime* down to *timeUnit*
+| CEIL(datetime TO timeUnit) | Rounds *datetime* up to *timeUnit*
+
+Not implemented:
+* EXTRACT(timeUnit FROM interval)
+* CEIL(interval)
+* FLOOR(interval)
+* datetime - datetime timeUnit [ TO timeUnit ]
+* interval OVERLAPS interval
+* + interval
+* - interval
+* interval + interval
+* interval - interval
+* interval / interval
+* datetime + interval
+* datetime - interval
+
+### System functions
+
+| Operator syntax | Description
+| --------------- | -----------
+| USER            | Equivalent to CURRENT_USER
+| CURRENT_USER    | User name of current execution context
+| SESSION_USER    | Session user name
+| SYSTEM_USER     | Returns the name of the current data store user as identified by the operating system
+| CURRENT_PATH    | Returns a character string representing the current lookup scope for references to user-defined routines and types
+| CURRENT_ROLE    | Returns the current active role
+
+### Conditional functions and operators
+
+| Operator syntax | Description
+| --------------- | -----------
+| CASE value<br/>WHEN value1 [, value11 ]* THEN result1<br/>[ WHEN valueN [, valueN1 ]* THEN resultN ]*<br/>[ ELSE resultZ ]<br/> END | Simple case
+| CASE<br/>WHEN condition1 THEN result1<br/>[ WHEN conditionN THEN resultN ]*<br/>[ ELSE resultZ ]<br/>END | Searched case
+| NULLIF(value, value) | Returns NULL if the values are the same. For example, <code>NULLIF(5, 5)</code> returns NULL; <code>NULLIF(5, 0)</code> returns 5.
+| COALESCE(value, value [, value]* ) | Provides a value if the first value is null. For example, <code>COALESCE(NULL, 5)</code> returns 5.
+
+### Type conversion
+
+| Operator syntax | Description
+| --------------- | -----------
+| CAST(value AS type) | Converts a value to a given type.
+
+### Value constructors
+
+| Operator syntax | Description
+| --------------- | -----------
+| ROW (value [, value]* ) | Creates a row from a list of values.
+| (value [, value]* )     | Creates a row from a list of values.
+| map [ key ]     | Returns the element of a map with a particular key.
+| array [ index ] | Returns the element at a particular location in an array.
+| ARRAY [ value [, value ]* ] | Creates an array from a list of values.
+| MAP [ key, value [, key, value ]* ] | Creates a map from a list of key-value pairs.
+
+### Collection functions
+
+| Operator syntax | Description
+| --------------- | -----------
+| ELEMENT(value)  | Returns the sole element of a array or multiset; null if the collection is empty; throws if it has more than one element.
+| CARDINALITY(value) | Returns the number of elements in an array or multiset.
+
+See also: UNNEST relational operator converts a collection to a relation.
+
+### JDBC function escape
+
+#### Numeric
+
+| Operator syntax                | Description
+| ------------------------------ | -----------
+| {fn LOG10(numeric)}            | Returns the base-10 logarithm of *numeric*
+| {fn POWER(numeric1, numeric2)} | Returns *numeric1* raised to the power of *numeric2*
+
+Not implemented:
+* {fn ABS(numeric)} - Returns the absolute value of *numeric*
+* {fn ACOS(numeric)} - Returns the arc cosine of *numeric*
+* {fn ASIN(numeric)} - Returns the arc sine of *numeric*
+* {fn ATAN(numeric)} - Returns the arc tangent of *numeric*
+* {fn ATAN2(numeric, numeric)}
+* {fn CEILING(numeric)} - Rounds *numeric* up, and returns the smallest number that is greater than or equal to *numeric*
+* {fn COS(numeric)} - Returns the cosine of *numeric*
+* {fn COT(numeric)}
+* {fn DEGREES(numeric)} - Converts *numeric* from radians to degrees
+* {fn EXP(numeric)} - Returns *e* raised to the power of *numeric*
+* {fn FLOOR(numeric)} - Rounds *numeric* down, and returns the largest number that is less than or equal to *numeric*
+* {fn LOG(numeric)} - Returns the natural logarithm (base *e*) of *numeric*
+* {fn MOD(numeric1, numeric2)} - Returns the remainder (modulus) of *numeric1* divided by *numeric2*. The result is negative only if *numeric1* is negative
+* {fn PI()} - Returns a value that is closer than any other value to *pi*
+* {fn RADIANS(numeric)} - Converts *numeric* from degrees to radians
+* {fn RAND(numeric)}
+* {fn ROUND(numeric, numeric)}
+* {fn SIGN(numeric)}
+* {fn SIN(numeric)} - Returns the sine of *numeric*
+* {fn SQRT(numeric)} - Returns the square root of *numeric*
+* {fn TAN(numeric)} - Returns the tangent of *numeric*
+* {fn TRUNCATE(numeric, numeric)}
+
+#### String
+
+| Operator syntax | Description
+| --------------- | -----------
+| {fn LOCATE(string1, string2)} | Returns the position in *string2* of the first occurrence of *string1*. Searches from the beginning of the second CharacterExpression, unless the startIndex parameter is specified.
+| {fn INSERT(string1, start, length, string2)} | Inserts *string2* into a slot in *string1*
+| {fn LCASE(string)}            | Returns a string in which all alphabetic characters in *string* have been converted to lower case
+
+Not implemented:
+* {fn ASCII(string)} - Convert a single-character string to the corresponding ASCII code, an integer between 0 and 255
+* {fn CHAR(string)}
+* {fn CONCAT(character, character)} - Returns the concatenation of character strings
+* {fn DIFFERENCE(string, string)}
+* {fn LEFT(string, integer)}
+* {fn LENGTH(string)}
+* {fn LOCATE(string1, string2 [, integer])} - Returns the position in *string2* of the first occurrence of *string1*. Searches from the beginning of *string2*, unless *integer* is specified.
+* {fn LTRIM(string)}
+* {fn REPEAT(string, integer)}
+* {fn REPLACE(string, string, string)}
+* {fn RIGHT(string, integer)}
+* {fn RTRIM(string)}
+* {fn SOUNDEX(string)}
+* {fn SPACE(integer)}
+* {fn SUBSTRING(string, integer, integer)}
+* {fn UCASE(string)} - Returns a string in which all alphabetic characters in *string* have been converted to upper case
+
+#### Date/time
+
+Not implemented:
+* {fn CURDATE()}
+* {fn CURTIME()}
+* {fn DAYNAME(date)}
+* {fn DAYOFMONTH(date)}
+* {fn DAYOFWEEK(date)}
+* {fn DAYOFYEAR(date)}
+* {fn HOUR(time)}
+* {fn MINUTE(time)}
+* {fn MONTH(date)}
+* {fn MONTHNAME(date)}
+* {fn NOW()}
+* {fn QUARTER(date)}
+* {fn SECOND(time)}
+* {fn TIMESTAMPADD(interval, count, timestamp)}
+* {fn TIMESTAMPDIFF(interval, timestamp, timestamp)}
+* {fn WEEK(date)}
+* {fn YEAR(date)}
+
+#### System
+
+Not implemented:
+* {fn DATABASE()}
+* {fn IFNULL(value, value)}
+* {fn USER(value, value)}
+* {fn CONVERT(value, type)}
+
+### Aggregate functions
+
+Syntax:
+
+```SQL
+aggregateCall:
+        agg( [ DISTINCT ] value [, value]* ) [ FILTER ( WHERE condition ) ]
+    |   agg(*) [ FILTER ( WHERE condition ) ]
+```
+
+If `FILTER` is present, the aggregate function only considers rows for which
+*condition* evaluates to TRUE.
+
+If `DISTINCT` is present, duplicate argument values are eliminated before being
+passed to the aggregate function.
+
+| Operator syntax                    | Description
+| ---------------------------------- | -----------
+| COUNT( [ DISTINCT ] value [, value]* ) | Returns the number of input rows for which *value* is not null (wholly not null if *value* is composite)
+| COUNT(*)                           | Returns the number of input rows
+| AVG( [ DISTINCT ] numeric)         | Returns the average (arithmetic mean) of *numeric* across all input values
+| SUM( [ DISTINCT ] numeric)         | Returns the sum of *numeric* across all input values
+| MAX( [ DISTINCT ] value)           | Returns the maximum value of *value* across all input values
+| MIN( [ DISTINCT ] value)           | Returns the minimum value of *value* across all input values
+| STDDEV_POP( [ DISTINCT ] numeric)  | Returns the population standard deviation of *numeric* across all input values
+| STDDEV_SAMP( [ DISTINCT ] numeric) | Returns the sample standard deviation of *numeric* across all input values
+| VAR_POP( [ DISTINCT ] value)       | Returns the population variance (square of the population standard deviation) of *numeric* across all input values
+| VAR_SAMP( [ DISTINCT ] numeric)    | Returns the sample variance (square of the sample standard deviation) of *numeric* across all input values
+| COVAR_POP(numeric1, numeric2)      | Returns the population covariance of the pair (*numeric1*, *numeric2*) across all input values
+| COVAR_SAMP(numeric1, numeric2)     | Returns the sample covariance of the pair (*numeric1*, *numeric2*) across all input values
+| REGR_SXX(numeric1, numeric2)       | Returns the sum of squares of the dependent expression in a linear regression model
+| REGR_SYY(numeric1, numeric2)       | Returns the sum of squares of the independent expression in a linear regression model
+
+Not implemented:
+* REGR_AVGX(numeric1, numeric2)
+* REGR_AVGY(numeric1, numeric2)
+* REGR_COUNT(numeric1, numeric2)
+* REGR_INTERCEPT(numeric1, numeric2)
+* REGR_R2(numeric1, numeric2)
+* REGR_SLOPE(numeric1, numeric2)
+* REGR_SXY(numeric1, numeric2)
+
+### Window functions
+
+| Operator syntax                           | Description
+| ----------------------------------------- | -----------
+| COUNT(value [, value ]* ) OVER window     | Returns the number of rows in *window* for which *value* is not null (wholly not null if *value* is composite)
+| COUNT(*) OVER window                      | Returns the number of rows in *window*
+| AVG(numeric) OVER window                  | Returns the average (arithmetic mean) of *numeric* across all values in *window*
+| SUM(numeric) OVER window                  | Returns the sum of *numeric* across all values in *window*
+| MAX(value) OVER window                    | Returns the maximum value of *value* across all values in *window*
+| MIN(value) OVER window                    | Returns the minimum value of *value* across all values in *window*
+| RANK() OVER window                        | Returns the rank of the current row with gaps; same as ROW_NUMBER of its first peer
+| DENSE_RANK() OVER window                  | Returns the rank of the current row without gaps; this function counts peer groups
+| ROW_NUMBER() OVER window                  | Returns the number of the current row within its partition, counting from 1
+| FIRST_VALUE(value) OVER window            | Returns *value* evaluated at the row that is the first row of the window frame
+| LAST_VALUE(value) OVER window             | Returns *value* evaluated at the row that is the last row of the window frame
+| LEAD(value, offset, default) OVER window  | Returns *value* evaluated at the row that is *offset* rows after the current row within the partition; if there is no such row, instead returns *default*. Both *offset* and *default* are evaluated with respect to the current row. If omitted, *offset* defaults to 1 and *default* to NULL
+| LAG(value, offset, default) OVER window   | Returns *value* evaluated at the row that is *offset* rows before the current row within the partition; if there is no such row, instead returns *default*. Both *offset* and *default* are evaluated with respect to the current row. If omitted, *offset* defaults to 1 and *default* to NULL
+| NTILE(value) OVER window                  | Returns an integer ranging from 1 to *value*, dividing the partition as equally as possible
+
+Not implemented:
+* COUNT(DISTINCT value) OVER window
+* FIRST_VALUE(value) IGNORE NULLS OVER window
+* LAST_VALUE(value) IGNORE NULLS OVER window
+* PERCENT_RANK(value) OVER window
+* CUME_DIST(value) OVER window
+* NTH_VALUE(value, nth) OVER window
+
+### Grouping functions
+
+| Operator syntax      | Description
+| -------------------- | -----------
+| GROUPING(expression) | Returns 1 if expression is rolled up in the current row's grouping set, 0 otherwise
+| GROUP_ID()           | Returns an integer that uniquely identifies the combination of grouping keys
+| GROUPING_ID(expression [, expression ] * ) | Returns a bit vector of the given grouping expressions


[12/13] incubator-calcite git commit: [CALCITE-505] Support modifiable view

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/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 6e7fdf3..36d7788 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSchema;
@@ -27,14 +29,27 @@ import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeComparability;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+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.ModifiableView;
+import org.apache.calcite.schema.Path;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlAccessType;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.ObjectSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -46,26 +61,31 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 
-import java.util.ArrayList;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 
 /**
  * Mock implementation of {@link SqlValidatorCatalogReader} which returns tables
  * "EMP", "DEPT", "BONUS", "SALGRADE" (same as Oracle's SCOTT schema).
+ * Also two streams "ORDERS", "SHIPMENTS";
+ * and a view "EMP_20".
  */
 public class MockCatalogReader implements Prepare.CatalogReader {
   //~ Static fields/initializers ---------------------------------------------
@@ -100,12 +120,11 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     this.typeFactory = typeFactory;
     this.caseSensitive = caseSensitive;
     if (caseSensitive) {
-      tables = new HashMap<List<String>, MockTable>();
-      schemas = new HashMap<String, MockSchema>();
+      tables = Maps.newHashMap();
+      schemas = Maps.newHashMap();
     } else {
-      tables = new TreeMap<List<String>, MockTable>(
-          CASE_INSENSITIVE_LIST_COMPARATOR);
-      schemas = new TreeMap<String, MockSchema>(String.CASE_INSENSITIVE_ORDER);
+      tables = Maps.newTreeMap(CASE_INSENSITIVE_LIST_COMPARATOR);
+      schemas = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
     }
   }
 
@@ -147,7 +166,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerSchema(salesSchema);
 
     // Register "EMP" table.
-    MockTable empTable = MockTable.create(this, salesSchema, "EMP", false);
+    final MockTable empTable =
+        MockTable.create(this, salesSchema, "EMP", false);
     empTable.addColumn("EMPNO", intType);
     empTable.addColumn("ENAME", varchar20Type);
     empTable.addColumn("JOB", varchar10Type);
@@ -228,6 +248,122 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     shipmentsStream.addColumn("ORDERID", intType);
     registerTable(shipmentsStream);
 
+    // Register "EMP_20" view.
+    // Same columns as "EMP",
+    // but "DEPTNO" not visible and set to 20 by default
+    // and "SAL" is visible but must be greater than 1000
+    MockTable emp20View = new MockTable(this, salesSchema.getCatalogName(),
+        salesSchema.name, "EMP_20", false) {
+      private final Table table = empTable.unwrap(Table.class);
+      private final ImmutableIntList mapping =
+          ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8);
+
+      @Override public RelNode toRel(ToRelContext context) {
+        // Expand to the equivalent of:
+        //   SELECT EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, SLACKER
+        //   FROM EMP
+        //   WHERE DEPTNO = 20 AND SAL > 1000
+        RelNode rel = LogicalTableScan.create(context.getCluster(), empTable);
+        final RexBuilder rexBuilder = context.getCluster().getRexBuilder();
+        rel = LogicalFilter.create(rel,
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.AND,
+                rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                    rexBuilder.makeInputRef(rel, 7),
+                    rexBuilder.makeExactLiteral(BigDecimal.valueOf(20))),
+                rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
+                    rexBuilder.makeInputRef(rel, 5),
+                    rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000)))));
+        final List<RelDataTypeField> fieldList =
+            rel.getRowType().getFieldList();
+        final List<Pair<RexNode, String>> projects =
+            new AbstractList<Pair<RexNode, String>>() {
+              @Override public Pair<RexNode, String> get(int index) {
+                return RexInputRef.of2(mapping.get(index), fieldList);
+              }
+
+              @Override public int size() {
+                return mapping.size();
+              }
+            };
+        return LogicalProject.create(rel, Pair.left(projects),
+            Pair.right(projects));
+      }
+
+      @Override public <T> T unwrap(Class<T> clazz) {
+        if (clazz.isAssignableFrom(ModifiableView.class)) {
+          return clazz.cast(
+              new JdbcTest.AbstractModifiableView() {
+                @Override public Table getTable() {
+                  return empTable.unwrap(Table.class);
+                }
+
+                @Override public Path getTablePath() {
+                  final ImmutableList.Builder<Pair<String, Schema>> builder =
+                      ImmutableList.builder();
+                  builder.add(Pair.<String, Schema>of(empTable.names.get(0), null));
+                  builder.add(Pair.<String, Schema>of(empTable.names.get(1), null));
+                  builder.add(Pair.<String, Schema>of(empTable.names.get(2), null));
+                  return Schemas.path(builder.build());
+//                  return empTable.names;
+                }
+
+                @Override public ImmutableIntList getColumnMapping() {
+                  return mapping;
+                }
+
+                @Override public RexNode getConstraint(RexBuilder rexBuilder,
+                    RelDataType tableRowType) {
+                  final RelDataTypeField deptnoField =
+                      tableRowType.getFieldList().get(7);
+                  final RelDataTypeField salField =
+                      tableRowType.getFieldList().get(5);
+                  final List<RexNode> nodes = Arrays.asList(
+                      rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                          rexBuilder.makeInputRef(deptnoField.getType(),
+                              deptnoField.getIndex()),
+                          rexBuilder.makeExactLiteral(BigDecimal.valueOf(20L),
+                              deptnoField.getType())),
+                      rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
+                          rexBuilder.makeInputRef(salField.getType(),
+                              salField.getIndex()),
+                          rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000L),
+                              salField.getType())));
+                  return RexUtil.composeConjunction(rexBuilder, nodes, false);
+                }
+
+                @Override public RelDataType
+                getRowType(final RelDataTypeFactory typeFactory) {
+                  return typeFactory.createStructType(
+                      new AbstractList<Map.Entry<String, RelDataType>>() {
+                        @Override public Map.Entry<String, RelDataType>
+                        get(int index) {
+                          return table.getRowType(typeFactory).getFieldList()
+                              .get(mapping.get(index));
+                        }
+
+                        @Override public int size() {
+                          return mapping.size();
+                        }
+                      }
+                  );
+                }
+              });
+        }
+        return super.unwrap(clazz);
+      }
+    };
+    salesSchema.addTable(Util.last(emp20View.getQualifiedName()));
+    emp20View.addColumn("EMPNO", intType);
+    emp20View.addColumn("ENAME", varchar20Type);
+    emp20View.addColumn("JOB", varchar10Type);
+    emp20View.addColumn("MGR", intTypeNull);
+    emp20View.addColumn("HIREDATE", timestampType);
+    emp20View.addColumn("SAL", intType);
+    emp20View.addColumn("COMM", intType);
+    emp20View.addColumn("SLACKER", booleanType);
+    registerTable(emp20View);
+
     return this;
   }
 
@@ -296,7 +432,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
           .build();
     case 1:
       // looking for schema names
-      result = new ArrayList<SqlMoniker>();
+      result = Lists.newArrayList();
       for (MockSchema schema : schemas.values()) {
         final String catalogName = names.get(0);
         if (schema.getCatalogName().equals(catalogName)) {
@@ -312,7 +448,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       if (schema == null) {
         return Collections.emptyList();
       }
-      result = new ArrayList<SqlMoniker>();
+      result = Lists.newArrayList();
       for (String tableName : schema.tableNames) {
         result.add(
             new SqlMonikerImpl(
@@ -330,30 +466,6 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     return ImmutableList.of(DEFAULT_CATALOG, DEFAULT_SCHEMA);
   }
 
-  private MockSchema getMockSchema(List<String> names) {
-    return schemas.get(names.get(0));
-  }
-
-  public List<SqlMoniker> getAllSchemaObjectNames2(List<String> names) {
-    List<SqlMoniker> result = new ArrayList<SqlMoniker>();
-    if (names.isEmpty()) {
-      for (MockSchema schema : schemas.values()) {
-        result.add(
-            new SqlMonikerImpl(schema.name, SqlMonikerType.SCHEMA));
-      }
-    }
-    // looking for table names in the given schema
-    MockSchema schema = getMockSchema(names);
-    if (schema != null) {
-      for (String tableName : schema.tableNames) {
-        result.add(
-            new SqlMonikerImpl(
-                tableName, SqlMonikerType.TABLE));
-      }
-    }
-    return result;
-  }
-
   public RelDataTypeField field(RelDataType rowType, String alias) {
     return SqlValidatorUtil.lookupField(caseSensitive, elideRecord, rowType,
         alias);
@@ -380,8 +492,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   private static List<RelCollation> deduceMonotonicity(
       Prepare.PreparingTable table) {
-    final List<RelCollation> collationList =
-        new ArrayList<RelCollation>();
+    final List<RelCollation> collationList = Lists.newArrayList();
 
     // Deduce which fields the table is sorted on.
     int i = -1;
@@ -407,7 +518,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   /** Mock schema. */
   public static class MockSchema {
-    private final List<String> tableNames = new ArrayList<String>();
+    private final List<String> tableNames = Lists.newArrayList();
     private String name;
 
     public MockSchema(String name) {
@@ -428,13 +539,13 @@ public class MockCatalogReader implements Prepare.CatalogReader {
    * {@link org.apache.calcite.prepare.Prepare.PreparingTable}.
    */
   public static class MockTable implements Prepare.PreparingTable {
-    private final MockCatalogReader catalogReader;
+    protected final MockCatalogReader catalogReader;
     private final boolean stream;
     private final List<Map.Entry<String, RelDataType>> columnList =
         Lists.newArrayList();
     private RelDataType rowType;
     private List<RelCollation> collationList;
-    private final List<String> names;
+    protected final List<String> names;
     private final Set<String> monotonicColumnSet = Sets.newHashSet();
 
     public MockTable(MockCatalogReader catalogReader, String catalogName,
@@ -457,6 +568,34 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       if (clazz.isInstance(this)) {
         return clazz.cast(this);
       }
+      if (clazz.isAssignableFrom(Table.class)) {
+        return clazz.cast(
+            new JdbcTest.AbstractModifiableTable(Util.last(names)) {
+              @Override public RelDataType
+              getRowType(RelDataTypeFactory typeFactory) {
+                return typeFactory.createStructType(rowType.getFieldList());
+              }
+
+              @Override public Collection getModifiableCollection() {
+                return null;
+              }
+
+              @Override public <T> Queryable<T>
+              asQueryable(QueryProvider queryProvider, SchemaPlus schema,
+                  String tableName) {
+                return null;
+              }
+
+              @Override public Type getElementType() {
+                return null;
+              }
+
+              @Override public Expression getExpression(SchemaPlus schema,
+                  String tableName, Class clazz) {
+                return null;
+              }
+            });
+      }
       return null;
     }
 
@@ -515,10 +654,6 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       throw new UnsupportedOperationException();
     }
 
-    public void addColumn(int index, String name, RelDataType type) {
-      columnList.add(index, Pair.of(name, type));
-    }
-
     public void addColumn(String name, RelDataType type) {
       columnList.add(Pair.of(name, type));
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index 7c5ba6f..aaec45d 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -198,7 +198,7 @@ public class ReflectiveSchemaTest {
     schema.add("emps_view",
         ViewTable.viewMacro(schema,
             "select * from \"hr\".\"emps\" where \"deptno\" = 10",
-            null));
+            null, null));
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
     ResultSet resultSet = connection.createStatement().executeQuery(
         "select *\n"
@@ -224,17 +224,17 @@ public class ReflectiveSchemaTest {
     schema.add("emps",
         ViewTable.viewMacro(schema,
             "select * from \"emps\" where \"deptno\" = 10",
-            Collections.singletonList("hr")));
+            Collections.singletonList("hr"), null));
     schema.add("hr_emps",
         ViewTable.viewMacro(schema,
             "select * from \"emps\"",
-            Collections.singletonList("hr")));
+            Collections.singletonList("hr"), null));
     schema.add("s_emps",
         ViewTable.viewMacro(schema,
             "select * from \"emps\"",
-            Collections.singletonList("s")));
+            Collections.singletonList("s"), null));
     schema.add("null_emps",
-        ViewTable.viewMacro(schema, "select * from \"emps\"", null));
+        ViewTable.viewMacro(schema, "select * from \"emps\"", null, null));
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
     final Statement statement = connection.createStatement();
     ResultSet resultSet;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 1853244..08a01a9 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1142,6 +1142,22 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
         .convertsTo("${plan}");
   }
 
+  @Test public void testInsert() {
+    sql("insert into emp (deptno, empno, ename) values (10, 150, 'Fred')")
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testSelectView() {
+    // translated condition: deptno = 20 and sal > 1000 and empno > 100
+    sql("select * from emp_20 where empno > 100")
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testInsertView() {
+    sql("insert into emp_20 (empno, ename) values (150, 'Fred')")
+        .convertsTo("${plan}");
+  }
+
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-695">[CALCITE-695]

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/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 a2e6c20..2bf64c7 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7079,6 +7079,12 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester1.checkQuery("select empNo from (select Empno from emP)");
   }
 
+  @Test public void testInsert() {
+    tester.checkQuery("insert into emp (empno, deptno) values (1, 1)");
+    tester.checkQuery("insert into emp (empno, deptno)\n"
+        + "select 1, 1 from (values 'a')");
+  }
+
   @Test public void testStream() {
     sql("select stream * from orders").ok();
     sql("select stream * from ^emp^")

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index ef17e98..b3be8ce 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2578,4 +2578,44 @@ LogicalProject(DEPTNO=[$0], B=[>($1, $2)])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testInsert">
+        <Resource name="sql">
+            <![CDATA[insert into emp (deptno, empno, ename) values (10, 150, 'Fred')]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalTableModify(table=[[CATALOG, SALES, EMP]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalProject(EMPNO=[$1], ENAME=[$2], JOB=[null], MGR=[null], HIREDATE=[null], SAL=[null], COMM=[null], DEPTNO=[$0], SLACKER=[null])
+    LogicalValues(tuples=[[{ 10, 150, 'Fred' }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSelectView">
+        <Resource name="sql">
+            <![CDATA[select * from emp_20 where empno > 100]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$7])
+  LogicalFilter(condition=[>($0, 100)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
+      LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testInsertView">
+        <Resource name="sql">
+            <![CDATA[insert into emp_20 (empno, ename) values (150, 'Fred')]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalTableModify(table=[[CATALOG, SALES, EMP]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalFilter(condition=[>($5, 1000)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[CAST($2):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], MGR=[$3], HIREDATE=[CAST($4):TIMESTAMP(0) NOT NULL], SAL=[CAST($5):INTEGER NOT NULL], COMM=[CAST($6):INTEGER NOT NULL], DEPTNO=[20], SLACKER=[CAST($7):BOOLEAN NOT NULL])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[null], MGR=[null], HIREDATE=[null], SAL=[null], COMM=[null], SLACKER=[null])
+        LogicalValues(tuples=[[{ 150, 'Fred' }]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/doc/model.md
----------------------------------------------------------------------
diff --git a/doc/model.md b/doc/model.md
index 1b2666d..3f49159 100644
--- a/doc/model.md
+++ b/doc/model.md
@@ -224,7 +224,8 @@ Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
 {
   name: 'female_emps',
   type: 'view',
-  sql: "select * from emps where gender = 'F'"
+  sql: "select * from emps where gender = 'F'",
+  modifiable: true
 }
 ```
 
@@ -236,6 +237,29 @@ Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
 `path` (optional list) is the SQL path to resolve the query. If not
 specified, defaults to the current schema.
 
+`modifiable` (optional boolean) is whether the view is modifiable.
+If null or not specified, Calcite deduces whether the view is modifiable.
+
+A view is modifiable if contains only SELECT, FROM, WHERE (no JOIN, aggregation
+or sub-queries) and every column:
+* is specified once in the SELECT clause; or
+* occurs in the WHERE clause with a `column = literal` predicate; or
+* is nullable.
+
+The second clause allows Calcite to automatically provide the correct value for
+hidden columns. It is useful in multi-tenant environments, where the `tenantId`
+column is hidden, mandatory (NOT NULL), and has a constant value for a
+particular view.
+
+Errors regarding modifiable views:
+* If a view is marked `modifiable: true` and is not modifiable, Calcite throws
+  an error while reading the schema.
+* If you submit an INSERT, UPDATE or UPSERT command to a non-modifiable view,
+  Calcite throws an error when validating the statement.
+* If a DML statement creates a row that would not appear in the view
+  (for example, a row in `female_emps`, above, with `gender = 'M'`),
+  Calcite throws an error when executing the statement.
+
 ### Custom Table
 
 Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
index c9a22de..54b49f1 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
@@ -43,7 +43,9 @@ public class ConstantExpression extends Expression {
         if (primitive != null) {
           clazz = primitive.boxClass;
         }
-        if (!clazz.isInstance(value)) {
+        if (!clazz.isInstance(value)
+            && !((clazz == Float.class || clazz == Double.class)
+                && value instanceof BigDecimal)) {
           throw new AssertionError(
               "value " + value + " does not match type " + type);
         }
@@ -85,6 +87,7 @@ public class ConstantExpression extends Expression {
       return writer;
     }
     final Primitive primitive = Primitive.of(type);
+    final BigDecimal bigDecimal;
     if (primitive != null) {
       switch (primitive) {
       case BYTE:
@@ -96,8 +99,28 @@ public class ConstantExpression extends Expression {
       case LONG:
         return writer.append(value).append("L");
       case FLOAT:
+        if (value instanceof BigDecimal) {
+          bigDecimal = (BigDecimal) value;
+        } else {
+          bigDecimal = BigDecimal.valueOf((Float) value);
+        }
+        if (bigDecimal.precision() > 6) {
+          return writer.append("Float.intBitsToFloat(")
+              .append(Float.floatToIntBits(bigDecimal.floatValue()))
+              .append(")");
+        }
         return writer.append(value).append("F");
       case DOUBLE:
+        if (value instanceof BigDecimal) {
+          bigDecimal = (BigDecimal) value;
+        } else {
+          bigDecimal = BigDecimal.valueOf((Double) value);
+        }
+        if (bigDecimal.precision() > 10) {
+          return writer.append("Double.longBitsToDouble(")
+              .append(Double.doubleToLongBits(bigDecimal.doubleValue()))
+              .append("L)");
+        }
         return writer.append(value).append("D");
       default:
         return writer.append(value);
@@ -115,7 +138,7 @@ public class ConstantExpression extends Expression {
           .append(((Enum) value).name());
     }
     if (value instanceof BigDecimal) {
-      BigDecimal bigDecimal = ((BigDecimal) value).stripTrailingZeros();
+      bigDecimal = ((BigDecimal) value).stripTrailingZeros();
       try {
         final int scale = bigDecimal.scale();
         final long exact = bigDecimal.scaleByPowerOfTen(scale).longValueExact();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
index 3936d5f..fae932e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
@@ -566,7 +566,11 @@ public abstract class Expressions {
       if (primitive != null) {
         clazz = primitive.boxClass;
       }
-      if (!clazz.isInstance(value)) {
+      if ((clazz == Float.class || clazz == Double.class)
+          && value instanceof BigDecimal) {
+        // Don't try to convert the value of float and double literals.
+        // We'd experience rounding, e.g. 3.2 becomes 3.1999998.
+      } else if (!clazz.isInstance(value)) {
         String stringValue = String.valueOf(value);
         if (type == BigDecimal.class) {
           value = new BigDecimal(stringValue);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
index 7a4f65a..338251b 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
@@ -34,6 +34,7 @@ import org.apache.calcite.linq4j.function.Functions;
 import org.apache.calcite.linq4j.function.IntegerFunction1;
 import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 
@@ -43,6 +44,7 @@ import com.google.common.collect.Lists;
 
 import org.junit.Test;
 
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -1660,6 +1662,37 @@ public class Linq4jTest {
     Linq4jExample.main(new String[0]);
   }
 
+  /** We use BigDecimal to represent literals of float and double using
+   * BigDecimal, because we want an exact representation. */
+  @Test public void testApproxConstant() {
+    ConstantExpression c;
+    c = Expressions.constant(new BigDecimal("3.1"), float.class);
+    assertThat(Expressions.toString(c), equalTo("3.1F"));
+    c = Expressions.constant(new BigDecimal("-5.156"), float.class);
+    assertThat(Expressions.toString(c), equalTo("-5.156F"));
+    c = Expressions.constant(new BigDecimal("-51.6"), Float.class);
+    assertThat(Expressions.toString(c), equalTo("Float.valueOf(-51.6F)"));
+    c = Expressions.constant(new BigDecimal(Float.MAX_VALUE), Float.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Float.valueOf(Float.intBitsToFloat(2139095039))"));
+    c = Expressions.constant(new BigDecimal(Float.MIN_VALUE), Float.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Float.valueOf(Float.intBitsToFloat(1))"));
+
+    c = Expressions.constant(new BigDecimal("3.1"), double.class);
+    assertThat(Expressions.toString(c), equalTo("3.1D"));
+    c = Expressions.constant(new BigDecimal("-5.156"), double.class);
+    assertThat(Expressions.toString(c), equalTo("-5.156D"));
+    c = Expressions.constant(new BigDecimal("-51.6"), Double.class);
+    assertThat(Expressions.toString(c), equalTo("Double.valueOf(-51.6D)"));
+    c = Expressions.constant(new BigDecimal(Double.MAX_VALUE), Double.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Double.valueOf(Double.longBitsToDouble(9218868437227405311L))"));
+    c = Expressions.constant(new BigDecimal(Double.MIN_VALUE), Double.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Double.valueOf(Double.longBitsToDouble(1L))"));
+  }
+
   /** Employee. */
   public static class Employee {
     public final int empno;


[03/13] incubator-calcite git commit: [CALCITE-515] Add Apache headers to markdown files

Posted by jh...@apache.org.
[CALCITE-515] Add Apache headers to markdown files


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

Branch: refs/heads/master
Commit: 85887fe3516e6dcee2e11f95571a40bed3e1cfbb
Parents: 88599ef
Author: Julian Hyde <jh...@apache.org>
Authored: Mon May 11 17:24:59 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon May 11 17:24:59 2015 -0700

----------------------------------------------------------------------
 README.md                 | 16 ++++++++++++++++
 doc/HISTORY.md            | 16 ++++++++++++++++
 doc/HOWTO.md              | 16 ++++++++++++++++
 doc/INCUBATOR_PROPOSAL.md | 16 ++++++++++++++++
 doc/MODEL.md              | 16 ++++++++++++++++
 doc/REFERENCE.md          | 16 ++++++++++++++++
 doc/STREAM.md             | 16 ++++++++++++++++
 doc/TUTORIAL.md           | 19 +++++++++++++++++--
 doc/avatica.md            | 16 ++++++++++++++++
 pom.xml                   |  2 --
 10 files changed, 145 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 2c9b4a9..bf00288 100644
--- a/README.md
+++ b/README.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 [![Build Status](https://travis-ci.org/julianhyde/incubator-calcite.svg?branch=master)](https://travis-ci.org/julianhyde/incubator-calcite)
 
 # Apache Calcite

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/HISTORY.md
----------------------------------------------------------------------
diff --git a/doc/HISTORY.md b/doc/HISTORY.md
index a30ba42..417cb20 100644
--- a/doc/HISTORY.md
+++ b/doc/HISTORY.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Apache Calcite release history
 
 For a full list of releases, see

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/HOWTO.md
----------------------------------------------------------------------
diff --git a/doc/HOWTO.md b/doc/HOWTO.md
index 3a42179..65f2ebd 100644
--- a/doc/HOWTO.md
+++ b/doc/HOWTO.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Calcite HOWTO
 
 Here's some miscellaneous documentation about using Calcite and its various

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/INCUBATOR_PROPOSAL.md
----------------------------------------------------------------------
diff --git a/doc/INCUBATOR_PROPOSAL.md b/doc/INCUBATOR_PROPOSAL.md
index 3f7455d..2b07205 100644
--- a/doc/INCUBATOR_PROPOSAL.md
+++ b/doc/INCUBATOR_PROPOSAL.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Optiq
 
 ## Abstract

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/MODEL.md
----------------------------------------------------------------------
diff --git a/doc/MODEL.md b/doc/MODEL.md
index f533e75..484a302 100644
--- a/doc/MODEL.md
+++ b/doc/MODEL.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Calcite JSON model reference
 
 ## Elements

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/REFERENCE.md
----------------------------------------------------------------------
diff --git a/doc/REFERENCE.md b/doc/REFERENCE.md
index b67ad57..03f66e3 100644
--- a/doc/REFERENCE.md
+++ b/doc/REFERENCE.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Calcite SQL language reference
 
 ## SQL constructs

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/STREAM.md
----------------------------------------------------------------------
diff --git a/doc/STREAM.md b/doc/STREAM.md
index f324b37..4052ac1 100644
--- a/doc/STREAM.md
+++ b/doc/STREAM.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Calcite SQL extensions for streaming
 
 ## Introduction

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/TUTORIAL.md
----------------------------------------------------------------------
diff --git a/doc/TUTORIAL.md b/doc/TUTORIAL.md
index d12fbeb..f0edafc 100644
--- a/doc/TUTORIAL.md
+++ b/doc/TUTORIAL.md
@@ -1,5 +1,20 @@
-CSV Adapter Tutorial
-=====================
+<!--
+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.
+-->
+# CSV Adapter Tutorial
 
 Calcite-example-CSV is a fully functional adapter for
 <a href="https://github.com/apache/incubator-calcite">Calcite</a> that reads

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/doc/avatica.md
----------------------------------------------------------------------
diff --git a/doc/avatica.md b/doc/avatica.md
index 32cc905..86899d3 100644
--- a/doc/avatica.md
+++ b/doc/avatica.md
@@ -1,3 +1,19 @@
+<!--
+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.
+-->
 # Avatica
 
 ## Introduction

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/85887fe3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1572a57..c573a84 100644
--- a/pom.xml
+++ b/pom.xml
@@ -594,8 +594,6 @@ limitations under the License.
             <artifactId>apache-rat-plugin</artifactId>
             <configuration>
               <excludes>
-                <exclude>*.md</exclude>
-                <exclude>doc/*.md</exclude>
                 <exclude>src/main/resources/META-INF/services/java.sql.Driver</exclude>
                 <exclude>**/src/test/resources/**/*.csv</exclude>
                 <exclude>**/src/test/resources/bug/archers.json</exclude>