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 2014/12/10 19:08:04 UTC

[1/2] incubator-calcite git commit: [CALCITE-493] Add EXTEND clause, for defining columns and their types at query/DML time

Repository: incubator-calcite
Updated Branches:
  refs/heads/master 155d24f1a -> a640e58cf


[CALCITE-493] Add EXTEND clause, for defining columns and their types at query/DML time


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

Branch: refs/heads/master
Commit: ecc5f4c2fff0a7a2a310b109b070d402c156f25a
Parents: 155d24f
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Dec 8 17:15:23 2014 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 10 08:38:18 2014 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       | 76 ++++++++++++++++++--
 .../calcite/plan/RelOptAbstractTable.java       |  5 ++
 .../org/apache/calcite/plan/RelOptTable.java    |  4 ++
 .../apache/calcite/prepare/RelOptTableImpl.java | 15 ++++
 .../apache/calcite/schema/ExtensibleTable.java  | 45 ++++++++++++
 .../org/apache/calcite/sql/SqlDataTypeSpec.java | 53 +++++++++++---
 .../java/org/apache/calcite/sql/SqlDelete.java  |  8 +--
 .../java/org/apache/calcite/sql/SqlInsert.java  |  6 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |  7 +-
 .../java/org/apache/calcite/sql/SqlMerge.java   |  6 +-
 .../java/org/apache/calcite/sql/SqlUpdate.java  |  6 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  5 ++
 .../sql/validate/IdentifierNamespace.java       | 52 ++++++++++++--
 .../calcite/sql/validate/SqlValidatorImpl.java  | 46 +++++++-----
 .../calcite/sql/validate/SqlValidatorUtil.java  | 35 ++++-----
 .../calcite/sql/validate/TableNamespace.java    | 41 +++++++++--
 .../calcite/sql/parser/SqlParserTest.java       | 37 +++++++++-
 .../apache/calcite/sql/test/SqlAdvisorTest.java |  2 +
 .../apache/calcite/test/MockCatalogReader.java  | 48 ++++++++-----
 .../apache/calcite/test/RelOptRulesTest.java    |  2 +-
 .../calcite/test/SqlToRelConverterTest.java     | 42 +++++------
 .../apache/calcite/test/SqlToRelTestBase.java   | 14 +++-
 .../apache/calcite/test/SqlValidatorTest.java   |  9 +++
 .../calcite/test/SqlToRelConverterTest.xml      | 11 +++
 24 files changed, 460 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 23c897e..4b28e3b 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -205,6 +205,12 @@ public class ${parser.class} extends SqlAbstractParserImpl
     {
         return SqlStmtEof();
     }
+
+    private SqlNode extend(SqlNode table, SqlNodeList extendList) {
+        return SqlStdOperatorTable.EXTEND.createCall(
+            table.getParserPosition().plus(extendList.getParserPosition()),
+            table, extendList);
+    }
 }
 
 PARSER_END(${parser.class})
@@ -1158,7 +1164,8 @@ SqlNode SqlInsert() :
  */
 SqlNode SqlDelete() :
 {
-    SqlIdentifier table;
+    SqlNode table;
+    SqlNodeList extendList = null;
     SqlIdentifier alias = null;
     SqlNode condition;
     SqlParserPos pos;
@@ -1172,6 +1179,12 @@ SqlNode SqlDelete() :
     {
 
     }
+    [
+        [ <EXTEND> ]
+        extendList = ExtendList() {
+            table = extend(table, extendList);
+        }
+    ]
     [ [ <AS> ] alias = SimpleIdentifier() ]
     condition = WhereOpt()
     {
@@ -1184,7 +1197,8 @@ SqlNode SqlDelete() :
  */
 SqlNode SqlUpdate() :
 {
-    SqlIdentifier table;
+    SqlNode table;
+    SqlNodeList extendList = null;
     SqlIdentifier alias = null;
     SqlNode condition;
     SqlNodeList sourceExpressionList;
@@ -1200,6 +1214,12 @@ SqlNode SqlUpdate() :
         targetColumnList = new SqlNodeList(pos);
         sourceExpressionList = new SqlNodeList(pos);
     }
+    [
+        [ <EXTEND> ]
+        extendList = ExtendList() {
+            table = extend(table, extendList);
+        }
+    ]
     [ [ <AS> ] alias = SimpleIdentifier() ]
     <SET> id = SimpleIdentifier()
     {
@@ -1233,7 +1253,8 @@ SqlNode SqlUpdate() :
  */
 SqlNode SqlMerge() :
 {
-    SqlIdentifier table;
+    SqlNode table;
+    SqlNodeList extendList = null;
     SqlIdentifier alias = null;
     SqlNode sourceTableRef;
     SqlNode condition;
@@ -1246,6 +1267,12 @@ SqlNode SqlMerge() :
     {
         mergePos = getPos();
     }
+    [
+        [ <EXTEND> ]
+        extendList = ExtendList() {
+            table = extend(table, extendList);
+        }
+    ]
     [ [ <AS> ] alias = SimpleIdentifier() ]
 
     <USING> sourceTableRef = TableRef()
@@ -1265,7 +1292,7 @@ SqlNode SqlMerge() :
     }
 }
 
-SqlUpdate WhenMatchedClause(SqlIdentifier table, SqlIdentifier alias) :
+SqlUpdate WhenMatchedClause(SqlNode table, SqlIdentifier alias) :
 {
     SqlIdentifier id;
     SqlParserPos pos;
@@ -1303,7 +1330,7 @@ SqlUpdate WhenMatchedClause(SqlIdentifier table, SqlIdentifier alias) :
     }
 }
 
-SqlInsert WhenNotMatchedClause(SqlIdentifier table) :
+SqlInsert WhenNotMatchedClause(SqlNode table) :
 {
     SqlParserPos pos, insertPos;
     List keywords = new ArrayList();
@@ -1592,6 +1619,7 @@ SqlNode TableRef() :
 {
     SqlNode tableRef;
     SqlNode over;
+    SqlNodeList extendList = null;
     String alias;
     SqlParserPos pos;
     SqlNodeList args;
@@ -1606,6 +1634,12 @@ SqlNode TableRef() :
     (
         LOOKAHEAD(2)
         tableRef = CompoundIdentifier()
+        [
+            [ <EXTEND> ]
+            extendList = ExtendList() {
+                tableRef = extend(tableRef, extendList);
+            }
+        ]
         over = TableOverOpt()
         {
             if (over != null) {
@@ -1730,6 +1764,37 @@ SqlNode TableRef() :
     }
 }
 
+SqlNodeList ExtendList() :
+{
+    SqlParserPos pos;
+    List<SqlNode> list = Lists.newArrayList();
+}
+{
+    <LPAREN> { pos = getPos(); }
+    ColumnType(list)
+    (
+        <COMMA> ColumnType(list)
+    )*
+    <RPAREN> {
+        return new SqlNodeList(list, pos.plus(getPos()));
+    }
+}
+
+void ColumnType(List<SqlNode> list) :
+{
+    SqlIdentifier name;
+    SqlDataTypeSpec type;
+}
+{
+    name = SimpleIdentifier()
+    type = DataType()
+    [ <NOT> <NULL> { type = type.withNullable(false); } ]
+    {
+        list.add(name);
+        list.add(type);
+    }
+}
+
 SqlNode TableFunctionCall(SqlParserPos pos) :
 {
     SqlNode call;
@@ -4631,6 +4696,7 @@ SqlPostfixOperator PostfixRowOperator() :
     | < EXP: "EXP" >
     | < EXPLAIN: "EXPLAIN" >
     | < EXTERNAL: "EXTERNAL" >
+    | < EXTEND: "EXTEND" >
     | < EXTRACT: "EXTRACT" >
     | < FALSE: "FALSE" >
     | < FETCH: "FETCH" >

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
index 2c3099d..f672ef3 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
@@ -94,6 +95,10 @@ public abstract class RelOptAbstractTable implements RelOptTable {
   public Expression getExpression(Class clazz) {
     throw new UnsupportedOperationException();
   }
+
+  public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+    throw new UnsupportedOperationException();
+  }
 }
 
 // End RelOptAbstractTable.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
index c628f71..7f069e0 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
@@ -20,6 +20,7 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import java.util.List;
@@ -96,6 +97,9 @@ public interface RelOptTable {
    */
   Expression getExpression(Class clazz);
 
+  /** Returns a table with the given extra fields. */
+  RelOptTable extend(List<RelDataTypeField> extendedFields);
+
   /** Can expand a view into relational expressions. */
   interface ViewExpander {
     RelNode expandView(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/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 68e0dcf..f91c45f 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -23,9 +23,12 @@ import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 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.ProjectableFilterableTable;
 import org.apache.calcite.schema.QueryableTable;
@@ -160,6 +163,18 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
     return expressionFunction.apply(clazz);
   }
 
+  public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+    if (table instanceof ExtensibleTable) {
+      final Table extendedTable =
+          ((ExtensibleTable) table).extend(extendedFields);
+      final RelDataType extendedRowType =
+          extendedTable.getRowType(schema.getTypeFactory());
+      return new RelOptTableImpl(schema, extendedRowType, names, extendedTable,
+          expressionFunction, rowCount);
+    }
+    throw new RuntimeException("Cannot extend " + table); // TODO: user error
+  }
+
   public double getRowCount() {
     if (rowCount != null) {
       return rowCount;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/schema/ExtensibleTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ExtensibleTable.java b/core/src/main/java/org/apache/calcite/schema/ExtensibleTable.java
new file mode 100644
index 0000000..7469cd7
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/ExtensibleTable.java
@@ -0,0 +1,45 @@
+/*
+ * 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.RelDataTypeField;
+
+import java.util.List;
+
+/**
+ * Table whose row type can be extended to include extra fields.
+ *
+ * <p>In some storage systems, especially those with "late schema", there may
+ * exist columns that have values in the table but which are not declared in
+ * the table schema. However, a particular query may wish to reference these
+ * columns as if they were defined in the schema. Calling the {@link #extend}
+ * method creates a temporarily extended table schema.
+ *
+ * <p>If the table implements extended interfaces such as
+ * {@link org.apache.calcite.schema.ScannableTable},
+ * {@link org.apache.calcite.schema.FilterableTable} or
+ * {@link org.apache.calcite.schema.ProjectableFilterableTable}, you may wish
+ * to make the table returned from {@link #extend} implement these interfaces
+ * as well.
+ */
+public interface ExtensibleTable extends Table {
+  /** Returns a table that has the row type of this table plus the given
+   * fields. */
+  Table extend(List<RelDataTypeField> fields);
+}
+
+// End ExtensibleTable.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
index e366204..fd4f22c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
@@ -18,6 +18,7 @@ package org.apache.calcite.sql;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
@@ -61,10 +62,17 @@ public class SqlDataTypeSpec extends SqlNode {
   private final String charSetName;
   private final TimeZone timeZone;
 
+  /** Whether data type is allows nulls.
+   *
+   * <p>Nullable is nullable! Null means "not specified". E.g.
+   * {@code CAST(x AS INTEGER)} preserves has the same nullability as {@code x}.
+   */
+  private Boolean nullable;
+
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a type specification.
+   * Creates a type specification representing a regular, non-collection type.
    */
   public SqlDataTypeSpec(
       final SqlIdentifier typeName,
@@ -73,13 +81,7 @@ public class SqlDataTypeSpec extends SqlNode {
       String charSetName,
       TimeZone timeZone,
       SqlParserPos pos) {
-    super(pos);
-    this.collectionsTypeName = null;
-    this.typeName = typeName;
-    this.scale = scale;
-    this.precision = precision;
-    this.charSetName = charSetName;
-    this.timeZone = timeZone;
+    this(null, typeName, precision, scale, charSetName, timeZone, null, pos);
   }
 
   /**
@@ -92,13 +94,30 @@ public class SqlDataTypeSpec extends SqlNode {
       int scale,
       String charSetName,
       SqlParserPos pos) {
+    this(collectionsTypeName, typeName, precision, scale, charSetName, null,
+        null, pos);
+  }
+
+  /**
+   * Creates a type specification.
+   */
+  public SqlDataTypeSpec(
+      SqlIdentifier collectionsTypeName,
+      SqlIdentifier typeName,
+      int precision,
+      int scale,
+      String charSetName,
+      TimeZone timeZone,
+      Boolean nullable,
+      SqlParserPos pos) {
     super(pos);
     this.collectionsTypeName = collectionsTypeName;
     this.typeName = typeName;
-    this.scale = scale;
     this.precision = precision;
+    this.scale = scale;
     this.charSetName = charSetName;
-    this.timeZone = null;
+    this.timeZone = timeZone;
+    this.nullable = nullable;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -139,6 +158,16 @@ public class SqlDataTypeSpec extends SqlNode {
     return timeZone;
   }
 
+  /** Returns a copy of this data type specification with a given
+   * nullability. */
+  public SqlDataTypeSpec withNullable(Boolean nullable) {
+    if (SqlFunctions.eq(nullable, this.nullable)) {
+      return this;
+    }
+    return new SqlDataTypeSpec(collectionsTypeName, typeName, precision, scale,
+        charSetName, timeZone, nullable, getParserPosition());
+  }
+
   /**
    * Returns a new SqlDataTypeSpec corresponding to the component type if the
    * type spec is a collections type spec.<br>
@@ -326,6 +355,10 @@ public class SqlDataTypeSpec extends SqlNode {
       }
     }
 
+    if (nullable != null) {
+      type = typeFactory.createTypeWithNullability(type, nullable);
+    }
+
     return type;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDelete.java b/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
index 5401e2b..ed41657 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
@@ -32,7 +32,7 @@ public class SqlDelete extends SqlCall {
   public static final SqlSpecialOperator OPERATOR =
       new SqlSpecialOperator("DELETE", SqlKind.DELETE);
 
-  SqlIdentifier targetTable;
+  SqlNode targetTable;
   SqlNode condition;
   SqlSelect sourceSelect;
   SqlIdentifier alias;
@@ -41,7 +41,7 @@ public class SqlDelete extends SqlCall {
 
   public SqlDelete(
       SqlParserPos pos,
-      SqlIdentifier targetTable,
+      SqlNode targetTable,
       SqlNode condition,
       SqlSelect sourceSelect,
       SqlIdentifier alias) {
@@ -69,7 +69,7 @@ public class SqlDelete extends SqlCall {
   @Override public void setOperand(int i, SqlNode operand) {
     switch (i) {
     case 0:
-      targetTable = (SqlIdentifier) operand;
+      targetTable = operand;
       break;
     case 1:
       condition = operand;
@@ -88,7 +88,7 @@ public class SqlDelete extends SqlCall {
   /**
    * @return the identifier for the target table of the deletion
    */
-  public SqlIdentifier getTargetTable() {
+  public SqlNode getTargetTable() {
     return targetTable;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
index 71132b0..43861a9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
@@ -32,7 +32,7 @@ public class SqlInsert extends SqlCall {
       new SqlSpecialOperator("INSERT", SqlKind.INSERT);
 
   SqlNodeList keywords;
-  SqlIdentifier targetTable;
+  SqlNode targetTable;
   SqlNode source;
   SqlNodeList columnList;
 
@@ -40,7 +40,7 @@ public class SqlInsert extends SqlCall {
 
   public SqlInsert(SqlParserPos pos,
       SqlNodeList keywords,
-      SqlIdentifier targetTable,
+      SqlNode targetTable,
       SqlNode source,
       SqlNodeList columnList) {
     super(pos);
@@ -87,7 +87,7 @@ public class SqlInsert extends SqlCall {
   /**
    * @return the identifier for the target table of the insertion
    */
-  public SqlIdentifier getTargetTable() {
+  public SqlNode getTargetTable() {
     return targetTable;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 359f113..4499055 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -565,6 +565,10 @@ public enum SqlKind {
    */
   REINTERPRET,
 
+  /** The internal {@code EXTEND} operator that qualifies a table name in the
+   * {@code FROM} clause. */
+  EXTEND,
+
   /** The internal {@code CUBE} operator that occurs within a {@code GROUP BY}
    * clause. */
   CUBE,
@@ -624,7 +628,8 @@ public enum SqlKind {
   public static final Set<SqlKind> EXPRESSION =
       EnumSet.complementOf(
           EnumSet.of(
-              AS, DESCENDING, SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM,
+              AS, DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
+              SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM,
               LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
               NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
               WITH, WITH_ITEM));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
index 59317e3..77be4e7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
@@ -34,7 +34,7 @@ public class SqlMerge extends SqlCall {
   public static final SqlSpecialOperator OPERATOR =
       new SqlSpecialOperator("MERGE", SqlKind.MERGE);
 
-  SqlIdentifier targetTable;
+  SqlNode targetTable;
   SqlNode condition;
   SqlNode source;
   SqlUpdate updateCall;
@@ -45,7 +45,7 @@ public class SqlMerge extends SqlCall {
   //~ Constructors -----------------------------------------------------------
 
   public SqlMerge(SqlParserPos pos,
-      SqlIdentifier targetTable,
+      SqlNode targetTable,
       SqlNode condition,
       SqlNode source,
       SqlUpdate updateCall,
@@ -108,7 +108,7 @@ public class SqlMerge extends SqlCall {
   /**
    * @return the identifier for the target table of the merge
    */
-  public SqlIdentifier getTargetTable() {
+  public SqlNode getTargetTable() {
     return targetTable;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java b/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
index 6773aa8..23afb22 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
@@ -33,7 +33,7 @@ public class SqlUpdate extends SqlCall {
   public static final SqlSpecialOperator OPERATOR =
       new SqlSpecialOperator("UPDATE", SqlKind.UPDATE);
 
-  SqlIdentifier targetTable;
+  SqlNode targetTable;
   SqlNodeList targetColumnList;
   SqlNodeList sourceExpressionList;
   SqlNode condition;
@@ -43,7 +43,7 @@ public class SqlUpdate extends SqlCall {
   //~ Constructors -----------------------------------------------------------
 
   public SqlUpdate(SqlParserPos pos,
-      SqlIdentifier targetTable,
+      SqlNode targetTable,
       SqlNodeList targetColumnList,
       SqlNodeList sourceExpressionList,
       SqlNode condition,
@@ -102,7 +102,7 @@ public class SqlUpdate extends SqlCall {
   /**
    * @return the identifier for the target table of the update
    */
-  public SqlIdentifier getTargetTable() {
+  public SqlNode getTargetTable() {
     return targetTable;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index b260312..427ce95 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -177,6 +177,11 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlGroupingIdFunction GROUPING_ID =
       new SqlGroupingIdFunction();
 
+  /** {@code EXTEND} operator to add columns to a table's schema, as in
+   * {@code SELECT ... FROM emp EXTEND (horoscope VARCHAR(100))}. */
+  public static final SqlInternalOperator EXTEND =
+      new SqlInternalOperator("EXTEND", SqlKind.EXTEND);
+
   /**
    * String concatenation operator, '<code>||</code>'.
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
index 8e08504..89d4132 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
@@ -18,16 +18,23 @@ package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
+import javax.annotation.Nullable;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -40,6 +47,7 @@ public class IdentifierNamespace extends AbstractNamespace {
 
   private final SqlIdentifier id;
   private final SqlValidatorScope parentScope;
+  private final SqlNodeList extendList;
 
   /**
    * The underlying namespace. Often a {@link TableNamespace}.
@@ -58,22 +66,40 @@ public class IdentifierNamespace extends AbstractNamespace {
    * Creates an IdentifierNamespace.
    *
    * @param validator     Validator
-   * @param id            Identifier node
+   * @param id            Identifier node (or "identifier EXTEND column-list")
+   * @param extendList    Extension columns, or null
    * @param enclosingNode Enclosing node
    * @param parentScope   Parent scope which this namespace turns to in order to
-   *                      resolve objects
    */
   IdentifierNamespace(SqlValidatorImpl validator, SqlIdentifier id,
-      SqlNode enclosingNode, SqlValidatorScope parentScope) {
+      @Nullable SqlNodeList extendList, SqlNode enclosingNode,
+      SqlValidatorScope parentScope) {
     super(validator, enclosingNode);
     this.id = id;
+    this.extendList = extendList;
     this.parentScope = parentScope;
     assert parentScope != null;
-    assert id != null;
+  }
+
+  IdentifierNamespace(SqlValidatorImpl validator, SqlNode node,
+      SqlNode enclosingNode, SqlValidatorScope parentScope) {
+    this(validator, split(node).left, split(node).right, enclosingNode,
+        parentScope);
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  protected static Pair<SqlIdentifier, SqlNodeList> split(SqlNode node) {
+    switch (node.getKind()) {
+    case EXTEND:
+      final SqlCall call = (SqlCall) node;
+      return Pair.of((SqlIdentifier) call.getOperandList().get(0),
+          (SqlNodeList) call.getOperandList().get(1));
+    default:
+      return Pair.of((SqlIdentifier) node, null);
+    }
+  }
+
   public RelDataType validateImpl() {
     resolvedNamespace = parentScope.getTableNamespace(id.names);
     if (resolvedNamespace == null) {
@@ -110,6 +136,24 @@ public class IdentifierNamespace extends AbstractNamespace {
 
     RelDataType rowType = resolvedNamespace.getRowType();
 
+    if (extendList != null) {
+      final List<RelDataTypeField> fields = Lists.newArrayList();
+      final Iterator<SqlNode> extendIterator = extendList.iterator();
+      while (extendIterator.hasNext()) {
+        SqlIdentifier id = (SqlIdentifier) extendIterator.next();
+        SqlDataTypeSpec type = (SqlDataTypeSpec) extendIterator.next();
+        fields.add(
+            new RelDataTypeFieldImpl(id.getSimple(), fields.size(),
+                type.deriveType(validator)));
+      }
+
+      if (!(resolvedNamespace instanceof TableNamespace)) {
+        throw new RuntimeException("cannot convert");
+      }
+      resolvedNamespace = ((TableNamespace) resolvedNamespace).extend(fields);
+      rowType = resolvedNamespace.getRowType();
+    }
+
     // Build a list of monotonic expressions.
     final ImmutableList.Builder<Pair<SqlNode, SqlMonotonicity>> builder =
         ImmutableList.builder();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index a02d86a..5e97cec 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -80,6 +80,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
@@ -1195,9 +1196,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               selfJoinCond,
               condition);
     }
-    SqlIdentifier target =
-        (SqlIdentifier) updateCall.getTargetTable().clone(
-            SqlParserPos.ZERO);
+    SqlNode target =
+        updateCall.getTargetTable().clone(SqlParserPos.ZERO);
 
     // For the source, we need to anonymize the fields, so
     // that for a statement like UPDATE T SET I = I + 1,
@@ -1248,7 +1248,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * @return expression for unique identifier, or null to prevent conversion
    */
   protected SqlNode getSelfJoinExprForUpdate(
-      SqlIdentifier table,
+      SqlNode table,
       String alias) {
     return null;
   }
@@ -1707,6 +1707,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * @param enclosingNode Outermost node for namespace, including decorations
    *                      such as alias and sample clause
    * @param alias         Alias
+   * @param extendList    Definitions of extended columns
    * @param forceNullable Whether to force the type of namespace to be
    *                      nullable because it is in an outer join
    * @return registered node, usually the same as {@code node}
@@ -1717,6 +1718,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final SqlNode node,
       SqlNode enclosingNode,
       String alias,
+      SqlNodeList extendList,
       boolean forceNullable) {
     final SqlKind kind = node.getKind();
 
@@ -1783,6 +1785,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               expr,
               enclosingNode,
               alias,
+              extendList,
               forceNullable);
       if (newExpr != expr) {
         call.setOperand(0, newExpr);
@@ -1809,6 +1812,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               expr,
               enclosingNode,
               alias,
+              extendList,
               forceNullable);
       if (newExpr != expr) {
         call.setOperand(0, newExpr);
@@ -1844,6 +1848,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               left,
               left,
               null,
+              null,
               forceLeftNullable);
       if (newLeft != left) {
         join.setLeft(newLeft);
@@ -1861,6 +1866,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               right,
               right,
               null,
+              null,
               forceRightNullable);
       if (newRight != right) {
         join.setRight(newRight);
@@ -1873,9 +1879,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final SqlIdentifier id = (SqlIdentifier) node;
       final IdentifierNamespace newNs =
           new IdentifierNamespace(
-              this,
-              id,
-              enclosingNode,
+              this, id, extendList, enclosingNode,
               parentScope);
       registerNamespace(usingScope, alias, newNs, forceNullable);
       return newNode;
@@ -1887,6 +1891,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           ((SqlCall) node).operand(0),
           enclosingNode,
           alias,
+          extendList,
           forceNullable);
 
     case COLLECTION_TABLE:
@@ -1899,6 +1904,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               operand,
               enclosingNode,
               alias,
+              extendList,
               forceNullable);
       if (newOperand != operand) {
         call.setOperand(0, newOperand);
@@ -1940,6 +1946,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               operand,
               enclosingNode,
               alias,
+              extendList,
               forceNullable);
       if (newOperand != operand) {
         call.setOperand(0, newOperand);
@@ -1955,6 +1962,16 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
       return newNode;
 
+    case EXTEND:
+      final SqlCall extend = (SqlCall) node;
+      return registerFrom(parentScope,
+          usingScope,
+          extend.getOperandList().get(0),
+          extend,
+          alias,
+          (SqlNodeList) extend.getOperandList().get(1),
+          forceNullable);
+
     default:
       throw Util.unexpected(kind);
     }
@@ -2091,6 +2108,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               from,
               from,
               null,
+              null,
               false);
       if (newFrom != from) {
         select.setFrom(newFrom);
@@ -3889,7 +3907,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * Common base class for DML statement namespaces.
    */
   public static class DmlNamespace extends IdentifierNamespace {
-    protected DmlNamespace(SqlValidatorImpl validator, SqlIdentifier id,
+    protected DmlNamespace(SqlValidatorImpl validator, SqlNode id,
         SqlNode enclosingNode, SqlValidatorScope parentScope) {
       super(validator, id, enclosingNode, parentScope);
     }
@@ -3904,8 +3922,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     public InsertNamespace(SqlValidatorImpl validator, SqlInsert node,
         SqlNode enclosingNode, SqlValidatorScope parentScope) {
       super(validator, node.getTargetTable(), enclosingNode, parentScope);
-      this.node = node;
-      assert node != null;
+      this.node = Preconditions.checkNotNull(node);
     }
 
     public SqlInsert getNode() {
@@ -3922,8 +3939,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     public UpdateNamespace(SqlValidatorImpl validator, SqlUpdate node,
         SqlNode enclosingNode, SqlValidatorScope parentScope) {
       super(validator, node.getTargetTable(), enclosingNode, parentScope);
-      this.node = node;
-      assert node != null;
+      this.node = Preconditions.checkNotNull(node);
     }
 
     public SqlUpdate getNode() {
@@ -3940,8 +3956,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     public DeleteNamespace(SqlValidatorImpl validator, SqlDelete node,
         SqlNode enclosingNode, SqlValidatorScope parentScope) {
       super(validator, node.getTargetTable(), enclosingNode, parentScope);
-      this.node = node;
-      assert node != null;
+      this.node = Preconditions.checkNotNull(node);
     }
 
     public SqlDelete getNode() {
@@ -3958,8 +3973,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     public MergeNamespace(SqlValidatorImpl validator, SqlMerge node,
         SqlNode enclosingNode, SqlValidatorScope parentScope) {
       super(validator, node.getTargetTable(), enclosingNode, parentScope);
-      this.node = node;
-      assert node != null;
+      this.node = Preconditions.checkNotNull(node);
     }
 
     public SqlMerge getNode() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 3f694e4..2ae4af5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -79,25 +79,26 @@ public class SqlValidatorUtil {
       Prepare.CatalogReader catalogReader,
       String datasetName,
       boolean[] usedDataset) {
-    if (namespace.isWrapperFor(TableNamespace.class)) {
-      TableNamespace tableNamespace =
-          namespace.unwrap(TableNamespace.class);
-      final List<String> names = tableNamespace.getTable().getQualifiedName();
-      if ((datasetName != null)
-          && (catalogReader instanceof RelOptSchemaWithSampling)) {
-        return ((RelOptSchemaWithSampling) catalogReader)
-            .getTableForMember(
-                names,
-                datasetName,
-                usedDataset);
-      } else {
-        // Schema does not support substitution. Ignore the dataset,
-        // if any.
-        return catalogReader.getTableForMember(names);
-      }
-    } else {
+    if (!namespace.isWrapperFor(TableNamespace.class)) {
       return null;
     }
+    final TableNamespace tableNamespace =
+        namespace.unwrap(TableNamespace.class);
+    final List<String> names = tableNamespace.getTable().getQualifiedName();
+    RelOptTable table;
+    if (datasetName != null
+        && catalogReader instanceof RelOptSchemaWithSampling) {
+      final RelOptSchemaWithSampling reader =
+          (RelOptSchemaWithSampling) catalogReader;
+      table = reader.getTableForMember(names, datasetName, usedDataset);
+    } else {
+      // Schema does not support substitution. Ignore the data set, if any.
+      table = catalogReader.getTableForMember(names);
+    }
+    if (!tableNamespace.extendedFields.isEmpty()) {
+      table = table.extend(tableNamespace.extendedFields);
+    }
+    return table;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
index 1ec155e..4c70157 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
@@ -17,21 +17,42 @@
 package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlNode;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import java.util.List;
+
 /** Namespace based on a table from the catalog. */
 class TableNamespace extends AbstractNamespace {
   private final SqlValidatorTable table;
+  public final ImmutableList<RelDataTypeField> extendedFields;
 
   /** Creates a TableNamespace. */
-  TableNamespace(SqlValidatorImpl validator, SqlValidatorTable table) {
+  TableNamespace(SqlValidatorImpl validator, SqlValidatorTable table,
+      ImmutableList<RelDataTypeField> fields) {
     super(validator, null);
-    this.table = table;
-    assert table != null;
+    this.table = Preconditions.checkNotNull(table);
+    this.extendedFields = fields;
+  }
+
+  public TableNamespace(SqlValidatorImpl validator, SqlValidatorTable table) {
+    this(validator, table, ImmutableList.<RelDataTypeField>of());
   }
 
   protected RelDataType validateImpl() {
-    return table.getRowType();
+    if (extendedFields.isEmpty()) {
+      return table.getRowType();
+    }
+    final RelDataTypeFactory.FieldInfoBuilder builder =
+        validator.getTypeFactory().builder();
+    builder.addAll(table.getRowType().getFieldList());
+    builder.addAll(extendedFields);
+    return builder.build();
   }
 
   public SqlNode getNode() {
@@ -42,6 +63,18 @@ class TableNamespace extends AbstractNamespace {
   @Override public SqlValidatorTable getTable() {
     return table;
   }
+
+  /** Creates a TableNamespace based on the same table as this one, but with
+   * extended fields.
+   *
+   * <p>Extended fields are "hidden" or undeclared fields that may nevertheless
+   * be present if you ask for them. Phoenix uses them, for instance, to access
+   * rarely used fields in the underlying HBase table. */
+  public TableNamespace extend(List<RelDataTypeField> extendedFields) {
+    return new TableNamespace(validator, table,
+        ImmutableList.copyOf(
+            Iterables.concat(this.extendedFields, extendedFields)));
+  }
 }
 
 // End TableNamespace.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 64b8f4c..d9a5e0f 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -792,7 +792,7 @@ public class SqlParserTest {
         + "order by a")
         .ok("SELECT `DEPTNO`\n"
             + "FROM `EMP`\n"
-            + "GROUP BY (GROUPING_SETS(`DEPTNO`, (GROUPING_SETS(`E`, `D`)),, (CUBE(`X`, `Y`)), (ROLLUP(`P`, `Q`))))\n"
+            + "GROUP BY (GROUPING_SETS(`DEPTNO`, GROUPING_SETS(`E`, `D`),, CUBE(`X`, `Y`), ROLLUP(`P`, `Q`)))\n"
             + "ORDER BY `A`");
 
     sql("select deptno from emp\n"
@@ -1911,6 +1911,41 @@ public class SqlParserTest {
         "(?s).*Encountered \"\\( \\)\" at .*");
   }
 
+  /** Test case for [CALCITE-493]. */
+  @Test public void testTableExtend() {
+    sql("select * from emp extend (x int, y varchar(10) not null)")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10)))");
+    sql("select * from emp extend (x int, y varchar(10) not null) where true")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10)))\n"
+            + "WHERE TRUE");
+    // with table alias
+    sql("select * from emp extend (x int, y varchar(10) not null) as t")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10))) AS `T`");
+    // as previous, without AS
+    sql("select * from emp extend (x int, y varchar(10) not null) t")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10))) AS `T`");
+    // with table alias and column alias list
+    sql("select * from emp extend (x int, y varchar(10) not null) as t(a, b)")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10))) AS `T` (`A`, `B`)");
+    // as previous, without AS
+    sql("select * from emp extend (x int, y varchar(10) not null) t(a, b)")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10))) AS `T` (`A`, `B`)");
+    // omit EXTEND
+    sql("select * from emp (x int, y varchar(10) not null) t(a, b)")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10))) AS `T` (`A`, `B`)");
+    sql("select * from emp (x int, y varchar(10) not null) where x = y")
+        .ok("SELECT *\n"
+            + "FROM (EXTEND(`EMP`, `X`, INTEGER, `Y`, VARCHAR(10)))\n"
+            + "WHERE (`X` = `Y`)");
+  }
+
   @Test public void testExplicitTable() {
     check("table emp", "(TABLE `EMP`)");
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/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 f1bf067..bd88a5c 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
@@ -271,6 +271,8 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(UNION)",
           "KEYWORD(FULL)",
           "KEYWORD(ORDER)",
+          "KEYWORD(()",
+          "KEYWORD(EXTEND)",
           "KEYWORD(AS)",
           "KEYWORD(USING)",
           "KEYWORD(RIGHT)",

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/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 f915e1b..f070f3d 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -19,6 +19,7 @@ package org.apache.calcite.test;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationImpl;
@@ -46,6 +47,7 @@ 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.Ordering;
 
 import java.util.ArrayList;
@@ -68,7 +70,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   public static final Ordering<Iterable<String>>
   CASE_INSENSITIVE_LIST_COMPARATOR =
-      Ordering.<String>from(String.CASE_INSENSITIVE_ORDER).lexicographical();
+      Ordering.from(String.CASE_INSENSITIVE_ORDER).lexicographical();
 
   //~ Instance fields --------------------------------------------------------
 
@@ -139,7 +141,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerSchema(salesSchema);
 
     // Register "EMP" table.
-    MockTable empTable = new MockTable(this, salesSchema, "EMP");
+    MockTable empTable = MockTable.create(this, salesSchema, "EMP");
     empTable.addColumn("EMPNO", intType);
     empTable.addColumn("ENAME", varchar20Type);
     empTable.addColumn("JOB", varchar10Type);
@@ -152,13 +154,13 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerTable(empTable);
 
     // Register "DEPT" table.
-    MockTable deptTable = new MockTable(this, salesSchema, "DEPT");
+    MockTable deptTable = MockTable.create(this, salesSchema, "DEPT");
     deptTable.addColumn("DEPTNO", intType);
     deptTable.addColumn("NAME", varchar10Type);
     registerTable(deptTable);
 
     // Register "BONUS" table.
-    MockTable bonusTable = new MockTable(this, salesSchema, "BONUS");
+    MockTable bonusTable = MockTable.create(this, salesSchema, "BONUS");
     bonusTable.addColumn("ENAME", varchar20Type);
     bonusTable.addColumn("JOB", varchar10Type);
     bonusTable.addColumn("SAL", intType);
@@ -166,7 +168,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerTable(bonusTable);
 
     // Register "SALGRADE" table.
-    MockTable salgradeTable = new MockTable(this, salesSchema, "SALGRADE");
+    MockTable salgradeTable = MockTable.create(this, salesSchema, "SALGRADE");
     salgradeTable.addColumn("GRADE", intType);
     salgradeTable.addColumn("LOSAL", intType);
     salgradeTable.addColumn("HISAL", intType);
@@ -174,7 +176,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     // Register "EMP_ADDRESS" table
     MockTable contactAddressTable =
-        new MockTable(this, salesSchema, "EMP_ADDRESS");
+        MockTable.create(this, salesSchema, "EMP_ADDRESS");
     contactAddressTable.addColumn("EMPNO", intType);
     contactAddressTable.addColumn("HOME_ADDRESS", addressType);
     contactAddressTable.addColumn("MAILING_ADDRESS", addressType);
@@ -185,7 +187,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerSchema(customerSchema);
 
     // Register "CONTACT" table.
-    MockTable contactTable = new MockTable(this, customerSchema, "CONTACT");
+    MockTable contactTable = MockTable.create(this, customerSchema, "CONTACT");
     contactTable.addColumn("CONTACTNO", intType);
     contactTable.addColumn("FNAME", varchar10Type);
     contactTable.addColumn("LNAME", varchar10Type);
@@ -194,7 +196,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerTable(contactTable);
 
     // Register "ACCOUNT" table.
-    MockTable accountTable = new MockTable(this, customerSchema, "ACCOUNT");
+    MockTable accountTable = MockTable.create(this, customerSchema, "ACCOUNT");
     accountTable.addColumn("ACCTNO", intType);
     accountTable.addColumn("TYPE", varchar20Type);
     accountTable.addColumn("BALANCE", intType);
@@ -362,20 +364,25 @@ public class MockCatalogReader implements Prepare.CatalogReader {
    */
   public static class MockTable implements Prepare.PreparingTable {
     private final MockCatalogReader catalogReader;
-    private final List<Pair<String, RelDataType>> columnList =
-        new ArrayList<Pair<String, RelDataType>>();
+    private final List<Map.Entry<String, RelDataType>> columnList =
+        Lists.newArrayList();
     private RelDataType rowType;
     private List<RelCollation> collationList;
     private final List<String> names;
 
-    public MockTable(
-        MockCatalogReader catalogReader,
-        MockSchema schema,
-        String name) {
+    public MockTable(MockCatalogReader catalogReader, String catalogName,
+        String schemaName, String name) {
       this.catalogReader = catalogReader;
-      this.names =
-          ImmutableList.of(schema.getCatalogName(), schema.name, name);
+      this.names = ImmutableList.of(catalogName, schemaName, name);
+    }
+
+    public static MockTable create(MockCatalogReader catalogReader,
+        MockSchema schema, String name) {
+      MockTable table =
+          new MockTable(catalogReader, schema.getCatalogName(), schema.name,
+              name);
       schema.addTable(name);
+      return table;
     }
 
     public <T> T unwrap(Class<T> clazz) {
@@ -437,6 +444,15 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     public void addColumn(String name, RelDataType type) {
       columnList.add(Pair.of(name, type));
     }
+
+    public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+      final MockTable table = new MockTable(catalogReader, names.get(0),
+          names.get(1), names.get(2));
+      table.columnList.addAll(columnList);
+      table.columnList.addAll(extendedFields);
+      table.onRegister(catalogReader.typeFactory);
+      return table;
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 0147413..74235e8 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -607,7 +607,7 @@ public class RelOptRulesTest extends RelOptTestBase {
                     typeFactory.createSqlType(SqlTypeName.INTEGER);
                 for (int i = 0; i < 10; i++) {
                   String t = String.valueOf((char) ('A' + i));
-                  MockTable table = new MockTable(this, schema, t);
+                  MockTable table = MockTable.create(this, schema, t);
                   table.addColumn(t, intType);
                   registerTable(table);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/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 15f09ee..76e18ce 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -616,6 +616,11 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
         "${plan}");
   }
 
+  @Test public void testTableExtend() {
+    sql("select * from dept extend (x varchar(5) not null)")
+        .convertsTo("${plan}");
+  }
+
   @Test public void testExplicitTable() {
     check(
         "table emp",
@@ -678,8 +683,7 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
 
   @Test public void testCollectionTableWithCursorParam() {
     tester.withDecorrelation(false).assertConvertsTo(
-        "select * from table(dedup("
-            + "cursor(select ename from emp),"
+        "select * from table(dedup(" + "cursor(select ename from emp),"
             + " cursor(select name from dept), 'NAME'))",
         "${plan}");
   }
@@ -691,9 +695,7 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testUnnestSubquery() {
-    check(
-        "select*from unnest(multiset(select*from dept))",
-        "${plan}");
+    check("select*from unnest(multiset(select*from dept))", "${plan}");
   }
 
   @Test public void testMultisetSubquery() {
@@ -761,12 +763,10 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
 
   @Test public void testInValueListLong() {
     // Go over the default threshold of 20 to force a subquery.
-    check(
-        "select empno from emp where deptno in"
+    check("select empno from emp where deptno in"
             + " (10, 20, 30, 40, 50, 60, 70, 80, 90, 100"
             + ", 110, 120, 130, 140, 150, 160, 170, 180, 190"
-            + ", 200, 210, 220, 230)",
-        "${plan}");
+            + ", 200, 210, 220, 230)", "${plan}");
   }
 
   @Test public void testInUncorrelatedSubquery() {
@@ -839,15 +839,11 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testElement() {
-    check(
-        "select element(multiset[5]) from emp",
-        "${plan}");
+    check("select element(multiset[5]) from emp", "${plan}");
   }
 
   @Test public void testElementInValues() {
-    check(
-        "values element(multiset[5])",
-        "${plan}");
+    check("values element(multiset[5])", "${plan}");
   }
 
   @Test public void testUnionAll() {
@@ -1028,19 +1024,13 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     rel.explain(planWriter);
     pw.flush();
     TestUtil.assertEqualsVerbose(
-        "<RelNode type=\"LogicalProject\">\n"
-            + "\t<Property name=\"EXPR$0\">\n"
-            + "\t\t+(1, 2)\t</Property>\n"
-            + "\t<Property name=\"EXPR$1\">\n"
-            + "\t\t3\t</Property>\n"
-            + "\t<Inputs>\n"
+        "<RelNode type=\"LogicalProject\">\n" + "\t<Property name=\"EXPR$0\">\n"
+            + "\t\t+(1, 2)\t</Property>\n" + "\t<Property name=\"EXPR$1\">\n"
+            + "\t\t3\t</Property>\n" + "\t<Inputs>\n"
             + "\t\t<RelNode type=\"LogicalValues\">\n"
             + "\t\t\t<Property name=\"tuples\">\n"
-            + "\t\t\t\t[{ true }]\t\t\t</Property>\n"
-            + "\t\t\t<Inputs/>\n"
-            + "\t\t</RelNode>\n"
-            + "\t</Inputs>\n"
-            + "</RelNode>\n",
+            + "\t\t\t\t[{ true }]\t\t\t</Property>\n" + "\t\t\t<Inputs/>\n"
+            + "\t\t</RelNode>\n" + "\t</Inputs>\n" + "</RelNode>\n",
         Util.toLinux(sw.toString()));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 1d88316..32e2dbe 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -299,7 +299,7 @@ public abstract class SqlToRelTestBase {
     public void registerRules(RelOptPlanner planner) throws Exception {
     }
 
-      /** Mock column set. */
+    /** Mock column set. */
     protected class MockColumnSet implements RelOptTable {
       private final List<String> names;
       private final RelDataType rowType;
@@ -360,6 +360,14 @@ public abstract class SqlToRelTestBase {
       public Expression getExpression(Class clazz) {
         return null;
       }
+
+      public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+        final RelDataType extendedRowType = typeFactory.builder()
+            .addAll(rowType.getFieldList())
+            .addAll(extendedFields)
+            .build();
+        return new MockColumnSet(names, extendedRowType, collationList);
+      }
     }
   }
 
@@ -382,6 +390,10 @@ public abstract class SqlToRelTestBase {
       return parent.getExpression(clazz);
     }
 
+    public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+      return parent.extend(extendedFields);
+    }
+
     public List<String> getQualifiedName() {
       return parent.getQualifiedName();
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/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 1cee06d..c735639 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -6302,6 +6302,15 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("SELECT DISTINCT 5, 10+5, 'string' from emp");
   }
 
+  @Test public void testTableExtend() {
+    checkResultType("select * from dept extend (x int not null)",
+        "RecordType(INTEGER NOT NULL DEPTNO, VARCHAR(10) NOT NULL NAME, INTEGER NOT NULL X) NOT NULL");
+    checkResultType("select deptno + x as z\n"
+        + "from dept extend (x int not null) as x\n"
+        + "where x > 10",
+        "RecordType(INTEGER NOT NULL Z) NOT NULL");
+  }
+
   @Test public void testExplicitTable() {
     final String empRecordType =
         "RecordType(INTEGER NOT NULL EMPNO,"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ecc5f4c2/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 adefd68..4958495 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2215,4 +2215,15 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[CASE($3, 1, 0)], EXPR$2=[$4], EXPR$3=[CASE($
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testTableExtend">
+        <Resource name="sql">
+            <![CDATA[select * from dept extend (x varchar(5) not null)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], NAME=[$1], X=[$2])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>


[2/2] incubator-calcite git commit: [CALCITE-478] Move CSV tutorial (Siva Narayanan)

Posted by jh...@apache.org.
[CALCITE-478] Move CSV tutorial (Siva Narayanan)

Close apache/incubator-calcite#21

Also, the following pull-requests were merged long ago:

Close apache/incubator-calcite#4
Close apache/incubator-calcite#19
Close apache/incubator-calcite#22


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

Branch: refs/heads/master
Commit: a640e58cf026cfc80aacc92ef393c3c9ea29d7dd
Parents: ecc5f4c
Author: Siva Narayanan <sn...@qubole.com>
Authored: Wed Dec 10 09:41:01 2014 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 10 09:41:01 2014 -0800

----------------------------------------------------------------------
 example/csv/README.md   |  92 ------
 example/csv/TUTORIAL.md | 731 -------------------------------------------
 2 files changed, 823 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a640e58c/example/csv/README.md
----------------------------------------------------------------------
diff --git a/example/csv/README.md b/example/csv/README.md
deleted file mode 100644
index 0570e3a..0000000
--- a/example/csv/README.md
+++ /dev/null
@@ -1,92 +0,0 @@
-Calcite-csv
-============
-
-Calcite adapter that reads
-<a href="http://en.wikipedia.org/wiki/Comma-separated_values">CSV</a> files.
-
-Calcite-csv is a nice simple example of how to connect
-<a href="https://github.com/apache/incubator-calcite">Apache Calcite</a>
-to your own data source and quickly get a full SQL/JDBC interface.
-
-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
-```
-
-Kick the tires
-==============
-
-Let's take a quick look at calcite-csv's (and Calcite's) features.
-We'll use <code>sqlline</code>, a SQL shell that connects to
-any JDBC data source and is included with calcite-csv.
-
-Connect to Calcite and try out some queries:
-
-```SQL
-$ ./sqlline
-!connect jdbc:calcite:model=target/test-classes/model.json admin admin
-
-VALUES char_length('Hello, ' || 'world!');
-
-!tables
-
-!describe emps
-
-SELECT * FROM emps;
-
-EXPLAIN PLAN FOR SELECT * FROM emps;
-
--- non-smart model does not optimize "fetch only required columns"
-EXPLAIN PLAN FOR SELECT empno, name FROM emps;
-```
-
-Connect to "smart" model and see how it uses just the required columns from the
-CSV.
-```SQL
-!connect jdbc:calcite:model=target/test-classes/smart.json admin admin
-EXPLAIN PLAN FOR SELECT * FROM emps;
-
--- Smart model optimizes "fetch only required columns"
-EXPLAIN PLAN FOR SELECT empno, name FROM emps;
-
-SELECT depts.name, count(*)
-  FROM emps JOIN depts USING (deptno)
- GROUP BY depts.name;
-
-EXPLAIN PLAN FOR SELECT empno, name FROM emps;
-
--- However it is not very smart, so it cannot see through calculations
--- This is solved in CALCITE-477
-EXPLAIN PLAN FOR SELECT empno*2, name FROM emps;
-```
-```
-!quit
-```
-
-(On Windows, the command is `sqlline.bat`.)
-
-As you can see, Calcite has a full SQL implementation that can efficiently
-query any data source.
-
-For a more leisurely walk through what Calcite can do and how it does it,
-try <a href="TUTORIAL.md">the Tutorial</a>.
-
-More information
-================
-
-* License: Apache License, Version 2.0.
-* Author: Julian Hyde
-* Blog: http://julianhyde.blogspot.com
-* Project page: http://calcite.incubator.apache.org/
-* Source code: https://git-wip-us.apache.org/repos/asf?p=incubator-calcite.git
-* Github mirror: https://github.com/apache/incubator-calcite
-* Developers list: http://mail-archives.apache.org/mod_mbox/incubator-calcite-dev/
-* <a href="TUTORIAL.md">Tutorial</a>
-* <a href="HISTORY.md">Release notes and history</a>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a640e58c/example/csv/TUTORIAL.md
----------------------------------------------------------------------
diff --git a/example/csv/TUTORIAL.md b/example/csv/TUTORIAL.md
deleted file mode 100644
index 51e63f6..0000000
--- a/example/csv/TUTORIAL.md
+++ /dev/null
@@ -1,731 +0,0 @@
-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.
-
-## First queries
-
-Without further ado, let's get started.
-
-If you haven't already installed, follow the instructions in the <a href="README.md">README</a>.
-It's just two commands: <code>git clone</code> followed <code>mvn install</code>.
-
-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: 'net.hydromatic.optiq.impl.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="src/main/java/net/hydromatic/optiq/impl/csv/CsvSchemaFactory.java">net.hydromatic.optiq.impl.csv.CsvSchemaFactory</a>, which is part of the
-optiq-csv project and implements the Calcite interface
-<a href="http://www.hydromatic.net/calcite/apidocs/net/hydromatic/optiq/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="src/main/java/net/hydromatic/optiq/impl/csv/CsvSchema.java">net.hydromatic.optiq.impl.csv.CsvSchema</a>
-and implements the Calcite interface <a
-href="http://www.hydromatic.net/calcite/apidocs/net/hydromatic/optiq/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/net/hydromatic/optiq/Table.html">Table</a>
-interface. <code>CsvSchema</code> produces tables that are instances of
-<a href="src/main/java/net/hydromatic/optiq/impl/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/net/hydromatic/optiq/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");
-
-    // Create different sub-types of table based on the "flavor" attribute.
-    final Table table;
-    switch (flavor) {
-    case SCANNABLE:
-      table = new CsvScannableTable(file, null);
-      break;
-    case FILTERABLE:
-      table = new CsvFilterableTable(file, null);
-      break;
-    case TRANSLATABLE:
-      table = new CsvTranslatableTable(file, null);
-      break;
-    default:
-      throw new AssertionError("Unknown flavor " + flavor);
-    }
-    builder.put(tableName, table);
-  }
-  return builder.build();
-}
-```
-
-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: 'net.hydromatic.optiq.impl.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: 'net.hydromatic.optiq.impl.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="src/main/java/net/hydromatic/optiq/impl/csv/CsvTableFactory.java">net.hydromatic.optiq.impl.csv.CsvTableFactory</a>,
-which implements the Calcite interface
-<a href="http://www.hydromatic.net/calcite/apidocs/net/hydromatic/optiq/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="src/main/java/net/hydromatic/optiq/impl/csv/CsvSmartTable.java">CsvSmartTable</a>
-rather than a <code>CsvScannableTable</code>.
-
-<code>CsvSmartTable</code> overrides the
-<code><a href="http://www.hydromatic.net/calcite/apidocs/net/hydromatic/optiq/TranslatableTable#toRel()">TranslatableTable.toRel()</a></code>
-method to create
-<a href="src/main/java/net/hydromatic/optiq/impl/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/net/hydromatic/optiq/impl/java/JavaRules.EnumerableTableAccessRel.html">EnumerableTableAccessRel</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 CsvPushProjectOntoTableRule extends RelOptRule {
-  public static final CsvPushProjectOntoTableRule INSTANCE =
-      new CsvPushProjectOntoTableRule();
-
-  private CsvPushProjectOntoTableRule() {
-    super(
-        operand(ProjectRel.class,
-            operand(CsvTableScan.class, none())),
-        "CsvPushProjectOntoTableRule");
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final ProjectRel 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/eigenbase/relopt/RelOptRuleCall.html#transformTo(org.eigenbase.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: 'net.hydromatic.optiq.impl.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: 'net.hydromatic.optiq.impl.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: 'net.hydromatic.optiq.impl.jdbc.JdbcSchema$Factory',
-      operand: {
-        jdbcDriver: 'com.mysql.jdbc.Driver',
-        jdbcUrl: 'jdbc:mysql://localhost/foodmart',
-        jdbcUser: 'foodmart',
-        jdbcPassword: 'foodmart'
-      }
-    },
-    {
-      name: 'FOODMART_CLONE',
-      type: 'custom',
-      factory: 'net.hydromatic.optiq.impl.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