You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by mm...@apache.org on 2017/09/05 14:36:44 UTC

[01/16] calcite git commit: [CALCITE-1943] Add back NavigationExpander and NavigationReplacer in SqlValidatorImpl (Dian Fu) [Forced Update!]

Repository: calcite
Updated Branches:
  refs/heads/branch-1.14 1e0472b7b -> 5410e79aa (forced update)


[CALCITE-1943] Add back NavigationExpander and NavigationReplacer in SqlValidatorImpl (Dian Fu)

It will replace clauses such as A.price with PREV(A.price, 0) and
makes the implementation of RexVisitor.visitPatternFieldRef(RexPatternFieldRef)
more unified.  Otherwise, it is difficult to implement this method. If
it returns the specified field, then the navigation such as
PREV(A.price, 1) becomes impossible; if not, then comparisons such as
A.price > PREV(A.price, 1) becomes meaningless.

Close apache/calcite#516


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

Branch: refs/heads/branch-1.14
Commit: d3a7c0d794cb9d0079fb37999dcd0dd6992f2041
Parents: a050ff5
Author: Dian Fu <fu...@alibaba-inc.com>
Authored: Tue Aug 15 17:19:00 2017 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:11:17 2017 -0700

----------------------------------------------------------------------
 .../calcite/sql/validate/SqlValidatorImpl.java  | 166 ++++++++-----
 .../rel/rel2sql/RelToSqlConverterTest.java      | 238 +++++++++----------
 .../calcite/test/SqlToRelConverterTest.java     |  21 ++
 .../calcite/test/SqlToRelConverterTest.xml      |  54 ++++-
 4 files changed, 285 insertions(+), 194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/d3a7c0d7/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 5509b9f..68106a9 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
@@ -31,6 +31,8 @@ import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelRecordType;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Feature;
@@ -4908,7 +4910,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       aliases.add(alias);
 
       SqlNode expand = expand(measure, scope);
-      validateMeasures(expand, allRows);
+      expand = navigationInMeasure(expand, allRows);
       setOriginal(expand, measure);
 
       inferUnknownTypes(unknownType, scope, expand);
@@ -4933,9 +4935,22 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return fields;
   }
 
-  private void validateMeasures(SqlNode node, boolean allRows) {
+  private SqlNode navigationInMeasure(SqlNode node, boolean allRows) {
     final Set<String> prefix = node.accept(new PatternValidator(true));
     Util.discard(prefix);
+    final List<SqlNode> ops = ((SqlCall) node).getOperandList();
+
+    final SqlOperator defaultOp =
+        allRows ? SqlStdOperatorTable.RUNNING : SqlStdOperatorTable.FINAL;
+    final SqlNode op0 = ops.get(0);
+    if (!isRunningOrFinal(op0.getKind())
+        || !allRows && op0.getKind() == SqlKind.RUNNING) {
+      SqlNode newNode = defaultOp.createCall(SqlParserPos.ZERO, op0);
+      node = SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, newNode, ops.get(1));
+    }
+
+    node = new NavigationExpander().go(node);
+    return node;
   }
 
   private void validateDefinitions(SqlMatchRecognize mr,
@@ -4956,7 +4971,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     for (SqlNode item : mr.getPatternDefList().getList()) {
       final String alias = alias(item);
       SqlNode expand = expand(item, scope);
-      validateDefine(expand, alias);
+      expand = navigationInDefine(expand, alias);
       setOriginal(expand, item);
 
       inferUnknownTypes(booleanType, scope, expand);
@@ -4992,10 +5007,15 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return identifier.getSimple();
   }
 
-  /** Checks that all pattern variables within a function are the same. */
-  private void validateDefine(SqlNode node, String alpha) {
+  /** Checks that all pattern variables within a function are the same,
+   * and canonizes expressions such as {@code PREV(B.price)} to
+   * {@code LAST(B.price, 0)}. */
+  private SqlNode navigationInDefine(SqlNode node, String alpha) {
     Set<String> prefix = node.accept(new PatternValidator(false));
     Util.discard(prefix);
+    node = new NavigationExpander().go(node);
+    node = new NavigationReplacer(alpha).go(node);
+    return node;
   }
 
   public void validateAggregateParams(SqlCall aggCall, SqlNode filter,
@@ -5764,44 +5784,35 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * Modify the nodes in navigation function
    * such as FIRST, LAST, PREV AND NEXT.
    */
-  private class NavigationModifier extends SqlBasicVisitor<SqlNode> {
-    @Override public SqlNode visit(SqlLiteral literal) {
-      return literal;
-    }
-
-    @Override public SqlNode visit(SqlIntervalQualifier intervalQualifier) {
-      return intervalQualifier;
-    }
-
-    @Override public SqlNode visit(SqlDataTypeSpec type) {
-      return type;
-    }
-
-    @Override public SqlNode visit(SqlDynamicParam param) {
-      return param;
-    }
-
+  private static class NavigationModifier extends SqlShuttle {
     public SqlNode go(SqlNode node) {
       return node.accept(this);
     }
   }
 
   /**
-   * Expand navigation expression :
-   * eg: PREV(A.price + A.amount) to PREV(A.price) + PREV(A.amount)
-   * eg: FIRST(A.price * 2) to FIST(A.PRICE) * 2
+   * Shuttle that expands navigation expressions in a MATCH_RECOGNIZE clause.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   *   <li>{@code PREV(A.price + A.amount)} &rarr;
+   *   {@code PREV(A.price) + PREV(A.amount)}
+   *
+   *   <li>{@code FIRST(A.price * 2)} &rarr; {@code FIRST(A.PRICE) * 2}
+   * </ul>
    */
-  private class NavigationExpander extends NavigationModifier {
-    SqlOperator currentOperator;
-    SqlNode currentOffset;
+  private static class NavigationExpander extends NavigationModifier {
+    final SqlOperator op;
+    final SqlNode offset;
 
     NavigationExpander() {
-
+      this(null, null);
     }
 
     NavigationExpander(SqlOperator operator, SqlNode offset) {
-      this.currentOffset = offset;
-      this.currentOperator = operator;
+      this.offset = offset;
+      this.op = operator;
     }
 
     @Override public SqlNode visit(SqlCall call) {
@@ -5826,33 +5837,59 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               innerOperands.get(0), offset);
           }
         }
-        return inner.accept(new NavigationExpander(call.getOperator(), offset));
+        SqlNode newInnerNode =
+            inner.accept(new NavigationExpander(call.getOperator(), offset));
+        if (op != null) {
+          newInnerNode = op.createCall(SqlParserPos.ZERO, newInnerNode,
+              this.offset);
+        }
+        return newInnerNode;
       }
 
-      for (SqlNode node : operands) {
-        SqlNode newNode = node.accept(new NavigationExpander());
-        if (currentOperator != null) {
-          newNode = currentOperator.createCall(SqlParserPos.ZERO, newNode, currentOffset);
+      if (operands.size() > 0) {
+        for (SqlNode node : operands) {
+          if (node != null) {
+            SqlNode newNode = node.accept(new NavigationExpander());
+            if (op != null) {
+              newNode = op.createCall(SqlParserPos.ZERO, newNode, offset);
+            }
+            newOperands.add(newNode);
+          } else {
+            newOperands.add(null);
+          }
+        }
+        return call.getOperator().createCall(SqlParserPos.ZERO, newOperands);
+      } else {
+        if (op == null) {
+          return call;
+        } else {
+          return op.createCall(SqlParserPos.ZERO, call, offset);
         }
-        newOperands.add(newNode);
       }
-      return call.getOperator().createCall(SqlParserPos.ZERO, newOperands);
     }
 
     @Override public SqlNode visit(SqlIdentifier id) {
-      if (currentOperator == null) {
+      if (op == null) {
         return id;
       } else {
-        return currentOperator.createCall(SqlParserPos.ZERO, id, currentOffset);
+        return op.createCall(SqlParserPos.ZERO, id, offset);
       }
     }
   }
 
   /**
-   * Replace {@code A as A.price > PREV(B.price)}
-   * with {@code PREV(A.price, 0) > last(B.price, 0)}.
+   * Shuttle that replaces {@code A as A.price > PREV(B.price)} with
+   * {@code PREV(A.price, 0) > LAST(B.price, 0)}.
+   *
+   * <p>Replacing {@code A.price} with {@code PREV(A.price, 0)} makes the
+   * implementation of
+   * {@link RexVisitor#visitPatternFieldRef(RexPatternFieldRef)} more unified.
+   * Otherwise, it's difficult to implement this method. If it returns the
+   * specified field, then the navigation such as {@code PREV(A.price, 1)}
+   * becomes impossible; if not, then comparisons such as
+   * {@code A.price > PREV(A.price, 1)} become meaningless.
    */
-  private class NavigationReplacer extends NavigationModifier {
+  private static class NavigationReplacer extends NavigationModifier {
     private final String alpha;
 
     NavigationReplacer(String alpha) {
@@ -5867,19 +5904,16 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         return call;
       }
 
-      List<SqlNode> operands = call.getOperandList();
       switch (kind) {
       case PREV:
-        String name = ((SqlIdentifier) operands.get(0)).names.get(0);
-        return name.equals(alpha) ? call
-          : SqlStdOperatorTable.LAST.createCall(SqlParserPos.ZERO, operands);
-      default:
-        List<SqlNode> newOperands = new ArrayList<>();
-        for (SqlNode op : operands) {
-          newOperands.add(op.accept(this));
+        final List<SqlNode> operands = call.getOperandList();
+        if (operands.get(0) instanceof SqlIdentifier) {
+          String name = ((SqlIdentifier) operands.get(0)).names.get(0);
+          return name.equals(alpha) ? call
+              : SqlStdOperatorTable.LAST.createCall(SqlParserPos.ZERO, operands);
         }
-        return call.getOperator().createCall(SqlParserPos.ZERO, newOperands);
       }
+      return super.visit(call);
     }
 
     @Override public SqlNode visit(SqlIdentifier id) {
@@ -5959,10 +5993,12 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
 
       for (SqlNode node : operands) {
-        vars.addAll(
-            node.accept(
-                new PatternValidator(isMeasure, firstLastCount, prevNextCount,
-                    aggregateCount)));
+        if (node != null) {
+          vars.addAll(
+              node.accept(
+                  new PatternValidator(isMeasure, firstLastCount, prevNextCount,
+                      aggregateCount)));
+        }
       }
 
       if (isSingle) {
@@ -5974,13 +6010,17 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           }
           break;
         default:
-          if (vars.isEmpty()) {
-            throw newValidationError(call,
-              Static.RESOURCE.patternFunctionNullCheck(call.toString()));
-          }
-          if (vars.size() != 1) {
-            throw newValidationError(call,
-                Static.RESOURCE.patternFunctionVariableCheck(call.toString()));
+          if (operands.size() == 0
+              || !(operands.get(0) instanceof SqlCall)
+              || ((SqlCall) operands.get(0)).getOperator() != SqlStdOperatorTable.CLASSIFIER) {
+            if (vars.isEmpty()) {
+              throw newValidationError(call,
+                  Static.RESOURCE.patternFunctionNullCheck(call.toString()));
+            }
+            if (vars.size() != 1) {
+              throw newValidationError(call,
+                  Static.RESOURCE.patternFunctionVariableCheck(call.toString()));
+            }
           }
           break;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d3a7c0d7/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index d0749b9..b72edfc 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -993,9 +993,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1016,9 +1016,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" + $)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1039,9 +1039,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (^ \"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1062,9 +1062,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (^ \"STRT\" \"DOWN\" + \"UP\" + $)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1085,9 +1085,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" * \"UP\" ?)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1108,9 +1108,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" {- \"DOWN\" -} \"UP\" ?)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1131,9 +1131,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" { 2 } \"UP\" { 3, })\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1154,9 +1154,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" { , 2 } \"UP\" { 3, 5 })\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1177,9 +1177,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" {- \"DOWN\" + -} {- \"UP\" * -})\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1203,9 +1203,9 @@ public class RelToSqlConverterTest {
         + "(\"A\" \"B\" \"C\" | \"A\" \"C\" \"B\" | \"B\" \"A\" \"C\" "
         + "| \"B\" \"C\" \"A\" | \"C\" \"A\" \"B\" | \"C\" \"B\" \"A\")\n"
         + "DEFINE "
-        + "\"A\" AS \"A\".\"net_weight\" < PREV(\"A\".\"net_weight\", 1), "
-        + "\"B\" AS \"B\".\"net_weight\" > PREV(\"B\".\"net_weight\", 1), "
-        + "\"C\" AS \"C\".\"net_weight\" < PREV(\"C\".\"net_weight\", 1))";
+        + "\"A\" AS PREV(\"A\".\"net_weight\", 0) < PREV(\"A\".\"net_weight\", 1), "
+        + "\"B\" AS PREV(\"B\".\"net_weight\", 0) > PREV(\"B\".\"net_weight\", 1), "
+        + "\"C\" AS PREV(\"C\".\"net_weight\", 0) < PREV(\"C\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
 
@@ -1225,9 +1225,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1248,9 +1248,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))\n"
         + "ORDER BY \"net_weight\"";
     sql(sql).ok(expected);
@@ -1289,9 +1289,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))\n"
         + "ORDER BY \"net_weight\"";
     sql(sql).ok(expected);
@@ -1313,10 +1313,10 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1336,9 +1336,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "FIRST(\"DOWN\".\"net_weight\", 0), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "LAST(\"UP\".\"net_weight\", 0))";
     sql(sql).ok(expected);
   }
@@ -1359,10 +1359,10 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "LAST(\"UP\".\"net_weight\" + \"UP\".\"gross_weight\", 0))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "LAST(\"UP\".\"net_weight\", 0) + LAST(\"UP\".\"gross_weight\", 0))";
     sql(sql).ok(expected);
   }
 
@@ -1383,11 +1383,11 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "PREV(LAST(\"UP\".\"net_weight\" + "
-        + "\"UP\".\"gross_weight\", 0), 3))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "PREV(LAST(\"UP\".\"net_weight\", 0) + "
+        + "LAST(\"UP\".\"gross_weight\", 0), 3))";
     sql(sql).ok(expected);
   }
 
@@ -1411,18 +1411,18 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "MATCH_NUMBER () AS \"MATCH_NUM\", "
-        + "CLASSIFIER() AS \"VAR_MATCH\", "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "LAST(\"UP\".\"net_weight\", 0) AS \"END_NW\"\n"
+        + "FINAL MATCH_NUMBER () AS \"MATCH_NUM\", "
+        + "FINAL CLASSIFIER() AS \"VAR_MATCH\", "
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
+        + "FINAL LAST(\"UP\".\"net_weight\", 0) AS \"END_NW\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1445,16 +1445,16 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
         + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "LAST(\"UP\".\"net_weight\", 0) AS \"END_NW\"\n"
+        + "FINAL LAST(\"UP\".\"net_weight\", 0) AS \"END_NW\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1477,16 +1477,16 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "RUNNING LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "LAST(\"UP\".\"net_weight\", 0) AS \"END_NW\"\n"
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL (RUNNING LAST(\"DOWN\".\"net_weight\", 0)) AS \"BOTTOM_NW\", "
+        + "FINAL LAST(\"UP\".\"net_weight\", 0) AS \"END_NW\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1509,17 +1509,17 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
         + "FINAL COUNT(\"UP\".\"net_weight\") AS \"UP_CNT\", "
         + "FINAL COUNT(\"*\".\"net_weight\") AS \"DOWN_CNT\", "
-        + "RUNNING COUNT(\"*\".\"net_weight\") AS \"RUNNING_CNT\"\n"
+        + "FINAL (RUNNING COUNT(\"*\".\"net_weight\")) AS \"RUNNING_CNT\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1543,17 +1543,17 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "FIRST(\"STRT\".\"net_weight\", 0) AS \"START_NW\", "
-        + "LAST(\"UP\".\"net_weight\", 0) AS \"UP_CNT\", "
-        + "SUM(\"DOWN\".\"net_weight\") / "
-        + "COUNT(\"DOWN\".\"net_weight\") AS \"DOWN_CNT\"\n"
+        + "FINAL FIRST(\"STRT\".\"net_weight\", 0) AS \"START_NW\", "
+        + "FINAL LAST(\"UP\".\"net_weight\", 0) AS \"UP_CNT\", "
+        + "FINAL (SUM(\"DOWN\".\"net_weight\") / "
+        + "COUNT(\"DOWN\".\"net_weight\")) AS \"DOWN_CNT\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1576,17 +1576,17 @@ public class RelToSqlConverterTest {
         + "FROM (SELECT *\n"
         + "FROM \"foodmart\".\"product\") MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "FIRST(\"STRT\".\"net_weight\", 0) AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"UP_CNT\", "
+        + "FINAL FIRST(\"STRT\".\"net_weight\", 0) AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"UP_CNT\", "
         + "FINAL SUM(\"DOWN\".\"net_weight\") AS \"DOWN_CNT\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN "
         + "(\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1609,17 +1609,17 @@ public class RelToSqlConverterTest {
         + "FROM (SELECT *\n"
         + "FROM \"foodmart\".\"product\") MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "FIRST(\"STRT\".\"net_weight\", 0) AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"UP_CNT\", "
+        + "FINAL FIRST(\"STRT\".\"net_weight\", 0) AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"UP_CNT\", "
         + "FINAL SUM(\"DOWN\".\"net_weight\") AS \"DOWN_CNT\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN "
         + "(\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))\n"
         + "ORDER BY \"START_NW\", \"UP_CNT\"";
     sql(sql).ok(expected);
@@ -1642,10 +1642,10 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1666,10 +1666,10 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP PAST LAST ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1689,10 +1689,10 @@ public class RelToSqlConverterTest {
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO FIRST \"DOWN\"\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
-        + "DEFINE \"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "DEFINE \"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1713,10 +1713,10 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO LAST \"DOWN\"\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1737,10 +1737,10 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO LAST \"DOWN\"\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1763,10 +1763,10 @@ public class RelToSqlConverterTest {
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "SUBSET \"STDN\" = (\"DOWN\", \"STRT\")\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
-        + "NEXT(\"UP\".\"net_weight\", 1))";
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
+        + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))";
     sql(sql).ok(expected);
   }
 
@@ -1789,18 +1789,18 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "SUM(\"STDN\".\"net_weight\") / "
-        + "COUNT(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
+        + "FINAL (SUM(\"STDN\".\"net_weight\") / "
+        + "COUNT(\"STDN\".\"net_weight\")) AS \"AVG_STDN\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "SUBSET \"STDN\" = (\"DOWN\", \"STRT\")\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1824,17 +1824,17 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
+        + "FINAL SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "SUBSET \"STDN\" = (\"DOWN\", \"STRT\")\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1858,17 +1858,17 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
+        + "FINAL SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "SUBSET \"STDN\" = (\"DOWN\", \"STRT\"), \"STDN2\" = (\"DOWN\", \"STRT\")\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1893,17 +1893,17 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
+        + "FINAL \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "FINAL LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
+        + "FINAL SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
         + "ONE ROW PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "SUBSET \"STDN\" = (\"DOWN\", \"STRT\"), \"STDN2\" = (\"DOWN\", \"STRT\")\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1928,17 +1928,17 @@ public class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\") "
         + "MATCH_RECOGNIZE(\n"
         + "MEASURES "
-        + "\"STRT\".\"net_weight\" AS \"START_NW\", "
-        + "LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
-        + "SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
+        + "RUNNING \"STRT\".\"net_weight\" AS \"START_NW\", "
+        + "RUNNING LAST(\"DOWN\".\"net_weight\", 0) AS \"BOTTOM_NW\", "
+        + "RUNNING SUM(\"STDN\".\"net_weight\") AS \"AVG_STDN\"\n"
         + "ALL ROWS PER MATCH\n"
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +)\n"
         + "SUBSET \"STDN\" = (\"DOWN\", \"STRT\"), \"STDN2\" = (\"DOWN\", \"STRT\")\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"net_weight\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"net_weight\", 0) < "
         + "PREV(\"DOWN\".\"net_weight\", 1), "
-        + "\"UP\" AS \"UP\".\"net_weight\" > "
+        + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > "
         + "PREV(\"UP\".\"net_weight\", 1))";
     sql(sql).ok(expected);
   }
@@ -1964,9 +1964,9 @@ public class RelToSqlConverterTest {
         + "AFTER MATCH SKIP TO NEXT ROW\n"
         + "PATTERN (\"STRT\" \"DOWN\" + \"UP\" +) WITHIN INTERVAL '3:12:22.123' HOUR TO SECOND\n"
         + "DEFINE "
-        + "\"DOWN\" AS \"DOWN\".\"salary\" < "
+        + "\"DOWN\" AS PREV(\"DOWN\".\"salary\", 0) < "
         + "PREV(\"DOWN\".\"salary\", 1), "
-        + "\"UP\" AS \"UP\".\"salary\" > "
+        + "\"UP\" AS PREV(\"UP\".\"salary\", 0) > "
         + "PREV(\"UP\".\"salary\", 1))";
     sql(sql).ok(expected);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d3a7c0d7/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 75dfe77..ca2623c 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -2653,6 +2653,27 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  @Test public void testPrevClassifier() {
+    final String sql = "SELECT *\n"
+        + "FROM emp\n"
+        + "MATCH_RECOGNIZE (\n"
+        + "  MEASURES\n"
+        + "    STRT.mgr AS start_mgr,\n"
+        + "    LAST(DOWN.mgr) AS up_days,\n"
+        + "    LAST(UP.mgr) AS total_days\n"
+        + "  PATTERN (STRT DOWN? UP+)\n"
+        + "  DEFINE\n"
+        + "    DOWN AS DOWN.mgr < PREV(DOWN.mgr),\n"
+        + "    UP AS CASE\n"
+        + "            WHEN PREV(CLASSIFIER()) = 'STRT'\n"
+        + "              THEN UP.mgr > 15\n"
+        + "            ELSE\n"
+        + "              UP.mgr > 20\n"
+        + "            END\n"
+        + ") AS T";
+    sql(sql).ok();
+  }
+
   /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/d3a7c0d7/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 c5ba797..f96583d 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2390,7 +2390,7 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[CASE($3, 1, 0)], EXPR$2=[$4], EXPR$3=[CASE($
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalMatch(partition=[[$2, $5]], order=[[2, 5 DESC, 0]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[$2, $5]], order=[[2, 5 DESC, 0]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2402,7 +2402,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
   (
    partition by job, sal
    order by job asc, sal desc
-   measures  MATCH_NUMBER() as match_num,    CLASSIFIER() as var_match,    STRT.mgr as start_nw,   LAST(DOWN.mgr) as bottom_nw,   LAST(up.mgr) as end_nw    pattern (strt down+ up+)
+   measures STRT.mgr as start_nw,   LAST(DOWN.mgr) as bottom_nw,   LAST(up.mgr) as end_nw    pattern (strt down+ up+)
     define
       down as down.mgr < PREV(down.mgr),
       up as up.mgr > prev(up.mgr)
@@ -2411,7 +2411,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="plan">
             <![CDATA[
 LogicalProject(JOB=[$0], SAL=[$1], MATCH_NUM=[$2], VAR_MATCH=[$3], START_NW=[$4], BOTTOM_NW=[$5], END_NW=[$6])
-  LogicalMatch(partition=[[$2, $5]], order=[[2, 5 DESC]], outputFields=[[JOB, SAL, MATCH_NUM, VAR_MATCH, START_NW, BOTTOM_NW, END_NW]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[$2, $5]], order=[[2, 5 DESC]], outputFields=[[JOB, SAL, MATCH_NUM, VAR_MATCH, START_NW, BOTTOM_NW, END_NW]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2423,7 +2423,7 @@ LogicalProject(JOB=[$0], SAL=[$1], MATCH_NUM=[$2], VAR_MATCH=[$3], START_NW=[$4]
   (
    partition by job
    order by sal
-   measures  MATCH_NUMBER() as match_num,    CLASSIFIER() as var_match,    STRT.mgr as start_nw,   LAST(DOWN.mgr) as bottom_nw,   LAST(up.mgr) as end_nw    pattern (strt down+ up+)
+   measures STRT.mgr as start_nw,   LAST(DOWN.mgr) as bottom_nw,   LAST(up.mgr) as end_nw    pattern (strt down+ up+)
     define
       down as down.mgr < PREV(down.mgr),
       up as up.mgr > prev(up.mgr)
@@ -2432,7 +2432,7 @@ LogicalProject(JOB=[$0], SAL=[$1], MATCH_NUM=[$2], VAR_MATCH=[$3], START_NW=[$4]
         <Resource name="plan">
             <![CDATA[
 LogicalProject(JOB=[$0], MATCH_NUM=[$1], VAR_MATCH=[$2], START_NW=[$3], BOTTOM_NW=[$4], END_NW=[$5])
-  LogicalMatch(partition=[[$2]], order=[[5]], outputFields=[[JOB, MATCH_NUM, VAR_MATCH, START_NW, BOTTOM_NW, END_NW]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[$2]], order=[[5]], outputFields=[[JOB, MATCH_NUM, VAR_MATCH, START_NW, BOTTOM_NW, END_NW]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2444,7 +2444,9 @@ LogicalProject(JOB=[$0], MATCH_NUM=[$1], VAR_MATCH=[$2], START_NW=[$3], BOTTOM_N
   (
    partition by job
    order by sal
-   measures  MATCH_NUMBER() as match_num,    CLASSIFIER() as var_match,    STRT.mgr as start_nw,   LAST(DOWN.mgr) as bottom_nw,   LAST(up.mgr) as end_nw   ALL ROWS PER MATCH    pattern (strt down+ up+)
+   measures STRT.mgr as start_nw,   LAST(DOWN.mgr) as bottom_nw,   LAST(up.mgr) as end_nw
+   ALL ROWS PER MATCH
+   pattern (strt down+ up+)
     define
       down as down.mgr < PREV(down.mgr),
       up as up.mgr > prev(up.mgr)
@@ -2453,7 +2455,7 @@ LogicalProject(JOB=[$0], MATCH_NUM=[$1], VAR_MATCH=[$2], START_NW=[$3], BOTTOM_N
         <Resource name="plan">
             <![CDATA[
 LogicalProject(JOB=[$0], SAL=[$1], EMPNO=[$2], ENAME=[$3], MGR=[$4], HIREDATE=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], MATCH_NUM=[$9], VAR_MATCH=[$10], START_NW=[$11], BOTTOM_NW=[$12], END_NW=[$13])
-  LogicalMatch(partition=[[$2]], order=[[5]], outputFields=[[JOB, SAL, EMPNO, ENAME, MGR, HIREDATE, COMM, DEPTNO, SLACKER, MATCH_NUM, VAR_MATCH, START_NW, BOTTOM_NW, END_NW]], allRows=[true], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[$2]], order=[[5]], outputFields=[[JOB, SAL, EMPNO, ENAME, MGR, HIREDATE, COMM, DEPTNO, SLACKER, MATCH_NUM, VAR_MATCH, START_NW, BOTTOM_NW, END_NW]], allRows=[true], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), PREV(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2473,7 +2475,7 @@ LogicalProject(JOB=[$0], SAL=[$1], EMPNO=[$2], ENAME=[$3], MGR=[$4], HIREDATE=[$
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalMatch(partition=[[]], order=[[]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, NEXT(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[]], order=[[]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), NEXT(PREV(UP.$3, 0), 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2481,7 +2483,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     <TestCase name="testMatchRecognizePrevDown">
         <Resource name="sql">
             <![CDATA[SELECT *
-FROM emp
+FROM tmp
 MATCH_RECOGNIZE (
   MEASURES
     STRT.mgr AS start_mgr,
@@ -2496,7 +2498,7 @@ MATCH_RECOGNIZE (
         <Resource name="plan">
             <![CDATA[
 LogicalProject(START_MGR=[$0], UP_DAYS=[$1], TOTAL_DAYS=[$2])
-  LogicalMatch(partition=[[]], order=[[]], outputFields=[[START_MGR, UP_DAYS, TOTAL_DAYS]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, PREV(DOWN.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[]], order=[[]], outputFields=[[START_MGR, UP_DAYS, TOTAL_DAYS]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), LAST(DOWN.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2520,7 +2522,7 @@ MATCH_RECOGNIZE (
         <Resource name="plan">
             <![CDATA[
 LogicalProject(START_MGR=[$0], BOTTOM_MGR=[$1], END_MGR=[$2])
-  LogicalMatch(partition=[[]], order=[[]], outputFields=[[START_MGR, BOTTOM_MGR, END_MGR]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, PREV(LAST(DOWN.$3, 1), 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[]], order=[[]], outputFields=[[START_MGR, BOTTOM_MGR, END_MGR]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), PREV(LAST(DOWN.$3, 1), 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -2541,7 +2543,35 @@ LogicalProject(START_MGR=[$0], BOTTOM_MGR=[$1], END_MGR=[$2])
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalMatch(partition=[[]], order=[[]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[SKIP TO LAST('DOWN')], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[[DOWN, STRT]]], patternDefinitions=[[<(DOWN.$3, PREV(DOWN.$3, 1)), >(UP.$3, NEXT(UP.$3, 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+  LogicalMatch(partition=[[]], order=[[]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[SKIP TO LAST('DOWN')], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[[DOWN, STRT]]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), NEXT(PREV(UP.$3, 0), 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testPrevClassifier">
+        <Resource name="sql">
+            <![CDATA[SELECT *
+FROM tmp
+MATCH_RECOGNIZE (
+  MEASURES
+    STRT.mgr AS start_mgr,
+    LAST(DOWN.mgr) AS up_days,
+    LAST(UP.mgr) AS total_days
+  PATTERN (STRT DOWN? UP+)
+  DEFINE
+    DOWN AS DOWN.mgr < PREV(DOWN.mgr),
+    UP AS CASE
+            WHEN PREV(CLASSIFIER()) = 'STRT'
+              THEN UP.mgr > 15
+            ELSE
+              UP.mgr > 20
+            END
+) AS T]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(START_MGR=[$0], UP_DAYS=[$1], TOTAL_DAYS=[$2])
+  LogicalMatch(partition=[[]], order=[[]], outputFields=[[START_MGR, UP_DAYS, TOTAL_DAYS]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 0, 1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), CASE(=(PREV(CLASSIFIER(), 1), 'STRT'), >(PREV(UP.$3, 0), 15), >(PREV(UP.$3, 0), 20))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>


[02/16] calcite git commit: [CALCITE-1953] Rewrite "NOT (x IS FALSE)" to "x IS NOT FALSE"; "x IS TRUE" would be wrong

Posted by mm...@apache.org.
[CALCITE-1953] Rewrite "NOT (x IS FALSE)" to "x IS NOT FALSE"; "x IS TRUE" would be wrong

SqlKind.negate() != SqlKind.negateNullSafe()

Close apache/calcite#521


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

Branch: refs/heads/branch-1.14
Commit: 2156d8265178f92c52f94ef47588a89ab1fdb7de
Parents: d3a7c0d
Author: MinJi Kim <mi...@apache.org>
Authored: Sat Oct 8 10:30:58 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:12:28 2017 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/rex/RexSimplify.java     |  2 +-
 .../java/org/apache/calcite/sql/SqlKind.java    | 43 +++++++++++++++++---
 .../apache/calcite/test/RelOptRulesTest.java    | 15 +++++++
 .../org/apache/calcite/test/RexProgramTest.java | 27 ++++++++++++
 .../org/apache/calcite/test/RelOptRulesTest.xml | 25 ++++++++++++
 .../calcite/test/SqlToRelConverterTest.xml      |  2 +-
 6 files changed, 106 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/2156d826/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index 34f01a0..67ac1b8 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -325,7 +325,7 @@ public class RexSimplify {
       // Note that
       //   (NOT x) IS TRUE !=> x IS FALSE
       // because of null values.
-      final SqlOperator notKind = RexUtil.op(kind.negate());
+      final SqlOperator notKind = RexUtil.op(kind.negateNullSafe());
       final RexNode arg = ((RexCall) a).operands.get(0);
       return simplify(rexBuilder.makeCall(notKind, arg));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/2156d826/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 62dedb6..ad7c4e2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1170,19 +1170,31 @@ public enum SqlKind {
 
   /** Returns the kind that you get if you apply NOT to this kind.
    *
-   * <p>For example, {@code IS_NOT_NULL.negate()} returns {@link #IS_NULL}. */
+   * <p>For example, {@code IS_NOT_NULL.negate()} returns {@link #IS_NULL}.
+   *
+   * <p>For {@link #IS_TRUE}, {@link #IS_FALSE}, {@link #IS_NOT_TRUE},
+   * {@link #IS_NOT_FALSE}, nullable inputs need to be treated carefully.
+   *
+   * <p>{@code NOT(IS_TRUE(null))} = {@code NOT(false)} = {@code true},
+   * while {@code IS_FALSE(null)} = {@code false},
+   * so {@code NOT(IS_TRUE(X))} should be {@code IS_NOT_TRUE(X)}.
+   * On the other hand,
+   * {@code IS_TRUE(NOT(null))} = {@code IS_TRUE(null)} = {@code false}.
+   *
+   * <p>This is why negate() != negateNullSafe() for these operators.
+   */
   public SqlKind negate() {
     switch (this) {
     case IS_TRUE:
-      return IS_FALSE;
+      return IS_NOT_TRUE;
     case IS_FALSE:
-      return IS_TRUE;
+      return IS_NOT_FALSE;
     case IS_NULL:
       return IS_NOT_NULL;
     case IS_NOT_TRUE:
-      return IS_NOT_FALSE;
+      return IS_TRUE;
     case IS_NOT_FALSE:
-      return IS_NOT_TRUE;
+      return IS_FALSE;
     case IS_NOT_NULL:
       return IS_NULL;
     case IS_DISTINCT_FROM:
@@ -1195,7 +1207,18 @@ public enum SqlKind {
   }
 
   /** Returns the kind that you get if you negate this kind.
-   * To conform to null semantics, null value should not be compared. */
+   * To conform to null semantics, null value should not be compared.
+   *
+   * <p>For {@link #IS_TRUE}, {@link #IS_FALSE}, {@link #IS_NOT_TRUE} and
+   * {@link #IS_NOT_FALSE}, nullable inputs need to be treated carefully:
+   *
+   * <ul>
+   * <li>NOT(IS_TRUE(null)) = NOT(false) = true
+   * <li>IS_TRUE(NOT(null)) = IS_TRUE(null) = false
+   * <li>IS_FALSE(null) = false
+   * <li>IS_NOT_TRUE(null) = true
+   * </ul>
+   */
   public SqlKind negateNullSafe() {
     switch (this) {
     case EQUALS:
@@ -1210,6 +1233,14 @@ public enum SqlKind {
       return GREATER_THAN;
     case GREATER_THAN_OR_EQUAL:
       return LESS_THAN;
+    case IS_TRUE:
+      return IS_FALSE;
+    case IS_FALSE:
+      return IS_TRUE;
+    case IS_NOT_TRUE:
+      return IS_NOT_FALSE;
+    case IS_NOT_FALSE:
+      return IS_NOT_TRUE;
     default:
       return this.negate();
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/2156d826/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 f43f624..d9aea0d 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -161,6 +161,21 @@ public class RelOptRulesTest extends RelOptTestBase {
     return DiffRepository.lookup(RelOptRulesTest.class);
   }
 
+  @Test public void testReduceNot() {
+    HepProgram preProgram = new HepProgramBuilder()
+        .build();
+
+    HepProgramBuilder builder = new HepProgramBuilder();
+    builder.addRuleClass(ReduceExpressionsRule.class);
+    HepPlanner hepPlanner = new HepPlanner(builder.build());
+    hepPlanner.addRule(ReduceExpressionsRule.FILTER_INSTANCE);
+
+    final String sql = "select *\n"
+        + "from (select (case when sal > 1000 then null else false end) as caseCol from emp)\n"
+        + "where NOT(caseCol)";
+    checkPlanning(tester, preProgram, hepPlanner, sql, true);
+  }
+
   @Test public void testReduceNestedCaseWhen() {
     HepProgram preProgram = new HepProgramBuilder()
         .build();

http://git-wip-us.apache.org/repos/asf/calcite/blob/2156d826/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index ca9e04f..7cbcfa6 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -1640,6 +1640,33 @@ public class RexProgramTest {
     }
     return map2.toString();
   }
+
+  @Test public void testSimplifyNot() {
+    final RelDataType booleanNullableType =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.BOOLEAN), true);
+    final RexNode booleanInput = rexBuilder.makeInputRef(booleanNullableType, 0);
+    final RexNode isFalse = rexBuilder.makeCall(SqlStdOperatorTable.IS_FALSE, booleanInput);
+    final RexCall result = (RexCall) simplify(isFalse);
+    assertThat(result.getType().isNullable(), is(false));
+    assertThat(result.getOperator(), is((SqlOperator) SqlStdOperatorTable.IS_FALSE));
+    assertThat(result.getOperands().size(), is(1));
+    assertThat(result.getOperands().get(0), is(booleanInput));
+
+    // Make sure that IS_FALSE(IS_FALSE(nullable boolean)) != IS_TRUE(nullable boolean)
+    // IS_FALSE(IS_FALSE(null)) = IS_FALSE(false) = true
+    // IS_TRUE(null) = false
+    final RexNode isFalseIsFalse = rexBuilder.makeCall(SqlStdOperatorTable.IS_FALSE, isFalse);
+    final RexCall result2 = (RexCall) simplify(isFalseIsFalse);
+    assertThat(result2.getType().isNullable(), is(false));
+    assertThat(result2.getOperator(), is((SqlOperator) SqlStdOperatorTable.IS_NOT_FALSE));
+    assertThat(result2.getOperands().size(), is(1));
+    assertThat(result2.getOperands().get(0), is(booleanInput));
+  }
+
+  private RexNode simplify(RexNode e) {
+    return new RexSimplify(rexBuilder, false, RexUtil.EXECUTOR).simplify(e);
+  }
 }
 
 // End RexProgramTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/2156d826/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index 28c3b9d..2da30be 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -16,6 +16,31 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 <Root>
+    <TestCase name="testReduceNot">
+        <Resource name="sql">
+            <![CDATA[select sal
+from emp
+where case when (sal = 1000) then
+(case when sal = 1000 then null else 1 end is null) else
+(case when sal = 2000 then null else 1 end is null) end is true]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(CASECOL=[$0])
+  LogicalFilter(condition=[NOT($0)])
+    LogicalProject(CASECOL=[CASE(>($5, 1000), null, false)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(CASECOL=[$0])
+  LogicalFilter(condition=[NOT($0)])
+    LogicalProject(CASECOL=[CASE(>($5, 1000), null, false)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testReduceNestedCaseWhen">
         <Resource name="sql">
             <![CDATA[select sal

http://git-wip-us.apache.org/repos/asf/calcite/blob/2156d826/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 f96583d..2e8f904 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2001,7 +2001,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[IS FALSE($11)])
+LogicalProject(EMPNO=[$0], EXPR$1=[IS NOT TRUE($11)])
   LogicalJoin(condition=[=($9, $10)], joinType=[left])
     LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$7])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])


[05/16] calcite git commit: [CALCITE-1967] Elasticsearch 5 adapter (Christian Beikov)

Posted by mm...@apache.org.
[CALCITE-1967] Elasticsearch 5 adapter (Christian Beikov)

Move code that is independent of Elasticsearch into a
org.apache.calcite.adapter.elasticsearch package under core. It
provides abstract classes that the elasticsearch2 and elasticsearch5
modules make concrete.

Close apache/calcite#528


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

Branch: refs/heads/branch-1.14
Commit: e1525926790b058ae1845db32dcbf59b80ed270e
Parents: 2156d82
Author: Christian Beikov <ch...@gmail.com>
Authored: Thu Aug 24 16:28:50 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:12:29 2017 -0700

----------------------------------------------------------------------
 .../AbstractElasticsearchTable.java             | 128 ++++++++
 .../elasticsearch/ElasticsearchFilter.java      | 297 +++++++++++++++++++
 .../elasticsearch/ElasticsearchMethod.java      |  50 ++++
 .../elasticsearch/ElasticsearchProject.java     |  98 ++++++
 .../adapter/elasticsearch/ElasticsearchRel.java |  58 ++++
 .../elasticsearch/ElasticsearchRules.java       | 238 +++++++++++++++
 .../elasticsearch/ElasticsearchSchema.java      |  33 +++
 .../elasticsearch/ElasticsearchSort.java        | 104 +++++++
 .../elasticsearch/ElasticsearchTableScan.java   |  90 ++++++
 .../ElasticsearchToEnumerableConverter.java     | 124 ++++++++
 .../ElasticsearchToEnumerableConverterRule.java |  42 +++
 .../MapProjectionFieldVisitor.java              |  42 +++
 .../adapter/elasticsearch/package-info.java     |  26 ++
 elasticsearch/pom.xml                           | 148 ---------
 .../elasticsearch/ElasticsearchEnumerator.java  | 151 ----------
 .../elasticsearch/ElasticsearchFilter.java      | 287 ------------------
 .../elasticsearch/ElasticsearchMethod.java      |  50 ----
 .../elasticsearch/ElasticsearchProject.java     |  95 ------
 .../adapter/elasticsearch/ElasticsearchRel.java |  58 ----
 .../elasticsearch/ElasticsearchRules.java       | 236 ---------------
 .../elasticsearch/ElasticsearchSchema.java      | 127 --------
 .../ElasticsearchSchemaFactory.java             |  63 ----
 .../elasticsearch/ElasticsearchSort.java        |  93 ------
 .../elasticsearch/ElasticsearchTable.java       | 152 ----------
 .../elasticsearch/ElasticsearchTableScan.java   |  88 ------
 .../ElasticsearchToEnumerableConverter.java     | 124 --------
 .../ElasticsearchToEnumerableConverterRule.java |  42 ---
 .../adapter/elasticsearch/package-info.java     |  26 --
 .../calcite/test/ElasticsearchAdapterIT.java    | 270 -----------------
 .../resources/elasticsearch-zips-model.json     |  50 ----
 .../src/test/resources/log4j.properties         |  24 --
 elasticsearch2/pom.xml                          | 148 +++++++++
 .../Elasticsearch2Enumerator.java               | 152 ++++++++++
 .../elasticsearch2/Elasticsearch2Schema.java    | 134 +++++++++
 .../Elasticsearch2SchemaFactory.java            |  63 ++++
 .../elasticsearch2/Elasticsearch2Table.java     |  66 +++++
 .../adapter/elasticsearch2/package-info.java    |  26 ++
 .../calcite/test/Elasticsearch2AdapterIT.java   | 270 +++++++++++++++++
 .../resources/elasticsearch-zips-model.json     |  50 ++++
 .../src/test/resources/log4j.properties         |  24 ++
 elasticsearch5/pom.xml                          | 153 ++++++++++
 .../Elasticsearch5Enumerator.java               | 154 ++++++++++
 .../elasticsearch5/Elasticsearch5Schema.java    | 138 +++++++++
 .../Elasticsearch5SchemaFactory.java            |  63 ++++
 .../elasticsearch5/Elasticsearch5Table.java     |  86 ++++++
 .../adapter/elasticsearch5/package-info.java    |  26 ++
 .../calcite/test/Elasticsearch5AdapterIT.java   | 270 +++++++++++++++++
 .../resources/elasticsearch-zips-model.json     |  50 ++++
 .../src/test/resources/log4j.properties         |  24 ++
 pom.xml                                         |   4 +-
 site/_docs/adapter.md                           |   5 +-
 site/_docs/elasticsearch_adapter.md             |   5 +-
 sqlline                                         |   2 +-
 sqlline.bat                                     |   2 +-
 54 files changed, 3240 insertions(+), 2089 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
new file mode 100644
index 0000000..0980469
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
@@ -0,0 +1,128 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Table based on an Elasticsearch type.
+ */
+public abstract class AbstractElasticsearchTable extends AbstractQueryableTable
+    implements TranslatableTable {
+  protected final String indexName;
+  protected final String typeName;
+
+  /**
+   * Creates an ElasticsearchTable.
+   */
+  public AbstractElasticsearchTable(String indexName, String typeName) {
+    super(Object[].class);
+    this.indexName = indexName;
+    this.typeName = typeName;
+  }
+
+  @Override public String toString() {
+    return "ElasticsearchTable{" + typeName + "}";
+  }
+
+  public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
+    final RelDataType mapType = relDataTypeFactory.createMapType(
+        relDataTypeFactory.createSqlType(SqlTypeName.VARCHAR),
+        relDataTypeFactory.createTypeWithNullability(
+            relDataTypeFactory.createSqlType(SqlTypeName.ANY),
+            true));
+    return relDataTypeFactory.builder().add("_MAP", mapType).build();
+  }
+
+  public <T> Queryable<T> asQueryable(QueryProvider queryProvider, SchemaPlus schema,
+      String tableName) {
+    return new ElasticsearchQueryable<>(queryProvider, schema, this, tableName);
+  }
+
+  public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
+    final RelOptCluster cluster = context.getCluster();
+    return new ElasticsearchTableScan(cluster, cluster.traitSetOf(ElasticsearchRel.CONVENTION),
+        relOptTable, this, null);
+  }
+
+  /** Executes a "find" operation on the underlying type.
+   *
+   * <p>For example,
+   * <code>client.prepareSearch(index).setTypes(type)
+   * .setSource("{\"fields\" : [\"state\"]}")</code></p>
+   *
+   * @param index Elasticsearch index
+   * @param ops List of operations represented as Json strings.
+   * @param fields List of fields to project; or null to return map
+   * @return Enumerator of results
+   */
+  protected abstract Enumerable<Object> find(String index, List<String> ops,
+      List<Map.Entry<String, Class>> fields);
+
+  /**
+   * Implementation of {@link Queryable} based on
+   * a {@link AbstractElasticsearchTable}.
+   *
+   * @param <T> element type
+   */
+  public static class ElasticsearchQueryable<T> extends AbstractTableQueryable<T> {
+    public ElasticsearchQueryable(QueryProvider queryProvider, SchemaPlus schema,
+        AbstractElasticsearchTable table, String tableName) {
+      super(queryProvider, schema, table, tableName);
+    }
+
+    public Enumerator<T> enumerator() {
+      return null;
+    }
+
+    private String getIndex() {
+      return schema.unwrap(ElasticsearchSchema.class).getIndex();
+    }
+
+    private AbstractElasticsearchTable getTable() {
+      return (AbstractElasticsearchTable) table;
+    }
+
+    /** Called via code-generation.
+     *
+     * @see ElasticsearchMethod#ELASTICSEARCH_QUERYABLE_FIND
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public Enumerable<Object> find(List<String> ops,
+        List<Map.Entry<String, Class>> fields) {
+      return getTable().find(getIndex(), ops, fields);
+    }
+  }
+}
+
+// End AbstractElasticsearchTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java
new file mode 100644
index 0000000..c1bc317
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java
@@ -0,0 +1,297 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.Pair;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+
+/**
+ * Implementation of a {@link org.apache.calcite.rel.core.Filter}
+ * relational expression in Elasticsearch.
+ */
+public class ElasticsearchFilter extends Filter implements ElasticsearchRel {
+  public ElasticsearchFilter(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
+      RexNode condition) {
+    super(cluster, traitSet, child, condition);
+    assert getConvention() == ElasticsearchRel.CONVENTION;
+    assert getConvention() == child.getConvention();
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(0.1);
+  }
+
+  @Override public Filter copy(RelTraitSet relTraitSet, RelNode input, RexNode condition) {
+    return new ElasticsearchFilter(getCluster(), relTraitSet, input, condition);
+  }
+
+  @Override public void implement(Implementor implementor) {
+    implementor.visitChild(0, getInput());
+    List<String> fieldNames;
+    if (input instanceof Project) {
+      final List<RexNode> projects = ((Project) input).getProjects();
+      fieldNames = new ArrayList<>(projects.size());
+      for (RexNode project : projects) {
+        String name = project.accept(MapProjectionFieldVisitor.INSTANCE);
+        fieldNames.add(name);
+      }
+    } else {
+      fieldNames = ElasticsearchRules.elasticsearchFieldNames(getRowType());
+    }
+    Translator translator = new Translator(fieldNames);
+    String match = translator.translateMatch(condition);
+    implementor.add(match);
+  }
+
+  /**
+   * Translates {@link RexNode} expressions into Elasticsearch expression strings.
+   */
+  static class Translator {
+    final JsonBuilder builder = new JsonBuilder();
+    final Multimap<String, Pair<String, RexLiteral>> multimap =
+        HashMultimap.create();
+    final Map<String, RexLiteral> eqMap = new LinkedHashMap<>();
+    private final List<String> fieldNames;
+
+    Translator(List<String> fieldNames) {
+      this.fieldNames = fieldNames;
+    }
+
+    private String translateMatch(RexNode condition) {
+      // filter node
+      final Map<String, Object> filterMap = new LinkedHashMap<>();
+      filterMap.put("filter", translateOr(condition));
+
+      // constant_score node
+      final Map<String, Object> map = builder.map();
+      map.put("constant_score", filterMap);
+
+      return "\"query\" : " + builder.toJsonString(map).replaceAll("\\s+", "")
+          .toLowerCase(Locale.ROOT);
+    }
+
+    private Object translateOr(RexNode condition) {
+      final List<Object> list = new ArrayList<>();
+
+      final List<RexNode> orNodes = RelOptUtil.disjunctions(condition);
+      for (RexNode node : orNodes) {
+        List<Map<String, Object>> andNodes = translateAnd(node);
+
+        if (andNodes.size() > 0) {
+          Map<String, Object> andClause = new HashMap<>();
+          andClause.put("must", andNodes);
+
+          // boolean filters
+          LinkedHashMap<String, Object> filterEvaluator = new LinkedHashMap<>();
+          filterEvaluator.put("bool", andClause);
+          list.add(filterEvaluator);
+        } else {
+          list.add(andNodes.get(0));
+        }
+      }
+
+      if (orNodes.size() > 1) {
+        Map<String, Object> map = builder.map();
+        map.put("should", list);
+
+        // boolean filters
+        LinkedHashMap<String, Object> filterEvaluator = new LinkedHashMap<>();
+        filterEvaluator.put("bool", map);
+        return filterEvaluator;
+      } else {
+        return list.get(0);
+      }
+    }
+
+    private void addPredicate(Map<String, Object> map, String op, Object v) {
+      if (map.containsKey(op) && stronger(op, map.get(op), v)) {
+        return;
+      }
+      map.put(op, v);
+    }
+
+    /**
+     * Translates a condition that may be an AND of other conditions. Gathers
+     * together conditions that apply to the same field.
+     */
+    private List<Map<String, Object>> translateAnd(RexNode node0) {
+      eqMap.clear();
+      multimap.clear();
+      for (RexNode node : RelOptUtil.conjunctions(node0)) {
+        translateMatch2(node);
+      }
+      List<Map<String, Object>> filters = new ArrayList<>();
+      for (Map.Entry<String, RexLiteral> entry : eqMap.entrySet()) {
+        multimap.removeAll(entry.getKey());
+
+        Map<String, Object> filter = new HashMap<>();
+        filter.put(entry.getKey(), literalValue(entry.getValue()));
+
+        Map<String, Object> map = new HashMap<>();
+        map.put("term", filter);
+        filters.add(map);
+      }
+      for (Map.Entry<String, Collection<Pair<String, RexLiteral>>> entry
+          : multimap.asMap().entrySet()) {
+        Map<String, Object> map2 = builder.map();
+
+        Map<String, Object> map = new HashMap<>();
+        for (Pair<String, RexLiteral> s : entry.getValue()) {
+          if (!s.left.equals("not")) {
+            addPredicate(map2, s.left, literalValue(s.right));
+
+            Map<String, Object> filter = new HashMap<>();
+            filter.put(entry.getKey(), map2);
+
+            map.put("range", filter);
+          } else {
+            map2.put(entry.getKey(), literalValue(s.right));
+
+            Map<String, Object> termMap = new HashMap<>();
+            termMap.put("term", map2);
+
+            map.put("not", termMap);
+          }
+        }
+        filters.add(map);
+      }
+      return filters;
+    }
+
+    private boolean stronger(String key, Object v0, Object v1) {
+      if (key.equals("lt") || key.equals("lte")) {
+        if (v0 instanceof Number && v1 instanceof Number) {
+          return ((Number) v0).doubleValue() < ((Number) v1).doubleValue();
+        }
+        if (v0 instanceof String && v1 instanceof String) {
+          return v0.toString().compareTo(v1.toString()) < 0;
+        }
+      }
+      if (key.equals("gt") || key.equals("gte")) {
+        return stronger("lt", v1, v0);
+      }
+      return false;
+    }
+
+    private static Object literalValue(RexLiteral literal) {
+      return literal.getValue2();
+    }
+
+    private Void translateMatch2(RexNode node) {
+      switch (node.getKind()) {
+      case EQUALS:
+        return translateBinary(null, null, (RexCall) node);
+      case LESS_THAN:
+        return translateBinary("lt", "gt", (RexCall) node);
+      case LESS_THAN_OR_EQUAL:
+        return translateBinary("lte", "gte", (RexCall) node);
+      case NOT_EQUALS:
+        return translateBinary("not", "not", (RexCall) node);
+      case GREATER_THAN:
+        return translateBinary("gt", "lt", (RexCall) node);
+      case GREATER_THAN_OR_EQUAL:
+        return translateBinary("gte", "lte", (RexCall) node);
+      default:
+        throw new AssertionError("cannot translate " + node);
+      }
+    }
+
+    /**
+     * Translates a call to a binary operator, reversing arguments if
+     * necessary.
+     */
+    private Void translateBinary(String op, String rop, RexCall call) {
+      final RexNode left = call.operands.get(0);
+      final RexNode right = call.operands.get(1);
+      boolean b = translateBinary2(op, left, right);
+      if (b) {
+        return null;
+      }
+      b = translateBinary2(rop, right, left);
+      if (b) {
+        return null;
+      }
+      throw new AssertionError("cannot translate op " + op + " call " + call);
+    }
+
+    /**
+     * Translates a call to a binary operator. Returns whether successful.
+     */
+    private boolean translateBinary2(String op, RexNode left, RexNode right) {
+      switch (right.getKind()) {
+      case LITERAL:
+        break;
+      default:
+        return false;
+      }
+      final RexLiteral rightLiteral = (RexLiteral) right;
+      switch (left.getKind()) {
+      case INPUT_REF:
+        final RexInputRef left1 = (RexInputRef) left;
+        String name = fieldNames.get(left1.getIndex());
+        translateOp2(op, name, rightLiteral);
+        return true;
+      case CAST:
+        return translateBinary2(op, ((RexCall) left).operands.get(0), right);
+      case OTHER_FUNCTION:
+        String itemName = ElasticsearchRules.isItem((RexCall) left);
+        if (itemName != null) {
+          translateOp2(op, itemName, rightLiteral);
+          return true;
+        }
+        // fall through
+      default:
+        return false;
+      }
+    }
+
+    private void translateOp2(String op, String name, RexLiteral right) {
+      if (op == null) {
+        eqMap.put(name, right);
+      } else {
+        multimap.put(name, Pair.of(op, right));
+      }
+    }
+  }
+}
+
+// End ElasticsearchFilter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java
new file mode 100644
index 0000000..72753e6
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.elasticsearch;
+
+import org.apache.calcite.linq4j.tree.Types;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.lang.reflect.Method;
+import java.util.List;
+
+/**
+ * Builtin methods in the Elasticsearch adapter.
+ */
+enum ElasticsearchMethod {
+  ELASTICSEARCH_QUERYABLE_FIND(AbstractElasticsearchTable.ElasticsearchQueryable.class,
+      "find", List.class, List.class);
+
+  public final Method method;
+
+  public static final ImmutableMap<Method, ElasticsearchMethod> MAP;
+
+  static {
+    final ImmutableMap.Builder<Method, ElasticsearchMethod> builder = ImmutableMap.builder();
+    for (ElasticsearchMethod value: ElasticsearchMethod.values()) {
+      builder.put(value.method, value);
+    }
+    MAP = builder.build();
+  }
+
+  ElasticsearchMethod(Class clazz, String methodName, Class... argumentTypes) {
+    this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
+  }
+}
+
+// End ElasticsearchMethod.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
new file mode 100644
index 0000000..b42abd7
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
@@ -0,0 +1,98 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implementation of {@link org.apache.calcite.rel.core.Project}
+ * relational expression in Elasticsearch.
+ */
+public class ElasticsearchProject extends Project implements ElasticsearchRel {
+  public ElasticsearchProject(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
+      List<? extends RexNode> projects, RelDataType rowType) {
+    super(cluster, traitSet, input, projects, rowType);
+    assert getConvention() == ElasticsearchRel.CONVENTION;
+    assert getConvention() == input.getConvention();
+  }
+
+  @Override public Project copy(RelTraitSet relTraitSet, RelNode input, List<RexNode> projects,
+      RelDataType relDataType) {
+    return new ElasticsearchProject(getCluster(), traitSet, input, projects, relDataType);
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(0.1);
+  }
+
+  @Override public void implement(Implementor implementor) {
+    implementor.visitChild(0, getInput());
+
+    final List<String> inFields =
+        ElasticsearchRules.elasticsearchFieldNames(getInput().getRowType());
+    final ElasticsearchRules.RexToElasticsearchTranslator translator =
+        new ElasticsearchRules.RexToElasticsearchTranslator(
+            (JavaTypeFactory) getCluster().getTypeFactory(), inFields);
+
+    final List<String> findItems = new ArrayList<>();
+    final List<String> scriptFieldItems = new ArrayList<>();
+    for (Pair<RexNode, String> pair: getNamedProjects()) {
+      final String name = pair.right;
+      final String expr = pair.left.accept(translator);
+
+      if (expr.equals("\"" + name + "\"")) {
+        findItems.add(ElasticsearchRules.quote(name));
+      } else if (expr.matches("\"literal\":.+")) {
+        scriptFieldItems.add(ElasticsearchRules.quote(name)
+            + ":{\"script\": "
+            + expr.split(":")[1] + "}");
+      } else {
+        scriptFieldItems.add(ElasticsearchRules.quote(name)
+            + ":{\"script\":\"params._source."
+            + expr.replaceAll("\"", "") + "\"}");
+      }
+    }
+    final String findString = Util.toString(findItems, "", ", ", "");
+    final String scriptFieldString = "\"script_fields\": {"
+        + Util.toString(scriptFieldItems, "", ", ", "") + "}";
+    final String fieldString = "\"_source\" : [" + findString + "]"
+        + ", " + scriptFieldString;
+
+    for (String opfield : implementor.list) {
+      if (opfield.startsWith("\"_source\"")) {
+        implementor.list.remove(opfield);
+      }
+    }
+    implementor.add(fieldString);
+  }
+}
+
+// End ElasticsearchProject.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java
new file mode 100644
index 0000000..436adf9
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java
@@ -0,0 +1,58 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Relational expression that uses Elasticsearch calling convention.
+ */
+public interface ElasticsearchRel extends RelNode {
+  void implement(Implementor implementor);
+
+  /**
+   * Calling convention for relational operations that occur in Elasticsearch.
+   */
+  Convention CONVENTION = new Convention.Impl("ELASTICSEARCH", ElasticsearchRel.class);
+
+  /**
+   * Callback for the implementation process that converts a tree of
+   * {@link ElasticsearchRel} nodes into an Elasticsearch query.
+   */
+  class Implementor {
+    final List<String> list = new ArrayList<>();
+
+    RelOptTable table;
+    AbstractElasticsearchTable elasticsearchTable;
+
+    public void add(String findOp) {
+      list.add(findOp);
+    }
+
+    public void visitChild(int ordinal, RelNode input) {
+      assert ordinal == 0;
+      ((ElasticsearchRel) input).implement(this);
+    }
+  }
+}
+
+// End ElasticsearchRel.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java
new file mode 100644
index 0000000..c4f6728
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java
@@ -0,0 +1,238 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rules and relational operators for
+ * {@link ElasticsearchRel#CONVENTION ELASTICSEARCH}
+ * calling convention.
+ */
+class ElasticsearchRules {
+  static final RelOptRule[] RULES = {
+      ElasticsearchSortRule.INSTANCE,
+      ElasticsearchFilterRule.INSTANCE,
+      ElasticsearchProjectRule.INSTANCE
+  };
+
+  private ElasticsearchRules() {}
+
+  /**
+   * Returns 'string' if it is a call to item['string'], null otherwise.
+   */
+  static String isItem(RexCall call) {
+    if (call.getOperator() != SqlStdOperatorTable.ITEM) {
+      return null;
+    }
+    final RexNode op0 = call.getOperands().get(0);
+    final RexNode op1 = call.getOperands().get(1);
+
+    if (op0 instanceof RexInputRef
+        && ((RexInputRef) op0).getIndex() == 0
+        && op1 instanceof RexLiteral
+        && ((RexLiteral) op1).getValue2() instanceof String) {
+      return (String) ((RexLiteral) op1).getValue2();
+    }
+    return null;
+  }
+
+  static List<String> elasticsearchFieldNames(final RelDataType rowType) {
+    return SqlValidatorUtil.uniquify(
+        new AbstractList<String>() {
+          @Override public String get(int index) {
+            final String name = rowType.getFieldList().get(index).getName();
+            return name.startsWith("$") ? "_" + name.substring(2) : name;
+          }
+
+          @Override public int size() {
+            return rowType.getFieldCount();
+          }
+        },
+        SqlValidatorUtil.EXPR_SUGGESTER, true);
+  }
+
+  static String quote(String s) {
+    return "\"" + s + "\"";
+  }
+
+  /**
+   * Translator from {@link RexNode} to strings in Elasticsearch's expression
+   * language.
+   */
+  static class RexToElasticsearchTranslator extends RexVisitorImpl<String> {
+    private final JavaTypeFactory typeFactory;
+    private final List<String> inFields;
+
+    RexToElasticsearchTranslator(JavaTypeFactory typeFactory, List<String> inFields) {
+      super(true);
+      this.typeFactory = typeFactory;
+      this.inFields = inFields;
+    }
+
+    @Override public String visitLiteral(RexLiteral literal) {
+      if (literal.getValue() == null) {
+        return "null";
+      }
+      return "\"literal\":\""
+        + RexToLixTranslator.translateLiteral(literal, literal.getType(),
+          typeFactory, RexImpTable.NullAs.NOT_POSSIBLE)
+        + "\"";
+    }
+
+    @Override public String visitInputRef(RexInputRef inputRef) {
+      return quote(inFields.get(inputRef.getIndex()));
+    }
+
+    @Override public String visitCall(RexCall call) {
+      final String name = isItem(call);
+      if (name != null) {
+        return "\"" + name + "\"";
+      }
+
+      final List<String> strings = visitList(call.operands);
+      if (call.getKind() == SqlKind.CAST) {
+        return strings.get(0).startsWith("$") ? strings.get(0).substring(1) : strings.get(0);
+      }
+      if (call.getOperator() == SqlStdOperatorTable.ITEM) {
+        final RexNode op1 = call.getOperands().get(1);
+        if (op1 instanceof RexLiteral && op1.getType().getSqlTypeName() == SqlTypeName.INTEGER) {
+          return stripQuotes(strings.get(0)) + "[" + ((RexLiteral) op1).getValue2() + "]";
+        }
+      }
+      throw new IllegalArgumentException("Translation of " + call.toString()
+        + "is not supported by ElasticsearchProject");
+    }
+
+    private String stripQuotes(String s) {
+      return s.startsWith("'") && s.endsWith("'") ? s.substring(1, s.length() - 1) : s;
+    }
+
+    List<String> visitList(List<RexNode> list) {
+      final List<String> strings = new ArrayList<>();
+      for (RexNode node: list) {
+        strings.add(node.accept(this));
+      }
+      return strings;
+    }
+  }
+
+  /**
+   * Base class for planner rules that convert a relational expression to
+   * Elasticsearch calling convention.
+   */
+  abstract static class ElasticsearchConverterRule extends ConverterRule {
+    final Convention out;
+
+    ElasticsearchConverterRule(Class<? extends RelNode> clazz, RelTrait in, Convention out,
+        String description) {
+      super(clazz, in, out, description);
+      this.out = out;
+    }
+  }
+
+  /**
+   * Rule to convert a {@link org.apache.calcite.rel.core.Sort} to an
+   * {@link ElasticsearchSort}.
+   */
+  private static class ElasticsearchSortRule extends ElasticsearchConverterRule {
+    private static final ElasticsearchSortRule INSTANCE =
+        new ElasticsearchSortRule();
+
+    private ElasticsearchSortRule() {
+      super(Sort.class, Convention.NONE, ElasticsearchRel.CONVENTION,
+          "ElasticsearchSortRule");
+    }
+
+    @Override public RelNode convert(RelNode relNode) {
+      final Sort sort = (Sort) relNode;
+      final RelTraitSet traitSet = sort.getTraitSet().replace(out).replace(sort.getCollation());
+      return new ElasticsearchSort(relNode.getCluster(), traitSet,
+        convert(sort.getInput(), traitSet.replace(RelCollations.EMPTY)), sort.getCollation(),
+        sort.offset, sort.fetch);
+    }
+  }
+
+  /**
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to an
+   * {@link ElasticsearchFilter}.
+   */
+  private static class ElasticsearchFilterRule extends ElasticsearchConverterRule {
+    private static final ElasticsearchFilterRule INSTANCE = new ElasticsearchFilterRule();
+
+    private ElasticsearchFilterRule() {
+      super(LogicalFilter.class, Convention.NONE, ElasticsearchRel.CONVENTION,
+        "ElasticsearchFilterRule");
+    }
+
+    @Override public RelNode convert(RelNode relNode) {
+      final LogicalFilter filter = (LogicalFilter) relNode;
+      final RelTraitSet traitSet = filter.getTraitSet().replace(out);
+      return new ElasticsearchFilter(relNode.getCluster(), traitSet,
+        convert(filter.getInput(), out),
+        filter.getCondition());
+    }
+  }
+
+  /**
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject}
+   * to an {@link ElasticsearchProject}.
+   */
+  private static class ElasticsearchProjectRule extends ElasticsearchConverterRule {
+    private static final ElasticsearchProjectRule INSTANCE = new ElasticsearchProjectRule();
+
+    private ElasticsearchProjectRule() {
+      super(LogicalProject.class, Convention.NONE, ElasticsearchRel.CONVENTION,
+        "ElasticsearchProjectRule");
+    }
+
+    @Override public RelNode convert(RelNode relNode) {
+      final LogicalProject project = (LogicalProject) relNode;
+      final RelTraitSet traitSet = project.getTraitSet().replace(out);
+      return new ElasticsearchProject(project.getCluster(), traitSet,
+        convert(project.getInput(), out), project.getProjects(), project.getRowType());
+    }
+  }
+}
+
+// End ElasticsearchRules.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java
new file mode 100644
index 0000000..7247d19
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java
@@ -0,0 +1,33 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.schema.Schema;
+
+/**
+ * Gives access to some basic information of the Elasticsearch schema.
+ */
+public interface ElasticsearchSchema extends Schema {
+  /**
+   * The name of the Elasticsearch index.
+   *
+   * @return The index name
+   */
+  String getIndex();
+}
+
+// End ElasticsearchSchema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java
new file mode 100644
index 0000000..851e51b
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java
@@ -0,0 +1,104 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implementation of {@link org.apache.calcite.rel.core.Sort}
+ * relational expression in Elasticsearch.
+ */
+public class ElasticsearchSort extends Sort implements ElasticsearchRel {
+  public ElasticsearchSort(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
+      RelCollation collation, RexNode offset, RexNode fetch) {
+    super(cluster, traitSet, child, collation, offset, fetch);
+    assert getConvention() == ElasticsearchRel.CONVENTION;
+    assert getConvention() == child.getConvention();
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(0.05);
+  }
+
+  @Override public Sort copy(RelTraitSet traitSet, RelNode relNode, RelCollation relCollation,
+      RexNode offset, RexNode fetch) {
+    return new ElasticsearchSort(getCluster(), traitSet, relNode, collation, offset, fetch);
+  }
+
+  @Override public void implement(Implementor implementor) {
+    implementor.visitChild(0, getInput());
+    if (!collation.getFieldCollations().isEmpty()) {
+      final List<String> keys = new ArrayList<>();
+      if (input instanceof Project) {
+        final List<RexNode> projects = ((Project) input).getProjects();
+
+        for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+          RexNode project = projects.get(fieldCollation.getFieldIndex());
+          String name = project.accept(MapProjectionFieldVisitor.INSTANCE);
+          keys.add(ElasticsearchRules.quote(name) + ": " + direction(fieldCollation));
+        }
+      } else {
+        final List<RelDataTypeField> fields = getRowType().getFieldList();
+
+        for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+          final String name = fields.get(fieldCollation.getFieldIndex()).getName();
+          keys.add(ElasticsearchRules.quote(name) + ": " + direction(fieldCollation));
+        }
+      }
+
+      implementor.add("\"sort\": [ " + Util.toString(keys, "{", "}, {", "}") + "]");
+    }
+
+    if (offset != null) {
+      implementor.add("\"from\": " + ((RexLiteral) offset).getValue());
+    }
+
+    if (fetch != null) {
+      implementor.add("\"size\": " + ((RexLiteral) fetch).getValue());
+    }
+  }
+
+  private String direction(RelFieldCollation fieldCollation) {
+    switch (fieldCollation.getDirection()) {
+    case DESCENDING:
+    case STRICTLY_DESCENDING:
+      return "\"desc\"";
+    case ASCENDING:
+    case STRICTLY_ASCENDING:
+    default:
+      return "\"asc\"";
+    }
+  }
+}
+
+// End ElasticsearchSort.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java
new file mode 100644
index 0000000..33ca14f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.elasticsearch;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ * Relational expression representing a scan of an Elasticsearch type.
+ *
+ * <p> Additional operations might be applied,
+ * using the "find" method.</p>
+ */
+public class ElasticsearchTableScan extends TableScan implements ElasticsearchRel {
+  private final AbstractElasticsearchTable elasticsearchTable;
+  private final RelDataType projectRowType;
+
+  /**
+   * Creates an ElasticsearchTableScan.
+   *
+   * @param cluster Cluster
+   * @param traitSet Trait set
+   * @param table Table
+   * @param elasticsearchTable Elasticsearch table
+   * @param projectRowType Fields and types to project; null to project raw row
+   */
+  protected ElasticsearchTableScan(RelOptCluster cluster, RelTraitSet traitSet,
+       RelOptTable table, AbstractElasticsearchTable elasticsearchTable,
+       RelDataType projectRowType) {
+    super(cluster, traitSet, table);
+    this.elasticsearchTable = Preconditions.checkNotNull(elasticsearchTable);
+    this.projectRowType = projectRowType;
+
+    assert getConvention() == ElasticsearchRel.CONVENTION;
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    assert inputs.isEmpty();
+    return this;
+  }
+
+  @Override public RelDataType deriveRowType() {
+    return projectRowType != null ? projectRowType : super.deriveRowType();
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    final float f = projectRowType == null ? 1f : (float) projectRowType.getFieldCount() / 100f;
+    return super.computeSelfCost(planner, mq).multiplyBy(.1 * f);
+  }
+
+  @Override public void register(RelOptPlanner planner) {
+    planner.addRule(ElasticsearchToEnumerableConverterRule.INSTANCE);
+    for (RelOptRule rule: ElasticsearchRules.RULES) {
+      planner.addRule(rule);
+    }
+  }
+
+  @Override public void implement(Implementor implementor) {
+    implementor.elasticsearchTable = elasticsearchTable;
+    implementor.table = table;
+  }
+}
+
+// End ElasticsearchTableScan.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java
new file mode 100644
index 0000000..768f4e5
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java
@@ -0,0 +1,124 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+
+import java.util.AbstractList;
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Relational expression representing a scan of a table in an Elasticsearch data source.
+ */
+public class ElasticsearchToEnumerableConverter extends ConverterImpl implements EnumerableRel {
+  protected ElasticsearchToEnumerableConverter(RelOptCluster cluster, RelTraitSet traits,
+      RelNode input) {
+    super(cluster, ConventionTraitDef.INSTANCE, traits, input);
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new ElasticsearchToEnumerableConverter(getCluster(), traitSet, sole(inputs));
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(.1);
+  }
+
+  @Override public Result implement(EnumerableRelImplementor implementor, Prefer prefer) {
+    final BlockBuilder list = new BlockBuilder();
+    final ElasticsearchRel.Implementor elasticsearchImplementor =
+        new ElasticsearchRel.Implementor();
+    elasticsearchImplementor.visitChild(0, getInput());
+    final RelDataType rowType = getRowType();
+    final PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), rowType,
+        prefer.prefer(JavaRowFormat.ARRAY));
+    final Expression fields = list.append("fields",
+        constantArrayList(
+            Pair.zip(ElasticsearchRules.elasticsearchFieldNames(rowType),
+                new AbstractList<Class>() {
+                  @Override public Class get(int index) {
+                    return physType.fieldClass(index);
+                  }
+
+                  @Override public int size() {
+                    return rowType.getFieldCount();
+                  }
+                }),
+            Pair.class));
+    final Expression table = list.append("table",
+        elasticsearchImplementor.table
+            .getExpression(AbstractElasticsearchTable.ElasticsearchQueryable.class));
+    List<String> opList = elasticsearchImplementor.list;
+    final Expression ops = list.append("ops", constantArrayList(opList, String.class));
+    Expression enumerable = list.append("enumerable",
+        Expressions.call(table, ElasticsearchMethod.ELASTICSEARCH_QUERYABLE_FIND.method, ops,
+            fields));
+    if (CalcitePrepareImpl.DEBUG) {
+      System.out.println("Elasticsearch: " + opList);
+    }
+    Hook.QUERY_PLAN.run(opList);
+    list.add(Expressions.return_(null, enumerable));
+    return implementor.result(physType, list.toBlock());
+  }
+
+  /** E.g. {@code constantArrayList("x", "y")} returns
+   * "Arrays.asList('x', 'y')". */
+  private static <T> MethodCallExpression constantArrayList(List<T> values, Class clazz) {
+    return Expressions.call(BuiltInMethod.ARRAYS_AS_LIST.method,
+        Expressions.newArrayInit(clazz, constantList(values)));
+  }
+
+  /** E.g. {@code constantList("x", "y")} returns
+   * {@code {ConstantExpression("x"), ConstantExpression("y")}}. */
+  private static <T> List<Expression> constantList(List<T> values) {
+    return Lists.transform(values,
+        new Function<T, Expression>() {
+          @Nullable
+          @Override public Expression apply(@Nullable T t) {
+            return Expressions.constant(t);
+          }
+        });
+  }
+}
+
+// End ElasticsearchToEnumerableConverter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java
new file mode 100644
index 0000000..1047757
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java
@@ -0,0 +1,42 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+
+/**
+ * Rule to convert a relational expression from
+ * {@link ElasticsearchRel#CONVENTION} to {@link EnumerableConvention}.
+ */
+public class ElasticsearchToEnumerableConverterRule extends ConverterRule {
+  public static final ConverterRule INSTANCE = new ElasticsearchToEnumerableConverterRule();
+
+  private ElasticsearchToEnumerableConverterRule() {
+    super(RelNode.class, ElasticsearchRel.CONVENTION, EnumerableConvention.INSTANCE,
+        "ElasticsearchToEnumerableConverterRule");
+  }
+
+  @Override public RelNode convert(RelNode relNode) {
+    RelTraitSet newTraitSet = relNode.getTraitSet().replace(getOutConvention());
+    return new ElasticsearchToEnumerableConverter(relNode.getCluster(), newTraitSet, relNode);
+  }
+}
+
+// End ElasticsearchToEnumerableConverterRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/MapProjectionFieldVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/MapProjectionFieldVisitor.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/MapProjectionFieldVisitor.java
new file mode 100644
index 0000000..bcb99a7
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/MapProjectionFieldVisitor.java
@@ -0,0 +1,42 @@
+/*
+ * 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.adapter.elasticsearch;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+
+/**
+ * Visitor that extracts the actual field name from an item expression.
+ */
+public class MapProjectionFieldVisitor extends RexVisitorImpl<String> {
+  public static final MapProjectionFieldVisitor INSTANCE = new MapProjectionFieldVisitor();
+
+  private MapProjectionFieldVisitor() {
+    super(true);
+  }
+
+  @Override public String visitCall(RexCall call) {
+    if (call.op == SqlStdOperatorTable.ITEM) {
+      return ((RexLiteral) call.getOperands().get(1)).getValueAs(String.class);
+    }
+    return super.visitCall(call);
+  }
+}
+
+// End MapProjectionFieldVisitor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/core/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java
new file mode 100644
index 0000000..98e1a92
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * Base classes for a query provider based on an Elasticsearch DB.
+ */
+@PackageMarker
+package org.apache.calcite.adapter.elasticsearch;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/elasticsearch/pom.xml b/elasticsearch/pom.xml
deleted file mode 100644
index aee216a..0000000
--- a/elasticsearch/pom.xml
+++ /dev/null
@@ -1,148 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to you under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.calcite</groupId>
-    <artifactId>calcite</artifactId>
-    <version>1.14.0-SNAPSHOT</version>
-  </parent>
-
-  <artifactId>calcite-elasticsearch</artifactId>
-  <packaging>jar</packaging>
-  <version>1.14.0-SNAPSHOT</version>
-  <name>Calcite Elasticsearch</name>
-  <description>Elasticsearch adapter for Calcite</description>
-
-  <properties>
-    <top.dir>${project.basedir}/..</top.dir>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-core</artifactId>
-      <type>jar</type>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-core</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-linq4j</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.elasticsearch</groupId>
-      <artifactId>elasticsearch</artifactId>
-      <version>${elasticsearch-java-driver.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.carrotsearch</groupId>
-      <artifactId>hppc</artifactId>
-      <version>${hppc.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <version>${maven-dependency-plugin.version}</version>
-        <executions>
-          <execution>
-            <id>analyze</id>
-            <goals>
-              <goal>analyze-only</goal>
-            </goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-              <!-- ignore "unused but declared" warnings -->
-              <ignoredUnusedDeclaredDependencies>
-                <ignoredUnusedDeclaredDependency>org.apache.calcite.avatica:avatica</ignoredUnusedDeclaredDependency>
-                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-api</ignoredUnusedDeclaredDependency>
-                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-log4j12</ignoredUnusedDeclaredDependency>
-              </ignoredUnusedDeclaredDependencies>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-release-plugin</artifactId>
-      </plugin>
-      <!-- Parent module has the same plugin and does the work of
-          generating -sources.jar for each project. But without the
-          plugin declared here, IDEs don't know the sources are
-          available. -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-source-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>attach-sources</id>
-            <phase>verify</phase>
-            <goals>
-              <goal>jar-no-fork</goal>
-              <goal>test-jar-no-fork</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchEnumerator.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchEnumerator.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchEnumerator.java
deleted file mode 100644
index e7478f5..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchEnumerator.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.calcite.linq4j.Enumerator;
-import org.apache.calcite.linq4j.function.Function1;
-import org.apache.calcite.linq4j.tree.Primitive;
-
-import org.elasticsearch.search.SearchHit;
-
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Enumerator that reads from an Elasticsearch type.
- */
-public class ElasticsearchEnumerator implements Enumerator<Object> {
-  private final Iterator<SearchHit> cursor;
-  private final Function1<SearchHit, Object> getter;
-  private Object current;
-
-  /**
-   * Creates an ElasticsearchEnumerator.
-   *
-   * @param cursor Iterator over Elasticsearch {@link SearchHit} objects
-   * @param getter Converts an object into a list of fields
-   */
-  public ElasticsearchEnumerator(Iterator<SearchHit> cursor, Function1<SearchHit, Object> getter) {
-    this.cursor = cursor;
-    this.getter = getter;
-  }
-
-  public Object current() {
-    return current;
-  }
-
-  public boolean moveNext() {
-    if (cursor.hasNext()) {
-      SearchHit map = cursor.next();
-      current = getter.apply(map);
-      return true;
-    } else {
-      current = null;
-      return false;
-    }
-  }
-
-  public void reset() {
-    throw new UnsupportedOperationException();
-  }
-
-  public void close() {
-    // nothing to do
-  }
-
-  private static Function1<SearchHit, Map> mapGetter() {
-    return new Function1<SearchHit, Map>() {
-      public Map apply(SearchHit searchHitFields) {
-        return (Map) searchHitFields.fields();
-      }
-    };
-  }
-
-  private static Function1<SearchHit, Object> singletonGetter(final String fieldName,
-      final Class fieldClass) {
-    return new Function1<SearchHit, Object>() {
-      public Object apply(SearchHit searchHitFields) {
-        if (searchHitFields.fields().isEmpty()) {
-          return convert(searchHitFields.getSource(), fieldClass);
-        } else {
-          return convert(searchHitFields.getFields(), fieldClass);
-        }
-      }
-    };
-  }
-
-  /**
-   * Function that extracts a given set of fields from {@link SearchHit}
-   * objects.
-   *
-   * @param fields List of fields to project
-   */
-  private static Function1<SearchHit, Object[]> listGetter(
-      final List<Map.Entry<String, Class>> fields) {
-    return new Function1<SearchHit, Object[]>() {
-      public Object[] apply(SearchHit searchHitFields) {
-        Object[] objects = new Object[fields.size()];
-        for (int i = 0; i < fields.size(); i++) {
-          final Map.Entry<String, Class> field = fields.get(i);
-          final String name = field.getKey();
-          if (searchHitFields.fields().isEmpty()) {
-            objects[i] = convert(searchHitFields.getSource().get(name), field.getValue());
-          } else {
-            objects[i] = convert(searchHitFields.field(name).getValue(), field.getValue());
-          }
-        }
-        return objects;
-      }
-    };
-  }
-
-  static Function1<SearchHit, Object> getter(List<Map.Entry<String, Class>> fields) {
-    //noinspection unchecked
-    return fields == null
-      ? (Function1) mapGetter()
-      : fields.size() == 1
-      ? singletonGetter(fields.get(0).getKey(), fields.get(0).getValue())
-      : (Function1) listGetter(fields);
-  }
-
-  private static Object convert(Object o, Class clazz) {
-    if (o == null) {
-      return null;
-    }
-    Primitive primitive = Primitive.of(clazz);
-    if (primitive != null) {
-      clazz = primitive.boxClass;
-    } else {
-      primitive = Primitive.ofBox(clazz);
-    }
-    if (clazz.isInstance(o)) {
-      return o;
-    }
-    if (o instanceof Date && primitive != null) {
-      o = ((Date) o).getTime() / DateTimeUtils.MILLIS_PER_DAY;
-    }
-    if (o instanceof Number && primitive != null) {
-      return primitive.number((Number) o);
-    }
-    return o;
-  }
-}
-
-// End ElasticsearchEnumerator.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java
deleted file mode 100644
index 8e923a1..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchFilter.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.JsonBuilder;
-import org.apache.calcite.util.Pair;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-
-
-/**
- * Implementation of a {@link org.apache.calcite.rel.core.Filter}
- * relational expression in Elasticsearch.
- */
-public class ElasticsearchFilter extends Filter implements ElasticsearchRel {
-  public ElasticsearchFilter(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
-      RexNode condition) {
-    super(cluster, traitSet, child, condition);
-    assert getConvention() == ElasticsearchRel.CONVENTION;
-    assert getConvention() == child.getConvention();
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return super.computeSelfCost(planner, mq).multiplyBy(0.1);
-  }
-
-  @Override public Filter copy(RelTraitSet relTraitSet, RelNode input, RexNode condition) {
-    return new ElasticsearchFilter(getCluster(), relTraitSet, input, condition);
-  }
-
-  @Override public void implement(Implementor implementor) {
-    implementor.visitChild(0, getInput());
-    final List<String> fieldNames =
-        ElasticsearchRules.elasticsearchFieldNames(getRowType());
-    final Translator translator = new Translator(fieldNames);
-    final String match = translator.translateMatch(condition);
-    implementor.add(match);
-  }
-
-  /**
-   * Translates {@link RexNode} expressions into Elasticsearch expression strings.
-   */
-  static class Translator {
-    final JsonBuilder builder = new JsonBuilder();
-    final Multimap<String, Pair<String, RexLiteral>> multimap =
-        HashMultimap.create();
-    final Map<String, RexLiteral> eqMap = new LinkedHashMap<>();
-    private final List<String> fieldNames;
-
-    Translator(List<String> fieldNames) {
-      this.fieldNames = fieldNames;
-    }
-
-    private String translateMatch(RexNode condition) {
-      // filter node
-      final Map<String, Object> filterMap = new LinkedHashMap<>();
-      filterMap.put("filter", translateOr(condition));
-
-      // constant_score node
-      final Map<String, Object> map = builder.map();
-      map.put("constant_score", filterMap);
-
-      return "\"query\" : " + builder.toJsonString(map).replaceAll("\\s+", "")
-          .toLowerCase(Locale.ROOT);
-    }
-
-    private Object translateOr(RexNode condition) {
-      final List<Object> list = new ArrayList<>();
-
-      final List<RexNode> orNodes = RelOptUtil.disjunctions(condition);
-      for (RexNode node : orNodes) {
-        List<Map<String, Object>> andNodes = translateAnd(node);
-
-        if (andNodes.size() > 0) {
-          Map<String, Object> andClause = new HashMap<>();
-          andClause.put("must", andNodes);
-
-          // boolean filters
-          LinkedHashMap<String, Object> filterEvaluator = new LinkedHashMap<>();
-          filterEvaluator.put("bool", andClause);
-          list.add(filterEvaluator);
-        } else {
-          list.add(andNodes.get(0));
-        }
-      }
-
-      if (orNodes.size() > 1) {
-        Map<String, Object> map = builder.map();
-        map.put("should", list);
-
-        // boolean filters
-        LinkedHashMap<String, Object> filterEvaluator = new LinkedHashMap<>();
-        filterEvaluator.put("bool", map);
-        return filterEvaluator;
-      } else {
-        return list.get(0);
-      }
-    }
-
-    private void addPredicate(Map<String, Object> map, String op, Object v) {
-      if (map.containsKey(op) && stronger(op, map.get(op), v)) {
-        return;
-      }
-      map.put(op, v);
-    }
-
-    /**
-     * Translates a condition that may be an AND of other conditions. Gathers
-     * together conditions that apply to the same field.
-     */
-    private List<Map<String, Object>> translateAnd(RexNode node0) {
-      eqMap.clear();
-      multimap.clear();
-      for (RexNode node : RelOptUtil.conjunctions(node0)) {
-        translateMatch2(node);
-      }
-      List<Map<String, Object>> filters = new ArrayList<>();
-      for (Map.Entry<String, RexLiteral> entry : eqMap.entrySet()) {
-        multimap.removeAll(entry.getKey());
-
-        Map<String, Object> filter = new HashMap<>();
-        filter.put(entry.getKey(), literalValue(entry.getValue()));
-
-        Map<String, Object> map = new HashMap<>();
-        map.put("term", filter);
-        filters.add(map);
-      }
-      for (Map.Entry<String, Collection<Pair<String, RexLiteral>>> entry
-          : multimap.asMap().entrySet()) {
-        Map<String, Object> map2 = builder.map();
-
-        Map<String, Object> map = new HashMap<>();
-        for (Pair<String, RexLiteral> s : entry.getValue()) {
-          if (!s.left.equals("not")) {
-            addPredicate(map2, s.left, literalValue(s.right));
-
-            Map<String, Object> filter = new HashMap<>();
-            filter.put(entry.getKey(), map2);
-
-            map.put("range", filter);
-          } else {
-            map2.put(entry.getKey(), literalValue(s.right));
-
-            Map<String, Object> termMap = new HashMap<>();
-            termMap.put("term", map2);
-
-            map.put("not", termMap);
-          }
-        }
-        filters.add(map);
-      }
-      return filters;
-    }
-
-    private boolean stronger(String key, Object v0, Object v1) {
-      if (key.equals("lt") || key.equals("lte")) {
-        if (v0 instanceof Number && v1 instanceof Number) {
-          return ((Number) v0).doubleValue() < ((Number) v1).doubleValue();
-        }
-        if (v0 instanceof String && v1 instanceof String) {
-          return v0.toString().compareTo(v1.toString()) < 0;
-        }
-      }
-      if (key.equals("gt") || key.equals("gte")) {
-        return stronger("lt", v1, v0);
-      }
-      return false;
-    }
-
-    private static Object literalValue(RexLiteral literal) {
-      return literal.getValue2();
-    }
-
-    private Void translateMatch2(RexNode node) {
-      switch (node.getKind()) {
-      case EQUALS:
-        return translateBinary(null, null, (RexCall) node);
-      case LESS_THAN:
-        return translateBinary("lt", "gt", (RexCall) node);
-      case LESS_THAN_OR_EQUAL:
-        return translateBinary("lte", "gte", (RexCall) node);
-      case NOT_EQUALS:
-        return translateBinary("not", "not", (RexCall) node);
-      case GREATER_THAN:
-        return translateBinary("gt", "lt", (RexCall) node);
-      case GREATER_THAN_OR_EQUAL:
-        return translateBinary("gte", "lte", (RexCall) node);
-      default:
-        throw new AssertionError("cannot translate " + node);
-      }
-    }
-
-    /**
-     * Translates a call to a binary operator, reversing arguments if
-     * necessary.
-     */
-    private Void translateBinary(String op, String rop, RexCall call) {
-      final RexNode left = call.operands.get(0);
-      final RexNode right = call.operands.get(1);
-      boolean b = translateBinary2(op, left, right);
-      if (b) {
-        return null;
-      }
-      b = translateBinary2(rop, right, left);
-      if (b) {
-        return null;
-      }
-      throw new AssertionError("cannot translate op " + op + " call " + call);
-    }
-
-    /**
-     * Translates a call to a binary operator. Returns whether successful.
-     */
-    private boolean translateBinary2(String op, RexNode left, RexNode right) {
-      switch (right.getKind()) {
-      case LITERAL:
-        break;
-      default:
-        return false;
-      }
-      final RexLiteral rightLiteral = (RexLiteral) right;
-      switch (left.getKind()) {
-      case INPUT_REF:
-        final RexInputRef left1 = (RexInputRef) left;
-        String name = fieldNames.get(left1.getIndex());
-        translateOp2(op, name, rightLiteral);
-        return true;
-      case CAST:
-        return translateBinary2(op, ((RexCall) left).operands.get(0), right);
-      case OTHER_FUNCTION:
-        String itemName = ElasticsearchRules.isItem((RexCall) left);
-        if (itemName != null) {
-          translateOp2(op, itemName, rightLiteral);
-          return true;
-        }
-        // fall through
-      default:
-        return false;
-      }
-    }
-
-    private void translateOp2(String op, String name, RexLiteral right) {
-      if (op == null) {
-        eqMap.put(name, right);
-      } else {
-        multimap.put(name, Pair.of(op, right));
-      }
-    }
-  }
-}
-
-// End ElasticsearchFilter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java
deleted file mode 100644
index a0b3af6..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchMethod.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.linq4j.tree.Types;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.lang.reflect.Method;
-import java.util.List;
-
-/**
- * Builtin methods in the Elasticsearch adapter.
- */
-enum ElasticsearchMethod {
-  ELASTICSEARCH_QUERYABLE_FIND(ElasticsearchTable.ElasticsearchQueryable.class, "find",
-      List.class, List.class);
-
-  public final Method method;
-
-  public static final ImmutableMap<Method, ElasticsearchMethod> MAP;
-
-  static {
-    final ImmutableMap.Builder<Method, ElasticsearchMethod> builder = ImmutableMap.builder();
-    for (ElasticsearchMethod value: ElasticsearchMethod.values()) {
-      builder.put(value.method, value);
-    }
-    MAP = builder.build();
-  }
-
-  ElasticsearchMethod(Class clazz, String methodName, Class... argumentTypes) {
-    this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
-  }
-}
-
-// End ElasticsearchMethod.java


[09/16] calcite git commit: [CALCITE-1968] OpenGIS Simple Feature Access SQL 1.2.1: add GEOMETRY data type and first 35 functions

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/test/resources/sql/spatial.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/spatial.iq b/core/src/test/resources/sql/spatial.iq
new file mode 100755
index 0000000..1a37b44
--- /dev/null
+++ b/core/src/test/resources/sql/spatial.iq
@@ -0,0 +1,1114 @@
+# spatial.iq - Geo-spatial functions
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to you under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+!use geo
+!set outputformat csv
+
+# Check that the data set is OK.
+select count(*) as c from GEO."countries";
+C
+245
+!ok
+
+#### Geometry conversion functions (2D)
+
+# ST_AsBinary(geom) Geometry to Well Known Binary
+# Not implemented
+
+# ST_AsGML(geom) Geometry to GML
+# Not implemented
+
+# ST_AsText(geom) Alias for `ST_AsWKT`
+SELECT ST_AsText(ST_GeomFromText('POINT(-71.064544 42.28787)'));
+EXPR$0
+POINT (-71.064544 42.28787)
+!ok
+
+# ST_AsWKT(geom) Converts *geom* → Well-Known Text
+
+SELECT ST_AsWKT(ST_GeomFromText('POINT(-71.064544 42.28787)'));
+EXPR$0
+POINT (-71.064544 42.28787)
+!ok
+
+# PostGIS can implicitly assign from CHAR to GEOMETRY; we can't
+!if (false) {
+# ST_AsWKT(geom) Geometry to Well Known Text
+SELECT ST_AsText('01030000000100000005000000000000000000
+000000000000000000000000000000000000000000000000
+F03F000000000000F03F000000000000F03F000000000000F03
+F000000000000000000000000000000000000000000000000');
+!ok
+!}
+
+SELECT ST_AsWKT(CAST(NULL AS GEOMETRY));
+EXPR$0
+null
+!ok
+
+# ST_Force2D(geom) 3D Geometry to 2D Geometry
+# Not implemented
+
+# ST_GeomFromGML(gml [, srid ]) GML to Geometry
+# Not implemented
+
+# ST_GeomFromText(wkt [, srid ]) Returns a specified geometry value from Well-Known Text representation
+
+SELECT ST_GeomFromText('LINESTRING(-71.160281 42.258729,-71.160837 42.259113,-71.161144 42.25932)');
+EXPR$0
+{"paths":[[[-71.160281,42.258729],[-71.160837,42.259113],[-71.161144,42.25932]]]}
+!ok
+
+SELECT ST_GeomFromText('LINESTRING(-71.160281 42.258729,-71.160837 42.259113,-71.161144 42.25932)',4269);
+EXPR$0
+{"paths":[[[-71.160281,42.258729],[-71.160837,42.259113],[-71.161144,42.25932]]],"spatialReference":{"wkid":4269}}
+!ok
+
+SELECT ST_GeomFromText('MULTILINESTRING((-71.160281 42.258729,-71.160837 42.259113,-71.161144 42.25932))');
+EXPR$0
+{"paths":[[[-71.160281,42.258729],[-71.160837,42.259113],[-71.161144,42.25932]]]}
+!ok
+
+SELECT ST_GeomFromText('POINT(-71.064544 42.28787)');
+EXPR$0
+{"x":-71.064544,"y":42.28787}
+!ok
+
+SELECT ST_GeomFromText('POLYGON((-71.1776585052917 42.3902909739571,-71.1776820268866 42.3903701743239,
+-71.1776063012595 42.3903825660754,-71.1775826583081 42.3903033653531,-71.1776585052917 42.3902909739571))');
+EXPR$0
+{"rings":[[[-71.1776585052917,42.3902909739571],[-71.1776820268866,42.3903701743239],[-71.1776063012595,42.3903825660754],[-71.1775826583081,42.3903033653531],[-71.1776585052917,42.3902909739571]]]}
+!ok
+
+SELECT ST_GeomFromText('MULTIPOLYGON(((-71.1031880899493 42.3152774590236,
+-71.1031627617667 42.3152960829043,-71.102923838298 42.3149156848307,
+-71.1023097974109 42.3151969047397,-71.1019285062273 42.3147384934248,
+-71.102505233663 42.3144722937587,-71.10277487471 42.3141658254797,
+-71.103113945163 42.3142739188902,-71.10324876416 42.31402489987,
+-71.1033002961013 42.3140393340215,-71.1033488797549 42.3139495090772,
+-71.103396240451 42.3138632439557,-71.1041521907712 42.3141153348029,
+-71.1041411411543 42.3141545014533,-71.1041287795912 42.3142114839058,
+-71.1041188134329 42.3142693656241,-71.1041112482575 42.3143272556118,
+-71.1041072845732 42.3143851580048,-71.1041057218871 42.3144430686681,
+-71.1041065602059 42.3145009876017,-71.1041097995362 42.3145589148055,
+-71.1041166403905 42.3146168544148,-71.1041258822717 42.3146748022936,
+-71.1041375307579 42.3147318674446,-71.1041492906949 42.3147711126569,
+-71.1041598612795 42.314808571739,-71.1042515013869 42.3151287620809,
+-71.1041173835118 42.3150739481917,-71.1040809891419 42.3151344119048,
+-71.1040438678912 42.3151191367447,-71.1040194562988 42.3151832057859,
+-71.1038734225584 42.3151140942995,-71.1038446938243 42.3151006300338,
+-71.1038315271889 42.315094347535,-71.1037393329282 42.315054824985,
+-71.1035447555574 42.3152608696313,-71.1033436658644 42.3151648370544,
+-71.1032580383161 42.3152269126061,-71.103223066939 42.3152517403219,
+-71.1031880899493 42.3152774590236)),
+((-71.1043632495873 42.315113108546,-71.1043583974082 42.3151211109857,
+-71.1043443253471 42.3150676015829,-71.1043850704575 42.3150793250568,-71.1043632495873 42.315113108546)))',4326);
+EXPR$0
+{"rings":[[[-71.1031880899493,42.3152774590236],[-71.1031627617667,42.3152960829043],[-71.102923838298,42.3149156848307],[-71.1023097974109,42.3151969047397],[-71.1019285062273,42.3147384934248],[-... (1697 characters)
+!ok
+
+# Disabled: Should not return null
+!if (false) {
+SELECT ST_GeomFromText('GEOMETRYCOLLECTION(
+  POLYGON((-7 4.2,-7.1 4.2,-7.1 4.3,-7 4.2))
+  POINT(5 5)
+  POINT(-2 3)
+  LINESTRING(5 5, 10 10)');
+EXPR$0
+!ok
+!}
+
+# PostGIS does CIRCULARSTRING; we don't currently
+!if (false) {
+SELECT ST_GeomFromText('CIRCULARSTRING(220268 150415,220227 150505,220227 150406)');
+!ok
+!}
+
+# In PostGIS prior to 2.0, ST_GeomFromText('GEOMETRYCOLLECTION(EMPTY)') was allowed
+# but ST_GeomFromText('GEOMETRYCOLLECTION EMPTY') is not preferred.
+SELECT ST_GeomFromText('GEOMETRYCOLLECTION EMPTY');
+EXPR$0
+null
+!ok
+
+# ST_GeomFromWKB(wkb [, srid ]) Well Known Binary to Geometry
+# Not implemented
+
+# ST_GoogleMapLink(geom [, layerType [, zoom ]]) Geometry to Google map link
+# Not implemented
+
+# ST_LineFromText(wkt [, srid ]) Well Known Text to LINESTRING
+SELECT ST_LineFromText('LINESTRING(1 2, 3 4)') AS aline,
+  ST_LineFromText('POINT(1 2)') AS null_return;
+ALINE, NULL_RETURN
+{"paths":[[[1,2],[3,4]]]}, {"x":1,"y":2}
+!ok
+
+# ST_LineFromWKB(wkb [, srid ]) Well Known Binary to LINESTRING
+# Not implemented
+
+# ST_MLineFromText(wkt [, srid ]) Well Known Text to MULTILINESTRING
+SELECT ST_MLineFromText('MULTILINESTRING((1 2, 3 4), (4 5, 6 7))');
+EXPR$0
+{"paths":[[[1,2],[3,4]],[[4,5],[6,7]]]}
+!ok
+
+# ST_MPointFromText(wkt [, srid ]) Well Known Text to MULTIPOINT
+SELECT ST_MPointFromText('MULTIPOINT(1 2, 3 4)');
+EXPR$0
+{"points":[[1,2],[3,4]]}
+!ok
+
+SELECT ST_MPointFromText('MULTIPOINT(-70.9590 42.1180, -70.9611 42.1223)', 4326);
+EXPR$0
+{"points":[[-70.959,42.118],[-70.9611,42.1223]],"spatialReference":{"wkid":4326}}
+!ok
+
+# ST_MPolyFromText(wkt [, srid ]) Well Known Text to MULTIPOLYGON
+SELECT ST_MPolyFromText('MULTIPOLYGON Z(((0 0 1,20 0 1,20 20 1,0 20 1,0 0 1),(5 5 3,5 7 3,7 7 3,7 5 3,5 5 3)))');
+EXPR$0
+{"hasZ":true,"rings":[[[0,0,1],[0,20,1],[20,20,1],[20,0,1],[0,0,1]],[[5,5,3],[7,5,3],[7,7,3],[5,7,3],[5,5,3]]]}
+!ok
+
+SELECt ST_MPolyFromText('MULTIPOLYGON(((-70.916 42.1002,-70.9468 42.0946,-70.9765 42.0872,-70.9754 42.0875,-70.9749 42.0879,-70.9752 42.0881,-70.9754 42.0891,-70.9758 42.0894,-70.9759 42.0897,-70.9759 42.0899,-70.9754 42.0902,-70.9756 42.0906,-70.9753 42.0907,-70.9753 42.0917,-70.9757 42.0924,-70.9755 42.0928,-70.9755 42.0942,-70.9751 42.0948,-70.9755 42.0953,-70.9751 42.0958,-70.9751 42.0962,-70.9759 42.0983,-70.9767 42.0987,-70.9768 42.0991,-70.9771 42.0997,-70.9771 42.1003,-70.9768 42.1005,-70.977 42.1011,-70.9766 42.1019,-70.9768 42.1026,-70.9769 42.1033,-70.9775 42.1042,-70.9773 42.1043,-70.9776 42.1043,-70.9778 42.1048,-70.9773 42.1058,-70.9774 42.1061,-70.9779 42.1065,-70.9782 42.1078,-70.9788 42.1085,-70.9798 42.1087,-70.9806 42.109,-70.9807 42.1093,-70.9806 42.1099,-70.9809 42.1109,-70.9808 42.1112,-70.9798 42.1116,-70.9792 42.1127,-70.979 42.1129,-70.9787 42.1134,-70.979 42.1139,-70.9791 42.1141,-70.9987 42.1116,-71.0022 42.1273,
+    -70.9408 42.1513,-70.9315 42.1165,-70.916 42.1002)))',4326);
+EXPR$0
+{"rings":[[[-70.916,42.1002],[-70.9468,42.0946],[-70.9765,42.0872],[-70.9754,42.0875],[-70.9749,42.0879],[-70.9752,42.0881],[-70.9754,42.0891],[-70.9758,42.0894],[-70.9759,42.0897],[-70.9759,42.089... (1123 characters)
+!ok
+
+# ST_OSMMapLink(geom [, marker ]) Geometry to OSM map link
+# Not implemented
+
+# ST_PointFromText(wkt [, srid ]) Well Known Text to POINT
+SELECT ST_PointFromText('POINT(-71.064544 42.28787)');
+EXPR$0
+{"x":-71.064544,"y":42.28787}
+!ok
+
+SELECT ST_PointFromText('POINT(-71.064544 42.28787)', 4326);
+EXPR$0
+{"x":-71.064544,"y":42.28787,"spatialReference":{"wkid":4326}}
+!ok
+
+# ST_PointFromWKB(wkb [, srid ]) Well Known Binary to POINT
+# Not implemented
+
+# ST_PolyFromText(wkt [, srid ]) Well Known Text to POLYGON
+SELECT ST_PolyFromText('POLYGON Z((0 0 1,20 0 1,20 20 1,0 20 1,0 0 1))');
+EXPR$0
+{"hasZ":true,"rings":[[[0,0,1],[0,20,1],[20,20,1],[20,0,1],[0,0,1]]]}
+!ok
+
+SELECT ST_PolyFromText(CAST(NULL AS VARCHAR));
+EXPR$0
+null
+!ok
+
+SELECT ST_PolyFromText('POLYGON((0 0))');
+EXPR$0
+{"rings":[[[0,0],[0,0]]]}
+!ok
+
+# ST_PolyFromWKB(wkb [, srid ]) Well Known Binary to POLYGON
+# Not implemented
+
+# ST_ToMultiLine(geom) Converts the coordinates of *geom* (which may be a geometry-collection) into a multi-line-string
+# Not implemented
+
+# ST_ToMultiPoint(geom)) Converts the coordinates of *geom* (which may be a geometry-collection) into a multi-point
+# Not implemented
+
+# ST_ToMultiSegments(geom) Converts *geom* (which may be a geometry-collection) into a set of distinct segments stored in a multi-line-string
+# Not implemented
+
+#### Geometry conversion functions (3D)
+
+# ST_Force3D(geom) 2D Geometry to 3D Geometry
+# Not implemented
+
+#### Geometry creation functions (2D)
+
+# ST_BoundingCircle(geom) Returns the minimum bounding circle of *geom*
+# Not implemented
+
+# ST_Expand(geom, distance) Expands *geom*'s envelope
+# Not implemented
+
+# ST_Expand(geom, deltaX, deltaY) Expands *geom*'s envelope
+# Not implemented
+
+# ST_MakeEllipse(point, width, height) Constructs an ellipse
+# Not implemented
+
+# ST_MakeEnvelope(xMin, yMin, xMax, yMax  [, srid ]) Creates a rectangular Polygon
+# Not implemented
+
+# ST_MakeGrid(geom, deltaX, deltaY) Calculates a regular grid of polygons based on *geom*
+# Not implemented
+
+# ST_MakeGridPoints(geom, deltaX, deltaY) Calculates a regular grid of points based on *geom*
+# Not implemented
+
+# ST_MakeLine(point1 [, point ]*) Creates a line-string from the given points (or multi-points)
+
+SELECT ST_MakeLine(ST_Point(1.0,1.0), ST_Point(-1.0,-1.0));
+EXPR$0
+{"paths":[[[1,1],[-1,-1]]]}
+!ok
+
+SELECT ST_MakeLine(ST_Point(1.0,1.0), ST_Point(-1.0,-1.0), ST_Point(-3.0,0.0));
+EXPR$0
+{"paths":[[[1,1],[-1,-1],[-3,0]]]}
+!ok
+
+# ST_MakePoint(x, y [, z ]) Constructs a point from two or three coordinates
+
+# Return point with unknown SRID
+SELECT ST_MakePoint(-71.1043443253471, 42.3150676015829);
+EXPR$0
+{"x":-71.1043443253471,"y":42.3150676015829}
+!ok
+
+# Return point marked as WGS 84 long lat
+SELECT ST_SetSRID(ST_MakePoint(-71.1043443253471, 42.3150676015829),4326);
+EXPR$0
+{"x":-71.1043443253471,"y":42.3150676015829}
+!ok
+
+# Return a 3D point (e.g. has altitude)
+SELECT ST_MakePoint(1.0, 2.0, 1.5);
+EXPR$0
+{"x":1,"y":2,"z":1.5}
+!ok
+
+# Get z of point
+SELECT ST_Z(ST_MakePoint(1.0, 2.0,1.5));
+EXPR$0
+1.5
+!ok
+
+select "name", ST_MakePoint("latitude", "longitude") AS p
+from GEO."countries" AS c
+ORDER BY "latitude" DESC LIMIT 3;
+name, P
+U.S.Minor Outlying Islands, null
+Svalbard and Jan Mayen, {"x":77.553604,"y":23.670272}
+Greenland, {"x":71.706936,"y":-42.604303}
+!ok
+
+# ST_MakePolygon(lineString [, hole ]*) Creates a polygon from *lineString* with the given holes (which are required to be closed line-strings)
+# Not implemented
+
+# ST_MinimumDiameter(geom) Returns the minimum diameter of *geom*
+# Not implemented
+
+# ST_MinimumRectangle(geom) Returns the minimum rectangle enclosing *geom*
+# Not implemented
+
+# ST_OctogonalEnvelope(geom) Returns the octogonal envelope of *geom*
+# Not implemented
+
+# ST_RingBuffer(geom, bufferSize, bufferCount [, endCapStyle [, doDifference]]) Returns a multi-polygon of buffers centered at *geom* and of increasing buffer size
+# Not implemented
+
+### Geometry creation functions (3D)
+
+# ST_Extrude(geom, height [, flag]) Extrudes a geometry
+# Not implemented
+
+# ST_GeometryShadow(geom, point, height) Computes the shadow footprint of *geom*
+# Not implemented
+
+# ST_GeometryShadow(geom, azimuth, altitude, height [, unify ]) Computes the shadow footprint of *geom*
+# Not implemented
+
+#### Geometry properties (2D)
+
+# ST_Boundary(geom [, srid ]) Returns the boundary of *geom*
+SELECT ST_AsText(ST_Boundary(ST_GeomFromText('LINESTRING(1 1,0 0, -1 1)')));
+EXPR$0
+MULTIPOINT ((1 1), (-1 1))
+!ok
+
+SELECT ST_AsText(ST_Boundary(ST_GeomFromText('POLYGON((1 1,0 0, -1 1, 1 1))')));
+EXPR$0
+MULTILINESTRING ((1 1, 0 0, -1 1, 1 1))
+!ok
+
+# Using a 3d polygon
+SELECT ST_AsText(ST_Boundary(ST_GeomFromText('POLYGON Z((1 1 1,0 0 1, -1 1 1, 1 1 1))')));
+
+EXPR$0
+MULTILINESTRING Z ((1 1 1, 0 0 1, -1 1 1, 1 1 1))
+!ok
+
+# Using a 3d multilinestring
+SELECT ST_AsText(ST_Boundary(ST_GeomFromText('MULTILINESTRING Z((1 1 1,0 0 0.5, -1 1 1),(1 1 0.5,0 0 0.5, -1 1 0.5, 1 1 0.5) )')));
+
+EXPR$0
+MULTIPOINT Z ((1 1 1), (-1 1 1))
+!ok
+
+# ST_Centroid(geom) Returns the centroid of *geom* (which may be a geometry-collection)
+# Not implemented
+
+# ST_CompactnessRatio(polygon) Returns the square root of *polygon*'s area divided by the area of the circle with circumference equal to its perimeter
+# Not implemented
+
+# ST_CoordDim(geom) Returns the dimension of the coordinates of *geom*
+# Not implemented
+
+# ST_Dimension(geom) Returns the dimension of *geom*
+# Not implemented
+
+# ST_Distance(geom1, geom2) Returns the distance between *geom1* and *geom2*
+
+SELECT ST_Distance(
+    ST_GeomFromText('POINT(10 10)'),
+    ST_GeomFromText('POINT(40 50)'));
+EXPR$0
+50.0
+!ok
+
+SELECT ST_Distance(
+    ST_GeomFromText('POINT(10 10)',4326),
+    ST_GeomFromText('POINT(40 50)', 4326));
+EXPR$0
+50.0
+!ok
+
+# Geometry example - units in planar degrees 4326 is WGS 84 long lat unit=degrees
+SELECT ST_Distance(
+    ST_GeomFromText('POINT(-72.1235 42.3521)',4326),
+    ST_GeomFromText('LINESTRING(-72.1260 42.45, -72.123 42.1546)', 4326));
+EXPR$0
+0.0015056772638282166
+!ok
+
+# Geometry example - units in meters (SRID: 26986 Massachusetts state plane meters) (most accurate for Massachusetts)
+SELECT ST_Distance(
+    ST_Transform(ST_GeomFromText('POINT(-72.1235 42.3521)',4326),26986),
+    ST_Transform(ST_GeomFromText('LINESTRING(-72.1260 42.45, -72.123 42.1546)', 4326),26986));
+EXPR$0
+0.0015056772638282166
+!ok
+
+# Geometry example - units in meters (SRID: 2163 US National Atlas Equal area) (least accurate)
+SELECT ST_Distance(
+    ST_Transform(ST_GeomFromText('POINT(-72.1235 42.3521)',4326),2163),
+    ST_Transform(ST_GeomFromText('LINESTRING(-72.1260 42.45, -72.123 42.1546)', 4326),2163));
+
+EXPR$0
+0.0015056772638282166
+!ok
+
+# Disabled: PostgreSQL does geography, Calcite does not
+!if (false) {
+# same as geometry example but note units in meters - use sphere for slightly faster less accurate
+SELECT ST_Distance(gg1, gg2) As spheroid_dist, ST_Distance(gg1, gg2, false) As sphere_dist
+FROM (SELECT
+    ST_GeogFromText('SRID=4326;POINT(-72.1235 42.3521)') As gg1,
+    ST_GeogFromText('SRID=4326;LINESTRING(-72.1260 42.45, -72.123 42.1546)') As gg2) As foo;
+
+  spheroid_dist   |   sphere_dist
+------------------+------------------
+ 123.802076746848 | 123.475736916397
+!ok
+!}
+
+# ST_EndPoint(lineString) Returns the last coordinate of *lineString*
+# Not implemented
+
+# ST_Envelope(geom [, srid ]) Returns the envelope of *geom* (which may be a geometry-collection) as a geometry
+
+SELECT ST_AsText(ST_Envelope(ST_GeomFromText('POINT(1 3)')));
+EXPR$0
+POLYGON ((1 3, 1 3, 1 3, 1 3, 1 3))
+!ok
+
+SELECT ST_AsText(ST_Envelope(ST_GeomFromText('LINESTRING(0 0, 1 3)')));
+EXPR$0
+POLYGON ((0 0, 1 0, 1 3, 0 3, 0 0))
+!ok
+
+SELECT ST_AsText(ST_Envelope(ST_GeomFromText('POLYGON((0 0, 0 1, 1.0000001 1, 1.0000001 0, 0 0))')));
+EXPR$0
+POLYGON ((0 0, 1.0000001 0, 1.0000001 1, 0 1, 0 0))
+!ok
+
+SELECT ST_AsText(ST_Envelope(ST_GeomFromText('POLYGON((0 0, 0 1, 1.0000000001 1, 1.0000000001 0, 0 0))')));
+EXPR$0
+POLYGON ((0 0, 1.0000000001 0, 1.0000000001 1, 0 1, 0 0))
+!ok
+
+# ST_Explode(query [, fieldName]) Explodes the geometry-collections in the *fieldName* column of a query into multiple geometries
+# Not implemented
+
+# ST_Extent(geom) Returns the minimum bounding box of *geom* (which may be a geometry-collection)
+# Not implemented
+
+# ST_ExteriorRing(polygon) Returns the exterior ring of *polygon* as a linear-ring
+# Not implemented
+
+# ST_GeometryN(geomCollection, n) Returns the *n*th geometry of *geomCollection*
+# Not implemented
+
+# ST_GeometryType(geom) Returns the type of *geom*
+
+SELECT ST_GeometryType(ST_Point(0.0, 0.0));
+EXPR$0
+POINT
+!ok
+
+# ST_GeometryTypeCode(geom) Returns the type code of *geom*
+
+SELECT id, ST_GeometryType(g), ST_GeometryTypeCode(g) FROM (VALUES
+ ('ls', ST_GeomFromText('LINESTRING(77.29 29.07,77.42 29.26,77.27 29.31,77.29 29.07)')),
+ ('p', ST_Point(0.0, 0.0)),
+ ('np', ST_Point(0.0, CAST(NULL AS DECIMAL))),
+ ('mp', ST_GeomFromText('MULTIPOLYGON(((1 1, 2 2, 5 3, 1 1)),
+                                       ((0 0, 2 2, 5 3, 0 0)))'))) AS t(id, g);
+ID, EXPR$1, EXPR$2
+ls, LINESTRING, 2
+mp, POLYGON, 3
+np, null, null
+p , POINT, 1
+!ok
+
+# ST_InteriorRingN(polygon, n) Returns the *n*th interior ring of *polygon*
+# Not implemented
+
+# ST_IsClosed(geom) Returns whether *geom* is a closed line-string or multi-line-string
+# Not implemented
+
+# ST_IsEmpty(geom) Returns whether *geom* is empty
+# Not implemented
+
+# ST_IsRectangle(geom) Returns whether *geom* is a rectangle
+# Not implemented
+
+# ST_IsRing(geom) Returns whether *geom* is a closed and simple line-string or multi-line-string
+# Not implemented
+
+# ST_IsSimple(geom) Returns whether *geom* is simple
+# Not implemented
+
+# ST_IsValid(geom) Returns whether *geom* is valid
+# Not implemented
+
+# ST_IsValidDetail(geom [, selfTouchValid ]) Returns a valid detail as an array of objects
+# Not implemented
+
+# ST_IsValidReason(geom [, selfTouchValid ]) Returns text stating whether *geom* is valid, and if not valid, a reason why
+# Not implemented
+
+# ST_NPoints(geom) Returns the number of points in *geom*
+# Not implemented
+
+# ST_NumGeometries(geom) Returns the number of geometries in *geom* (1 if it is not a geometry-collection)
+# Not implemented
+
+# ST_NumInteriorRing(geom) Alias for `ST_NumInteriorRings`
+# Not implemented
+
+# ST_NumInteriorRings(geom) Returns the number of interior rings of *geom*
+# Not implemented
+
+# ST_NumPoints(lineString) Returns the number of points in *lineString*
+# Not implemented
+
+# ST_PointN(geom, n) Returns the *n*th point of a *lineString*
+# Not implemented
+
+# ST_PointOnSurface(geom) Returns an interior or boundary point of *geom*
+# Not implemented
+
+# ST_SRID(geom) Returns SRID value of *geom* or 0 if it does not have one
+# Not implemented
+
+# ST_StartPoint(lineString) Returns the first coordinate of *lineString*
+# Not implemented
+
+# ST_X(geom) Returns the x-value of the first coordinate of *geom*
+# Not implemented
+
+# ST_XMax(geom) Returns the maximum x-value of *geom*
+# Not implemented
+
+# ST_XMin(geom) Returns the minimum x-value of *geom*
+# Not implemented
+
+# ST_Y(geom) Returns the y-value of the first coordinate of *geom*
+# Not implemented
+
+# ST_YMax(geom) Returns the maximum y-value of *geom*
+# Not implemented
+
+# ST_YMin(geom) Returns the minimum y-value of *geom*
+# Not implemented
+
+#### Geometry properties (3D)
+
+# ST_Is3D(s) Returns whether *geom* has at least one z-coordinate
+
+SELECT ST_Is3D(ST_GeomFromText('POINT Z(1 2 0)'));
+EXPR$0
+true
+!ok
+
+SELECT ST_Is3D(ST_GeomFromText('POINT (1 2)'));
+EXPR$0
+false
+!ok
+
+# ST_Z(geom) Returns the z-value of the first coordinate of *geom*
+
+SELECT ST_Z(ST_GeomFromText('POINT Z(1 2 3)'));
+EXPR$0
+3.0
+!ok
+
+SELECT ST_Z(ST_GeomFromText('POINT (1 2)'));
+EXPR$0
+null
+!ok
+
+# Not implemented
+
+# ST_ZMax(geom) Returns the maximum z-value of *geom*
+# Not implemented
+
+# ST_ZMin(geom) Returns the minimum z-value of *geom*
+# Not implemented
+
+### Geometry predicates
+
+# ST_Contains(geom1, geom2) Returns whether *geom1* contains *geom2*
+
+SELECT ST_Contains(ST_Point(0.0, 0.0), ST_Point(1.0, 2.0));
+EXPR$0
+false
+!ok
+
+SELECT ST_Contains(ST_Point(0.0, 0.0), ST_Point(0.0, 0.0));
+EXPR$0
+true
+!ok
+
+# ST_ContainsProperly(geom1, geom2) Returns whether *geom1* contains *geom2*
+
+-- Example demonstrating difference between contains and contains properly
+SELECT ST_GeometryType(geomA) As geomtype, ST_Contains(geomA,geomA) AS acontainsa, ST_ContainsProperly(geomA, geomA) AS acontainspropa,
+   ST_Contains(geomA, ST_Boundary(geomA)) As acontainsba, ST_ContainsProperly(geomA, ST_Boundary(geomA)) As acontainspropba
+FROM (VALUES ( ST_Buffer(ST_Point(1.0,1.0), 5/*,1*/) ),
+             ( ST_MakeLine(ST_Point(1.0,1.0), ST_Point(-1.0,-1.0) ) ),
+             ( ST_Point(1.0,1.0))) As foo(geomA);
+
+GEOMTYPE, ACONTAINSA, ACONTAINSPROPA, ACONTAINSBA, ACONTAINSPROPBA
+LINESTRING, true, true, false, false
+POINT, true, true, false, false
+POLYGON, true, true, false, false
+!ok
+
+# ST_Covers(geom1, geom2) Returns whether no point in *geom2* is outside *geom1*
+# Not implemented
+
+# ST_Crosses(geom1, geom2) Returns whether *geom1* crosses *geom2*
+
+SELECT ST_Crosses(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                  ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+true
+!ok
+
+# ST_DWithin(geom1, geom2, distance) Returns whether *geom1* and *geom* are within *distance* of one another
+
+# Countries within 10 degrees of London
+select "name" from GEO."countries" AS c
+where ST_Distance(ST_MakePoint(51.5, -0.12), ST_MakePoint("latitude", "longitude")) < 10;
+name
+Andorra
+Belgium
+France
+Guernsey
+Ireland
+Isle of Man
+Jersey
+Luxembourg
+Netherlands
+Switzerland
+United Kingdom
+!ok
+
+# Countries within 10 degrees of London, formulated a different way
+select "name" from GEO."countries" AS c
+where ST_DWithin(ST_MakePoint(51.5, -0.12), ST_MakePoint("latitude", "longitude"), 10);
+name
+Andorra
+Belgium
+France
+Guernsey
+Ireland
+Isle of Man
+Jersey
+Luxembourg
+Netherlands
+Switzerland
+United Kingdom
+!ok
+
+# ST_Disjoint(geom1, geom2) Returns whether *geom1* and *geom2* are disjoint
+
+SELECT ST_Disjoint(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                   ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+false
+!ok
+
+
+# ST_EnvelopesIntersect(geom1, geom2) Returns whether the envelope of *geom1* intersects the envelope of *geom2*
+
+SELECT ST_EnvelopesIntersect(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                             ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+true
+!ok
+
+# ST_Equals(geom1, geom2) Returns whether *geom1* equals *geom2*
+
+SELECT ST_Equals(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                 ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+false
+!ok
+
+# ST_Intersects(geom1, geom2) Returns whether *geom1* intersects *geom2*
+
+SELECT ST_Intersects(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                     ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+true
+!ok
+
+# ST_OrderingEquals(geom1, geom2) Returns whether *geom1* equals *geom2* and their coordinates and component Geometries are listed in the same order
+# Not implemented
+
+# ST_Overlaps(geom1, geom2) Returns whether *geom1* overlaps *geom2*
+
+SELECT ST_Overlaps(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                   ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+false
+!ok
+
+# ST_Relate(geom1, geom2) Returns the DE-9IM intersection matrix of *geom1* and *geom2*
+# Not implemented
+
+# ST_Relate(geom1, geom2, iMatrix) Returns whether *geom1* and *geom2* are related by the given intersection matrix *iMatrix*
+# Not implemented
+
+# ST_Touches(geom1, geom2) Returns whether *geom1* touches *geom2*
+
+SELECT ST_Touches(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                  ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+false
+!ok
+
+# ST_Within(geom1, geom2) Returns whether *geom1* is within *geom2*
+
+SELECT ST_Within(ST_GeomFromText('LINESTRING(1 3, 5 3)'),
+                 ST_GeomFromText('LINESTRING(1 1, 5 2, 2 5)'));
+EXPR$0
+false
+!ok
+
+#### Geometry operators (2D)
+
+# ST_Buffer(geom, bufferSize [, quadSegs | style ]) Computes a buffer around *geom*
+
+SELECT ST_Buffer(
+ ST_GeomFromText('POINT(100 90)'),
+ 50);
+EXPR$0
+{"rings":[[[150,90],[149.89294616193018,86.72984353849284],[149.5722430686905,83.47369038899743],[149.0392640201615,80.24548389919359],[148.2962913144534,77.05904774487396],[147.34650647475527,73.9... (3574 characters)
+!ok
+
+SELECT ST_Buffer(
+ ST_GeomFromText('LINESTRING(10 10,30 10)'),
+ 5);
+EXPR$0
+{"rings":[[[10,5],[9.672984353849284,5.010705383806982],[9.347369038899743,5.042775693130948],[9.02454838991936,5.096073597983848],[8.705904774487397,5.1703708685546585],[8.39280267348419,5.2653493... (3532 characters)
+!ok
+
+SELECT ST_Buffer(
+  ST_GeomFromText('POLYGON((-71.1776585052917 42.3902909739571,-71.1776820268866 42.3903701743239,
+    -71.1776063012595 42.3903825660754,-71.1775826583081 42.3903033653531,-71.1776585052917 42.3902909739571))'),
+ 50);
+EXPR$0
+{"rings":[[[-21.17763234259735,42.39033677001625],[-21.284686180667173,39.12018030850909],[-21.60538927390683,35.864027159013666],[-22.138368322435824,32.635820669209835],[-22.881341028143936,29.44... (3872 characters)
+!ok
+
+# Negative buffer size makes the polgyon smaller
+SELECT ST_Buffer(
+  ST_GeomFromText('POLYGON((10 10,10 20,20 20,20 10))'),
+ -1);
+EXPR$0
+{"rings":[[[11,11],[11,19],[19,19],[19,11],[11,11]]]}
+!ok
+
+# ST_BUFFER(geom, bufferSize, style) variant - not implemented
+SELECT ST_Buffer(
+ ST_GeomFromText('POINT(100 90)'),
+ 50, 'quad_segs=8');
+at org.apache.calcite.runtime.GeoFunctions.todo
+!error GeoFunctions
+
+# ST_BUFFER(geom, bufferSize, quadSegs) variant - not implemented
+# When implemented, remove comment from ST_Contains test case
+SELECT ST_Buffer(
+ ST_GeomFromText('POINT(100 90)'),
+ 50, 2);
+at org.apache.calcite.runtime.GeoFunctions.todo
+!error GeoFunctions
+
+# ST_ConvexHull(geom) Computes the smallest convex polygon that contains all the points in the Geometry
+# Not implemented
+
+# ST_Difference(geom1, geom2) Computes the difference between two geometries
+# Not implemented
+
+# ST_Intersection(geom1, geom2) Computes the intersection of two geometries
+# Not implemented
+
+# ST_SymDifference(geom1, geom2) Computes the symmetric difference between two geometries
+# Not implemented
+
+# ST_Union(geom1, geom2) Computes the union of two or more geometries
+
+# NOTE: PostGIS altered the order: it returned MULTIPOINT(-2 3,1 2)
+SELECT ST_AsText(ST_Union(ST_GeomFromText('POINT(1 2)'),
+    ST_GeomFromText('POINT(-2 3)')));
+
+EXPR$0
+MULTIPOINT ((1 2), (-2 3))
+!ok
+
+# NOTE: PostGIS returned a point not a multipoint: POINT(1 2). ESRI bug?
+SELECT ST_AsText(ST_Union(ST_GeomFromText('POINT(1 2)'),
+    ST_GeomFromText('POINT(1 2)')));
+EXPR$0
+MULTIPOINT ((1 2))
+!ok
+
+# ST_Union(geomCollection) Computes the union of two or more geometries
+
+# Disabled: ST_GeomFromText cannot handle GEOMETRYCOLLECTION
+!if (false) {
+SELECT ST_AsText(st_union(ST_GeomFromText('GEOMETRYCOLLECTION(
+  POLYGON((-7 4.2,-7.1 4.2,-7.1 4.3,-7 4.2))
+  POINT(5 5)
+  POINT(-2 3)
+  LINESTRING(5 5, 10 10)')));
+EXPR$0
+null
+!ok
+!}
+
+# ST_UNION(ARRAY[GEOMETRY]) is a PostGIS extension
+# We don't support it
+!if (false) {
+
+SELECT ST_Union(ARRAY(SELECT the_geom FROM sometable));
+!ok
+
+SELECT ST_AsText(ST_Union(ARRAY[ST_GeomFromText('LINESTRING(1 2, 3 4)'),
+    ST_GeomFromText('LINESTRING(3 4, 4 5)')])) As wktunion;
+
+--wktunion---
+MULTILINESTRING((3 4,4 5),(1 2,3 4))
+!ok
+!}
+
+#### Affine transformation functions (3D and 2D)
+
+# ST_Rotate(geom, angle [, origin | x, y]) Rotates a *geom* counter-clockwise by *angle* (in radians) about *origin* (or the point (*x*, *y*))
+# Not implemented
+
+# ST_Scale(geom, xFactor, yFactor [, zFactor ]) Scales *geom* by multiplying the ordinates by the indicated scale factors
+# Not implemented
+
+# ST_Translate(geom, x, y, [, z]) Translates *geom*
+# Not implemented
+
+#### Geometry editing functions (2D)
+
+# ST_AddPoint(geom, point [, tolerance ]) Adds *point* to *geom* with a given *tolerance* (default 0)
+# Not implemented
+
+# ST_CollectionExtract(geom, dimension) Filters *geom*, returning a multi-geometry of those members with a given *dimension* (1 = point, 2 = line-string, 3 = polygon)
+# Not implemented
+
+# ST_Densify(geom, tolerance) Inserts extra vertices every *tolerance* along the line segments of *geom*
+# Not implemented
+
+# ST_FlipCoordinates(geom) Flips the X and Y coordinates of *geom*
+# Not implemented
+
+# ST_Holes(geom) Returns the holes in *geom* (which may be a geometry-collection)
+# Not implemented
+
+# ST_Normalize(geom) Converts *geom* to normal form
+# Not implemented
+
+# ST_RemoveDuplicatedCoordinates(geom) Removes duplicated coordinates from *geom*
+# Not implemented
+
+# ST_RemoveHoles(geom) Removes a *geom*'s holes
+# Not implemented
+
+# ST_RemovePoints(geom, poly) Removes all coordinates of *geom* located within *poly*; null if all coordinates are removed
+# Not implemented
+
+# ST_RemoveRepeatedPoints(geom, tolerance) Removes from *geom* all repeated points (or points within *tolerance* of another point)
+# Not implemented
+
+# ST_Reverse(geom) Reverses the vertex order of *geom*
+# Not implemented
+
+#### Geometry editing functions (3D)
+
+# ST_AddZ(geom, zToAdd) Adds *zToAdd* to the z-coordinate of *geom*
+# Not implemented
+
+# ST_Interpolate3DLine(geom) Returns *geom* with a interpolation of z values, or null if it is not a line-string or multi-line-string
+# Not implemented
+
+# ST_MultiplyZ(geom, zFactor) Returns *geom* with its z-values multiplied by *zFactor*
+# Not implemented
+
+# ST_Reverse3DLine(geom [, sortOrder ]) Potentially reverses *geom* according to the z-values of its first and last coordinates
+# Not implemented
+
+# ST_UpdateZ(geom, newZ [, updateCondition ]) Updates the z-values of *geom*
+# Not implemented
+
+# ST_ZUpdateLineExtremities(geom, startZ, endZ [, interpolate ]) Updates the start and end z-values of *geom*
+# Not implemented
+
+#### Geometry measurement functions (2D)
+
+# ST_Area(geom) Returns the area of *geom* (which may be a geometry collection)
+# Not implemented
+
+# ST_ClosestCoordinate(geom, point) Returns the coordinate(s) of *geom* closest to *point*
+# Not implemented
+
+# ST_ClosestPoint(geom1, geom2) Returns the point of *geom1* closest to *geom2*
+# Not implemented
+
+# ST_FurthestCoordinate(geom, point) Returns the coordinate(s) of *geom* that are furthest from *point*
+# Not implemented
+
+# ST_Length(lineString) Returns the length of *lineString*
+# Not implemented
+
+# ST_LocateAlong(geom, segmentLengthFraction, offsetDistance) Returns a multi-point containing points along the line segments of *geom* at *segmentLengthFraction* and *offsetDistance*
+# Not implemented
+
+# ST_LongestLine(geom1, geom2) Returns the 2-dimensional longest line-string between the points of *geom1* and *geom2*
+# Not implemented
+
+# ST_MaxDistance(geom1, geom2) Computes the maximum distance between *geom1* and *geom2*
+# Not implemented
+
+# ST_Perimeter(polygon) Returns the length of the perimeter of *polygon* (which may be a multi-polygon)
+# Not implemented
+
+# ST_ProjectPoint(point, lineString) Projects *point* onto a *lineString* (which may be a multi-line-string)
+# Not implemented
+
+#### Geometry measurement functions (3D)
+
+# ST_3DArea(geom) Return a polygon's 3D area
+# Not implemented
+
+# ST_3DLength(geom) Returns the 3D length of a line-string
+# Not implemented
+
+# ST_3DPerimeter(geom) Returns the 3D perimeter of a polygon or multi-polygon
+# Not implemented
+
+# ST_SunPosition(point [, timestamp ]) Computes the sun position at *point* and *timestamp* (now by default)
+# Not implemented
+
+#### Geometry processing functions (2D)
+
+# ST_LineIntersector(geom1, geom2) Splits *geom1* (a line-string) with *geom2*
+# Not implemented
+
+# ST_LineMerge(geom) Merges a collection of linear components to form a line-string of maximal length
+# Not implemented
+
+# ST_MakeValid(geom [, preserveGeomDim [, preserveDuplicateCoord [, preserveCoordDim]]]) Makes *geom* valid
+# Not implemented
+
+# ST_Polygonize(geom) Creates a multi-polygon from edges of *geom*
+# Not implemented
+
+# ST_PrecisionReducer(geom, n) Reduces *geom*'s precision to *n* decimal places
+# Not implemented
+
+# ST_RingSideBuffer(geom, bufferSize, bufferCount [, endCapStyle [, doDifference]]) Computes a ring buffer on one side
+# Not implemented
+
+# ST_SideBuffer(geom, bufferSize [, bufferStyle ]) Compute a single buffer on one side
+# Not implemented
+
+# ST_Simplify(geom, distance) Simplifies *geom* using the Douglas-Peuker algorithm with a *distance* tolerance
+# Not implemented
+
+# ST_SimplifyPreserveTopology(geom) Simplifies *geom*, preserving its topology
+# Not implemented
+
+# ST_Snap(geom1, geom2, tolerance) Snaps *geom1* and *geom2* together
+# Not implemented
+
+# ST_Split(geom1, geom2 [, tolerance]) Splits *geom1* by *geom2* using *tolerance* (default 1E-6) to determine where the point splits the line
+# Not implemented
+
+#### Geometry projection functions
+
+# ST_SetSRID(geom, srid) Returns a copy of *geom* with a new SRID
+
+SELECT ST_SetSRID(ST_MakePoint(-123.365556, 48.428611),4326) As wgs84long_lat;
+WGS84LONG_LAT
+{"x":-123.365556,"y":48.428611}
+!ok
+
+# Mark a point as WGS 84 long lat and then transform to web mercator (Spherical Mercator)
+SELECT ST_Transform(ST_SetSRID(ST_MakePoint(-123.365556, 48.428611),4326),3785) As sphere_merc;
+SPHERE_MERC
+{"x":-123.365556,"y":48.428611,"spatialReference":{"wkid":102113,"latestWkid":3785}}
+!ok
+
+# ST_Transform(geom, srid) Transforms *geom* from one coordinate reference system (CRS) to the CRS specified by *srid*
+
+SELECT ST_AsText(ST_Transform(ST_GeomFromText('POLYGON((743238 2967416,743238 2967450,
+  743265 2967450,743265.625 2967416,743238 2967416))',2249),4326)) As wgs_geom;
+
+WGS_GEOM
+MULTIPOLYGON (((743238 2967416, 743265.625 2967416, 743265 2967450, 743238 2967450, 743238 2967416)))
+!ok
+
+#### Trigonometry functions
+
+# ST_Azimuth(point1, point2) Return the azimuth of the segment from *point1* to *point2*
+# Not implemented
+
+#### Topography functions
+
+# ST_TriangleAspect(geom) Returns the aspect of a triangle
+# Not implemented
+
+# ST_TriangleContouring(query \[, z1, z2, z3 ]\[, varArgs]*) Splits triangles into smaller triangles according to classes
+# Not implemented
+
+# ST_TriangleDirection(geom) Computes the direction of steepest ascent of a triangle and returns it as a line-string
+# Not implemented
+
+# ST_TriangleSlope(geom) Computes the slope of a triangle as a percentage
+# Not implemented
+
+# ST_Voronoi(geom [, outDimension [, envelopePolygon ]]) Creates a Voronoi diagram
+# Not implemented
+
+#### Triangulation functions
+
+# ST_ConstrainedDelaunay(geom [, flag [, quality ]]) Computes a constrained Delaunay triangulation based on *geom*
+# Not implemented
+
+# ST_Delaunay(geom [, flag [, quality ]]) Computes a Delaunay triangulation based on points
+# Not implemented
+
+# ST_Tessellate(polygon) Tessellates *polygon* (may be multi-polygon) with adaptive triangles
+# Not implemented
+
+#### Geometry aggregate functions
+
+# ST_Accum(geom) Accumulates *geom* into a geometry-collection (or multi-point, multi-line-string or multi-polygon if possible)
+# Not implemented
+
+# ST_Collect(geom) Alias for `ST_Accum`
+# Not implemented
+
+# ST_Union(geom) Computes the union of geometries
+# Not implemented
+
+# Disabled - ST_Union agg function is not implemented
+!if (false) {
+SELECT ST_AsText(st_union(the_geom))
+FROM (VALUES ST_GeomFromText('POLYGON((-7 4.2,-7.1 4.2,-7.1 4.3,-7 4.2))'),
+  ST_GeomFromText('POINT(5 5)'),
+  ST_GeomFromText('POINT(-2 3)'),
+  ST_GeomFromText('LINESTRING(5 5, 10 10)')) as foo(the_geom);
+
+st_asewkt
+---------
+GEOMETRYCOLLECTION(POINT(-2 3 1),LINESTRING(5 5 5,10 10 10),POLYGON((-7 4.2 5,-7.1 4.2 5,-7.1 4.3 5,-7 4.2 5)))
+!ok
+!}
+
+# 3d example - sort of supports 3d (and with mixed dimensions!)
+# WRONG: Currently returns 4 rows, should return 1 row when ST_Union is aggregate function
+SELECT ST_AsText(st_union(the_geom))
+FROM (
+  SELECT ST_GeomFromText('POLYGON((-7 4.2,-7.1 4.2,-7.1 4.3,-7 4.2))') as the_geom
+  UNION ALL
+  SELECT ST_GeomFromText('POINT Z(5 5 5)') as the_geom
+  UNION ALL
+  SELECT ST_GeomFromText('POINT Z(-2 3 1)') as the_geom
+  UNION ALL
+  SELECT ST_GeomFromText('LINESTRING Z(5 5 5, 10 10 10)') as the_geom ) as foo;
+
+EXPR$0
+MULTILINESTRING Z ((5 5 5, 10 10 10))
+MULTIPOLYGON (((-7 4.2, -7.1 4.3, -7.1 4.2, -7 4.2)))
+POINT Z (-2 3 1)
+POINT Z (5 5 5)
+!ok
+
+# 3d example not mixing dimensions
+# WRONG: Currently returns 4 rows, should return 1 row when ST_Union is aggregate function
+SELECT ST_AsText(st_union(the_geom))
+FROM (
+  SELECT ST_GeomFromText('POLYGON Z((-7 4.2 2,-7.1 4.2 3,-7.1 4.3 2,-7 4.2 2))') as the_geom
+  UNION ALL
+  SELECT ST_GeomFromText('POINT Z(5 5 5)') as the_geom
+  UNION ALL
+  SELECT ST_GeomFromText('POINT Z(-2 3 1)') as the_geom
+  UNION ALL
+  SELECT ST_GeomFromText('LINESTRING Z(5 5 5, 10 10 10)') as the_geom ) as foo;
+
+EXPR$0
+MULTILINESTRING Z ((5 5 5, 10 10 10))
+MULTIPOLYGON Z (((-7 4.2 2, -7.1 4.3 2, -7.1 4.2 3, -7 4.2 2)))
+POINT Z (-2 3 1)
+POINT Z (5 5 5)
+!ok
+
+# End spatial.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/file/src/test/resources/geo/countries.csv
----------------------------------------------------------------------
diff --git a/file/src/test/resources/geo/countries.csv b/file/src/test/resources/geo/countries.csv
new file mode 100644
index 0000000..9ae51a2
--- /dev/null
+++ b/file/src/test/resources/geo/countries.csv
@@ -0,0 +1,246 @@
+country:string,latitude:decimal,longitude:decimal,name:string
+AD,42.546245,1.601554,Andorra
+AE,23.424076,53.847818,United Arab Emirates
+AF,33.93911,67.709953,Afghanistan
+AG,17.060816,-61.796428,Antigua and Barbuda
+AI,18.220554,-63.068615,Anguilla
+AL,41.153332,20.168331,Albania
+AM,40.069099,45.038189,Armenia
+AN,12.226079,-69.060087,Netherlands Antilles
+AO,-11.202692,17.873887,Angola
+AQ,-75.250973,-0.071389,Antarctica
+AR,-38.416097,-63.616672,Argentina
+AS,-14.270972,-170.132217,American Samoa
+AT,47.516231,14.550072,Austria
+AU,-25.274398,133.775136,Australia
+AW,12.52111,-69.968338,Aruba
+AZ,40.143105,47.576927,Azerbaijan
+BA,43.915886,17.679076,Bosnia and Herzegovina
+BB,13.193887,-59.543198,Barbados
+BD,23.684994,90.356331,Bangladesh
+BE,50.503887,4.469936,Belgium
+BF,12.238333,-1.561593,Burkina Faso
+BG,42.733883,25.48583,Bulgaria
+BH,25.930414,50.637772,Bahrain
+BI,-3.373056,29.918886,Burundi
+BJ,9.30769,2.315834,Benin
+BM,32.321384,-64.75737,Bermuda
+BN,4.535277,114.727669,Brunei
+BO,-16.290154,-63.588653,Bolivia
+BR,-14.235004,-51.92528,Brazil
+BS,25.03428,-77.39628,Bahamas
+BT,27.514162,90.433601,Bhutan
+BV,-54.423199,3.413194,Bouvet Island
+BW,-22.328474,24.684866,Botswana
+BY,53.709807,27.953389,Belarus
+BZ,17.189877,-88.49765,Belize
+CA,56.130366,-106.346771,Canada
+CC,-12.164165,96.870956,Cocos [Keeling] Islands
+CD,-4.038333,21.758664,Congo [DRC]
+CF,6.611111,20.939444,Central African Republic
+CG,-0.228021,15.827659,Congo [Republic]
+CH,46.818188,8.227512,Switzerland
+CI,7.539989,-5.54708,Côte d'Ivoire
+CK,-21.236736,-159.777671,Cook Islands
+CL,-35.675147,-71.542969,Chile
+CM,7.369722,12.354722,Cameroon
+CN,35.86166,104.195397,China
+CO,4.570868,-74.297333,Colombia
+CR,9.748917,-83.753428,Costa Rica
+CU,21.521757,-77.781167,Cuba
+CV,16.002082,-24.013197,Cape Verde
+CX,-10.447525,105.690449,Christmas Island
+CY,35.126413,33.429859,Cyprus
+CZ,49.817492,15.472962,Czech Republic
+DE,51.165691,10.451526,Germany
+DJ,11.825138,42.590275,Djibouti
+DK,56.26392,9.501785,Denmark
+DM,15.414999,-61.370976,Dominica
+DO,18.735693,-70.162651,Dominican Republic
+DZ,28.033886,1.659626,Algeria
+EC,-1.831239,-78.183406,Ecuador
+EE,58.595272,25.013607,Estonia
+EG,26.820553,30.802498,Egypt
+EH,24.215527,-12.885834,Western Sahara
+ER,15.179384,39.782334,Eritrea
+ES,40.463667,-3.74922,Spain
+ET,9.145,40.489673,Ethiopia
+FI,61.92411,25.748151,Finland
+FJ,-16.578193,179.414413,Fiji
+FK,-51.796253,-59.523613,Falkland Islands [Islas Malvinas]
+FM,7.425554,150.550812,Micronesia
+FO,61.892635,-6.911806,Faroe Islands
+FR,46.227638,2.213749,France
+GA,-0.803689,11.609444,Gabon
+GB,55.378051,-3.435973,United Kingdom
+GD,12.262776,-61.604171,Grenada
+GE,42.315407,43.356892,Georgia
+GF,3.933889,-53.125782,French Guiana
+GG,49.465691,-2.585278,Guernsey
+GH,7.946527,-1.023194,Ghana
+GI,36.137741,-5.345374,Gibraltar
+GL,71.706936,-42.604303,Greenland
+GM,13.443182,-15.310139,Gambia
+GN,9.945587,-9.696645,Guinea
+GP,16.995971,-62.067641,Guadeloupe
+GQ,1.650801,10.267895,Equatorial Guinea
+GR,39.074208,21.824312,Greece
+GS,-54.429579,-36.587909,South Georgia and the South Sandwich Islands
+GT,15.783471,-90.230759,Guatemala
+GU,13.444304,144.793731,Guam
+GW,11.803749,-15.180413,Guinea-Bissau
+GY,4.860416,-58.93018,Guyana
+GZ,31.354676,34.308825,Gaza Strip
+HK,22.396428,114.109497,Hong Kong
+HM,-53.08181,73.504158,Heard Island and McDonald Islands
+HN,15.199999,-86.241905,Honduras
+HR,45.1,15.2,Croatia
+HT,18.971187,-72.285215,Haiti
+HU,47.162494,19.503304,Hungary
+ID,-0.789275,113.921327,Indonesia
+IE,53.41291,-8.24389,Ireland
+IL,31.046051,34.851612,Israel
+IM,54.236107,-4.548056,Isle of Man
+IN,20.593684,78.96288,India
+IO,-6.343194,71.876519,British Indian Ocean Territory
+IQ,33.223191,43.679291,Iraq
+IR,32.427908,53.688046,Iran
+IS,64.963051,-19.020835,Iceland
+IT,41.87194,12.56738,Italy
+JE,49.214439,-2.13125,Jersey
+JM,18.109581,-77.297508,Jamaica
+JO,30.585164,36.238414,Jordan
+JP,36.204824,138.252924,Japan
+KE,-0.023559,37.906193,Kenya
+KG,41.20438,74.766098,Kyrgyzstan
+KH,12.565679,104.990963,Cambodia
+KI,-3.370417,-168.734039,Kiribati
+KM,-11.875001,43.872219,Comoros
+KN,17.357822,-62.782998,Saint Kitts and Nevis
+KP,40.339852,127.510093,North Korea
+KR,35.907757,127.766922,South Korea
+KW,29.31166,47.481766,Kuwait
+KY,19.513469,-80.566956,Cayman Islands
+KZ,48.019573,66.923684,Kazakhstan
+LA,19.85627,102.495496,Laos
+LB,33.854721,35.862285,Lebanon
+LC,13.909444,-60.978893,Saint Lucia
+LI,47.166,9.555373,Liechtenstein
+LK,7.873054,80.771797,Sri Lanka
+LR,6.428055,-9.429499,Liberia
+LS,-29.609988,28.233608,Lesotho
+LT,55.169438,23.881275,Lithuania
+LU,49.815273,6.129583,Luxembourg
+LV,56.879635,24.603189,Latvia
+LY,26.3351,17.228331,Libya
+MA,31.791702,-7.09262,Morocco
+MC,43.750298,7.412841,Monaco
+MD,47.411631,28.369885,Moldova
+ME,42.708678,19.37439,Montenegro
+MG,-18.766947,46.869107,Madagascar
+MH,7.131474,171.184478,Marshall Islands
+MK,41.608635,21.745275,Macedonia [FYROM]
+ML,17.570692,-3.996166,Mali
+MM,21.913965,95.956223,Myanmar [Burma]
+MN,46.862496,103.846656,Mongolia
+MO,22.198745,113.543873,Macau
+MP,17.33083,145.38469,Northern Mariana Islands
+MQ,14.641528,-61.024174,Martinique
+MR,21.00789,-10.940835,Mauritania
+MS,16.742498,-62.187366,Montserrat
+MT,35.937496,14.375416,Malta
+MU,-20.348404,57.552152,Mauritius
+MV,3.202778,73.22068,Maldives
+MW,-13.254308,34.301525,Malawi
+MX,23.634501,-102.552784,Mexico
+MY,4.210484,101.975766,Malaysia
+MZ,-18.665695,35.529562,Mozambique
+NA,-22.95764,18.49041,Namibia
+NC,-20.904305,165.618042,New Caledonia
+NE,17.607789,8.081666,Niger
+NF,-29.040835,167.954712,Norfolk Island
+NG,9.081999,8.675277,Nigeria
+NI,12.865416,-85.207229,Nicaragua
+NL,52.132633,5.291266,Netherlands
+NO,60.472024,8.468946,Norway
+NP,28.394857,84.124008,Nepal
+NR,-0.522778,166.931503,Nauru
+NU,-19.054445,-169.867233,Niue
+NZ,-40.900557,174.885971,New Zealand
+OM,21.512583,55.923255,Oman
+PA,8.537981,-80.782127,Panama
+PE,-9.189967,-75.015152,Peru
+PF,-17.679742,-149.406843,French Polynesia
+PG,-6.314993,143.95555,Papua New Guinea
+PH,12.879721,121.774017,Philippines
+PK,30.375321,69.345116,Pakistan
+PL,51.919438,19.145136,Poland
+PM,46.941936,-56.27111,Saint Pierre and Miquelon
+PN,-24.703615,-127.439308,Pitcairn Islands
+PR,18.220833,-66.590149,Puerto Rico
+PS,31.952162,35.233154,Palestinian Territories
+PT,39.399872,-8.224454,Portugal
+PW,7.51498,134.58252,Palau
+PY,-23.442503,-58.443832,Paraguay
+QA,25.354826,51.183884,Qatar
+RE,-21.115141,55.536384,Réunion
+RO,45.943161,24.96676,Romania
+RS,44.016521,21.005859,Serbia
+RU,61.52401,105.318756,Russia
+RW,-1.940278,29.873888,Rwanda
+SA,23.885942,45.079162,Saudi Arabia
+SB,-9.64571,160.156194,Solomon Islands
+SC,-4.679574,55.491977,Seychelles
+SD,12.862807,30.217636,Sudan
+SE,60.128161,18.643501,Sweden
+SG,1.352083,103.819836,Singapore
+SH,-24.143474,-10.030696,Saint Helena
+SI,46.151241,14.995463,Slovenia
+SJ,77.553604,23.670272,Svalbard and Jan Mayen
+SK,48.669026,19.699024,Slovakia
+SL,8.460555,-11.779889,Sierra Leone
+SM,43.94236,12.457777,San Marino
+SN,14.497401,-14.452362,Senegal
+SO,5.152149,46.199616,Somalia
+SR,3.919305,-56.027783,Suriname
+ST,0.18636,6.613081,São Tomé and Príncipe
+SV,13.794185,-88.89653,El Salvador
+SY,34.802075,38.996815,Syria
+SZ,-26.522503,31.465866,Swaziland
+TC,21.694025,-71.797928,Turks and Caicos Islands
+TD,15.454166,18.732207,Chad
+TF,-49.280366,69.348557,French Southern Territories
+TG,8.619543,0.824782,Togo
+TH,15.870032,100.992541,Thailand
+TJ,38.861034,71.276093,Tajikistan
+TK,-8.967363,-171.855881,Tokelau
+TL,-8.874217,125.727539,Timor-Leste
+TM,38.969719,59.556278,Turkmenistan
+TN,33.886917,9.537499,Tunisia
+TO,-21.178986,-175.198242,Tonga
+TR,38.963745,35.243322,Turkey
+TT,10.691803,-61.222503,Trinidad and Tobago
+TV,-7.109535,177.64933,Tuvalu
+TW,23.69781,120.960515,Taiwan
+TZ,-6.369028,34.888822,Tanzania
+UA,48.379433,31.16558,Ukraine
+UG,1.373333,32.290275,Uganda
+UM,U.S.,Minor,Outlying Islands
+US,37.09024,-95.712891,United States
+UY,-32.522779,-55.765835,Uruguay
+UZ,41.377491,64.585262,Uzbekistan
+VA,41.902916,12.453389,Vatican City
+VC,12.984305,-61.287228,Saint Vincent and the Grenadines
+VE,6.42375,-66.58973,Venezuela
+VG,18.420695,-64.639968,British Virgin Islands
+VI,18.335765,-64.896335,U.S. Virgin Islands
+VN,14.058324,108.277199,Vietnam
+VU,-15.376706,166.959158,Vanuatu
+WF,-13.768752,-177.156097,Wallis and Futuna
+WS,-13.759029,-172.104629,Samoa
+XK,42.602636,20.902977,Kosovo
+YE,15.552727,48.516388,Yemen
+YT,-12.8275,45.166244,Mayotte
+ZA,-30.559482,22.937506,South Africa
+ZM,-13.133897,27.849332,Zambia
+ZW,-19.015438,29.154857,Zimbabwe

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a8f4df3..cb83c87 100644
--- a/pom.xml
+++ b/pom.xml
@@ -62,6 +62,7 @@ limitations under the License.
     <commons-logging.version>1.1.3</commons-logging.version>
     <elasticsearch-java-driver.version>2.3.2</elasticsearch-java-driver.version>
     <elasticsearch5-java-driver.version>5.5.2</elasticsearch5-java-driver.version>
+    <esri-geometry-api.version>2.0.0</esri-geometry-api.version>
     <findbugs.version>3.0.1</findbugs.version>
     <fmpp-maven-plugin.version>1.0</fmpp-maven-plugin.version>
     <foodmart-data-hsqldb.version>0.3</foodmart-data-hsqldb.version>
@@ -216,6 +217,11 @@ limitations under the License.
         <version>${cassandra-driver-core.version}</version>
       </dependency>
       <dependency>
+        <groupId>com.esri.geometry</groupId>
+        <artifactId>esri-geometry-api</artifactId>
+        <version>${esri-geometry-api.version}</version>
+      </dependency>
+      <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-core</artifactId>
         <version>${jackson.version}</version>

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/site/_data/docs.yml
----------------------------------------------------------------------
diff --git a/site/_data/docs.yml b/site/_data/docs.yml
index 93e7154..ea38df9 100644
--- a/site/_data/docs.yml
+++ b/site/_data/docs.yml
@@ -24,6 +24,7 @@
 - title: Advanced
   docs:
   - adapter
+  - spatial
   - stream
   - lattice
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index cbe48a6..209d39a 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -507,6 +507,7 @@ FRAC_SECOND,
 G,
 GENERAL,
 GENERATED,
+GEOMETRY,
 **GET**,
 **GLOBAL**,
 GO,
@@ -979,6 +980,7 @@ name will have been converted to upper case also.
 | TIMESTAMP [ WITHOUT TIME ZONE ] | Date and time | Example: TIMESTAMP '1969-07-20 20:17:40'
 | TIMESTAMP WITH TIME ZONE | Date and time with time zone | Example: TIMESTAMP '1969-07-20 20:17:40 America/Los Angeles'
 | INTERVAL timeUnit [ TO timeUnit ] | Date time interval | Examples: INTERVAL '1-5' YEAR TO MONTH, INTERVAL '45' DAY, INTERVAL '1 2:34:56.789' DAY TO SECOND
+| GEOMETRY | Geometry | Examples: ST_GeomFromText('POINT (30 10)')
 
 Where:
 
@@ -992,6 +994,8 @@ Note:
 * DATE, TIME and TIMESTAMP have no time zone. There is not even an implicit
   time zone, such as UTC (as in Java) or the local time zone. It is left to
   the user or application to supply a time zone.
+* GEOMETRY is allowed only in certain
+  [conformance levels]({{ site.apiRoot }}/org/apache/calcite/sql/validate/SqlConformance.html#allowGeometry--).
 
 ### Non-scalar types
 
@@ -1004,6 +1008,32 @@ Note:
 | ARRAY    | Ordered, contiguous collection that may contain duplicates
 | CURSOR   | Cursor over the result of executing a query
 
+### Spatial types
+
+Spatial data is represented as character strings encoded as
+[well-known text (WKT)](https://en.wikipedia.org/wiki/Well-known_text)
+or binary strings encoded as
+[well-known binary (WKB)](https://en.wikipedia.org/wiki/Well-known_binary).
+
+Where you would use a literal, apply the `ST_GeomFromText` function,
+for example `ST_GeomFromText('POINT (30 10)')`.
+
+| Data type   | Type code | Examples in WKT
+|:----------- |:--------- |:---------------------
+| GEOMETRY           |  0 | generalization of Point, Curve, Surface, GEOMETRYCOLLECTION
+| POINT              |  1 | <tt>ST_GeomFromText(&#8203;'POINT (30 10)')</tt> is a point in 2D space; <tt>ST_GeomFromText(&#8203;'POINT Z(30 10 2)')</tt> is point in 3D space
+| CURVE            | 13 | generalization of LINESTRING
+| LINESTRING         |  2 | <tt>ST_GeomFromText(&#8203;'LINESTRING (30 10, 10 30, 40 40)')</tt>
+| SURFACE            | 14 | generalization of Polygon, PolyhedralSurface
+| POLYGON            |  3 | <tt>ST_GeomFromText(&#8203;'POLYGON ((30 10, 40 40, 20 40, 10 20, 30 10))')</tt> is a pentagon; <tt>ST_GeomFromText(&#8203;'POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))')</tt> is a pentagon with a quadrilateral hole
+| POLYHEDRALSURFACE  | 15 |
+| GEOMETRYCOLLECTION |  7 | a collection of zero or more GEOMETRY instances; a generalization of MULTIPOINT, MULTILINESTRING, MULTIPOLYGON
+| MULTIPOINT         |  4 | <tt>ST_GeomFromText(&#8203;'MULTIPOINT ((10 40), (40 30), (20 20), (30 10))')</tt> is equivalent to <tt>ST_GeomFromText(&#8203;'MULTIPOINT (10 40, 40 30, 20 20, 30 10)')</tt>
+| MULTICURVE         |  - | generalization of MULTILINESTRING
+| MULTILINESTRING    |  5 | <tt>ST_GeomFromText(&#8203;'MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10))')</tt>
+| MULTISURFACE       |  - | generalization of MULTIPOLYGON
+| MULTIPOLYGON       |  6 | <tt>ST_GeomFromText(`&#8203;'MULTIPOLYGON (((30 20, 45 40, 10 40, 30 20)), ((15 5, 40 10, 10 20, 5 10, 15 5)))')</tt>
+
 ## Operators and functions
 
 ### Operator precedence
@@ -1536,7 +1566,317 @@ by a grouped window function.
 | TUMBLE_END(expression, interval [, time ]) | Returns the value of *expression* at the end of the window defined by a `TUMBLE` function call
 | TUMBLE_START(expression, interval [, time ]) | Returns the value of *expression* at the beginning of the window defined by a `TUMBLE` function call
 
-### User-defined functions
+### Spatial functions
+
+In the following:
+
+* *geom* is a GEOMETRY;
+* *geomCollection* is a GEOMETRYCOLLECTION;
+* *point* is a POINT;
+* *lineString* is a LINESTRING;
+* *iMatrix* is a [DE-9IM intersection matrix](https://en.wikipedia.org/wiki/DE-9IM);
+* *distance*, *tolerance*, *segmentLengthFraction*, *offsetDistance* are of type double;
+* *dimension*, *quadSegs*, *srid*, *zoom* are of type integer;
+* *layerType* is a character string;
+* *gml* is a character string containing [Geography Markup Language (GML)](https://en.wikipedia.org/wiki/Geography_Markup_Language);
+* *wkt* is a character string containing [well-known text (WKT)](https://en.wikipedia.org/wiki/Well-known_text);
+* *wkb* is a binary string containing [well-known binary (WKB)](https://en.wikipedia.org/wiki/Well-known_binary).
+
+In the "C" (for "compatibility") column, "o" indicates that the function
+implements the OpenGIS Simple Features Implementation Specification for SQL,
+[version 1.2.1](http://www.opengeospatial.org/standards/sfs);
+"p" indicates that the function is a
+[PostGIS](http://www.postgis.net/docs/reference.html) extension to OpenGIS.
+
+#### Geometry conversion functions (2D)
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| p | ST_AsText(geom) | Alias for `ST_AsWKT`
+| o | ST_AsWKT(geom) | Converts *geom* → WKT
+| o | ST_GeomFromText(wkt [, srid ]) | Returns a specified GEOMETRY value from WKT representation
+| o | ST_LineFromText(wkt [, srid ]) | Converts WKT → LINESTRING
+| o | ST_MLineFromText(wkt [, srid ]) | Converts WKT → MULTILINESTRING
+| o | ST_MPointFromText(wkt [, srid ]) | Converts WKT → MULTIPOINT
+| o | ST_MPolyFromText(wkt [, srid ]) Converts WKT → MULTIPOLYGON
+| o | ST_PointFromText(wkt [, srid ]) | Converts WKT → POINT
+| o | ST_PolyFromText(wkt [, srid ]) | Converts WKT → POLYGON
+
+Not implemented:
+
+* ST_AsBinary(geom) GEOMETRY → WKB
+* ST_AsGML(geom) GEOMETRY → GML
+* ST_Force2D(geom) 3D GEOMETRY → 2D GEOMETRY
+* ST_GeomFromGML(gml [, srid ]) GML → GEOMETRY
+* ST_GeomFromWKB(wkb [, srid ]) WKB → GEOMETRY
+* ST_GoogleMapLink(geom [, layerType [, zoom ]]) GEOMETRY → Google map link
+* ST_LineFromWKB(wkb [, srid ]) WKB → LINESTRING
+* ST_OSMMapLink(geom [, marker ]) GEOMETRY → OSM map link
+* ST_PointFromWKB(wkb [, srid ]) WKB → POINT
+* ST_PolyFromWKB(wkb [, srid ]) WKB → POLYGON
+* ST_ToMultiLine(geom) Converts the coordinates of *geom* (which may be a GEOMETRYCOLLECTION) into a MULTILINESTRING
+* ST_ToMultiPoint(geom)) Converts the coordinates of *geom* (which may be a GEOMETRYCOLLECTION) into a MULTIPOINT
+* ST_ToMultiSegments(geom) Converts *geom* (which may be a GEOMETRYCOLLECTION) into a set of distinct segments stored in a MULTILINESTRING
+
+#### Geometry conversion functions (3D)
+
+Not implemented:
+
+* ST_Force3D(geom) 2D GEOMETRY → 3D GEOMETRY
+
+#### Geometry creation functions (2D)
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| o | ST_MakeLine(point1 [, point ]*) | Creates a line-string from the given POINTs (or MULTIPOINTs)
+| p | ST_MakePoint(x, y [, z ]) | Alias for `ST_Point`
+| o | ST_Point(x, y [, z ]) | Constructs a point from two or three coordinates
+
+Not implemented:
+
+* ST_BoundingCircle(geom) Returns the minimum bounding circle of *geom*
+* ST_Expand(geom, distance) Expands *geom*'s envelope
+* ST_Expand(geom, deltaX, deltaY) Expands *geom*'s envelope
+* ST_MakeEllipse(point, width, height) Constructs an ellipse
+* ST_MakeEnvelope(xMin, yMin, xMax, yMax  [, srid ]) Creates a rectangular POLYGON
+* ST_MakeGrid(geom, deltaX, deltaY) Calculates a regular grid of POLYGONs based on *geom*
+* ST_MakeGridPoints(geom, deltaX, deltaY) Calculates a regular grid of points based on *geom*
+* ST_MakePolygon(lineString [, hole ]*) Creates a POLYGON from *lineString* with the given holes (which are required to be closed LINESTRINGs)
+* ST_MinimumDiameter(geom) Returns the minimum diameter of *geom*
+* ST_MinimumRectangle(geom) Returns the minimum rectangle enclosing *geom*
+* ST_OctogonalEnvelope(geom) Returns the octogonal envelope of *geom*
+* ST_RingBuffer(geom, distance, bufferCount [, endCapStyle [, doDifference]]) Returns a MULTIPOLYGON of buffers centered at *geom* and of increasing buffer size
+
+### Geometry creation functions (3D)
+
+Not implemented:
+
+* ST_Extrude(geom, height [, flag]) Extrudes a GEOMETRY
+* ST_GeometryShadow(geom, point, height) Computes the shadow footprint of *geom*
+* ST_GeometryShadow(geom, azimuth, altitude, height [, unify ]) Computes the shadow footprint of *geom*
+
+#### Geometry properties (2D)
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| o | ST_Boundary(geom [, srid ]) | Returns the boundary of *geom*
+| o | ST_Distance(geom1, geom2) | Returns the distance between *geom1* and *geom2*
+| o | ST_GeometryType(geom) | Returns the type of *geom*
+| o | ST_GeometryTypeCode(geom) | Returns the OGC SFS type code of *geom*
+| o | ST_Envelope(geom [, srid ]) | Returns the envelope of *geom* (which may be a GEOMETRYCOLLECTION) as a GEOMETRY
+
+Not implemented:
+
+* ST_Centroid(geom) Returns the centroid of *geom* (which may be a GEOMETRYCOLLECTION)
+* ST_CompactnessRatio(polygon) Returns the square root of *polygon*'s area divided by the area of the circle with circumference equal to its perimeter
+* ST_CoordDim(geom) Returns the dimension of the coordinates of *geom*
+* ST_Dimension(geom) Returns the dimension of *geom*
+* ST_EndPoint(lineString) Returns the last coordinate of *lineString*
+* ST_Envelope(geom [, srid ]) Returns the envelope of *geom* (which may be a GEOMETRYCOLLECTION) as a GEOMETRY
+* ST_Explode(query [, fieldName]) Explodes the GEOMETRYCOLLECTIONs in the *fieldName* column of a query into multiple geometries
+* ST_Extent(geom) Returns the minimum bounding box of *geom* (which may be a GEOMETRYCOLLECTION)
+* ST_ExteriorRing(polygon) Returns the exterior ring of *polygon* as a linear-ring
+* ST_GeometryN(geomCollection, n) Returns the *n*th GEOMETRY of *geomCollection*
+* ST_InteriorRingN(polygon, n) Returns the *n*th interior ring of *polygon*
+* ST_IsClosed(geom) Returns whether *geom* is a closed LINESTRING or MULTILINESTRING
+* ST_IsEmpty(geom) Returns whether *geom* is empty
+* ST_IsRectangle(geom) Returns whether *geom* is a rectangle
+* ST_IsRing(geom) Returns whether *geom* is a closed and simple line-string or MULTILINESTRING
+* ST_IsSimple(geom) Returns whether *geom* is simple
+* ST_IsValid(geom) Returns whether *geom* is valid
+* ST_IsValidDetail(geom [, selfTouchValid ]) Returns a valid detail as an array of objects
+* ST_IsValidReason(geom [, selfTouchValid ]) Returns text stating whether *geom* is valid, and if not valid, a reason why
+* ST_NPoints(geom) Returns the number of points in *geom*
+* ST_NumGeometries(geom) Returns the number of geometries in *geom* (1 if it is not a GEOMETRYCOLLECTION)
+* ST_NumInteriorRing(geom) Alias for `ST_NumInteriorRings`
+* ST_NumInteriorRings(geom) Returns the number of interior rings of *geom*
+* ST_NumPoints(lineString) Returns the number of points in *lineString*
+* ST_PointN(geom, n) Returns the *n*th point of a *lineString*
+* ST_PointOnSurface(geom) Returns an interior or boundary point of *geom*
+* ST_SRID(geom) Returns SRID value of *geom* or 0 if it does not have one
+* ST_StartPoint(lineString) Returns the first coordinate of *lineString*
+* ST_X(geom) Returns the x-value of the first coordinate of *geom*
+* ST_XMax(geom) Returns the maximum x-value of *geom*
+* ST_XMin(geom) Returns the minimum x-value of *geom*
+* ST_Y(geom) Returns the y-value of the first coordinate of *geom*
+* ST_YMax(geom) Returns the maximum y-value of *geom*
+* ST_YMin(geom) Returns the minimum y-value of *geom*
+
+#### Geometry properties (3D)
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| p | ST_Is3D(s) | Returns whether *geom* has at least one z-coordinate
+| o | ST_Z(geom) | Returns the z-value of the first coordinate of *geom*
+
+Not implemented:
+
+* ST_ZMax(geom) Returns the maximum z-value of *geom*
+* ST_ZMin(geom) Returns the minimum z-value of *geom*
+
+### Geometry predicates
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| o | ST_Contains(geom1, geom2) | Returns whether *geom1* contains *geom2*
+| p | ST_ContainsProperly(geom1, geom2) | Returns whether *geom1* contains *geom2* but does not intersect its boundary
+| o | ST_Crosses(geom1, geom2) | Returns whether *geom1* crosses *geom2*
+| o | ST_Disjoint(geom1, geom2) | Returns whether *geom1* and *geom2* are disjoint
+| p | ST_DWithin(geom1, geom2, distance) | Returns whether *geom1* and *geom* are within *distance* of one another
+| o | ST_EnvelopesIntersect(geom1, geom2) | Returns whether the envelope of *geom1* intersects the envelope of *geom2*
+| o | ST_Equals(geom1, geom2) | Returns whether *geom1* equals *geom2*
+| o | ST_Intersects(geom1, geom2) | Returns whether *geom1* intersects *geom2*
+| o | ST_Overlaps(geom1, geom2) | Returns whether *geom1* overlaps *geom2*
+| o | ST_Touches(geom1, geom2) | Returns whether *geom1* touches *geom2*
+| o | ST_Within(geom1, geom2) | Returns whether *geom1* is within *geom2*
+
+Not implemented:
+
+* ST_Covers(geom1, geom2) Returns whether no point in *geom2* is outside *geom1*
+* ST_OrderingEquals(geom1, geom2) Returns whether *geom1* equals *geom2* and their coordinates and component Geometries are listed in the same order
+* ST_Relate(geom1, geom2) Returns the DE-9IM intersection matrix of *geom1* and *geom2*
+* ST_Relate(geom1, geom2, iMatrix) Returns whether *geom1* and *geom2* are related by the given intersection matrix *iMatrix*
+
+#### Geometry operators (2D)
+
+The following functions combine 2D geometries.
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| o | ST_Buffer(geom, distance [, quadSegs \| style ]) | Computes a buffer around *geom*
+| o | ST_Union(geom1, geom2) | Computes the union of *geom1* and *geom2*
+| o | ST_Union(geomCollection) | Computes the union of the geometries in *geomCollection*
+
+See also: the `ST_Union` aggregate function.
+
+Not implemented:
+
+* ST_ConvexHull(geom) Computes the smallest convex polygon that contains all the points in *geom*
+* ST_Difference(geom1, geom2) Computes the difference between two geometries
+* ST_Intersection(geom1, geom2) Computes the intersection of two geometries
+* ST_SymDifference(geom1, geom2) Computes the symmetric difference between two geometries
+
+#### Affine transformation functions (3D and 2D)
+
+Not implemented:
+
+* ST_Rotate(geom, angle [, origin \| x, y]) Rotates a *geom* counter-clockwise by *angle* (in radians) about *origin* (or the point (*x*, *y*))
+* ST_Scale(geom, xFactor, yFactor [, zFactor ]) Scales *geom* by multiplying the ordinates by the indicated scale factors
+* ST_Translate(geom, x, y, [, z]) Translates *geom*
+
+#### Geometry editing functions (2D)
+
+The following functions modify 2D geometries.
+
+Not implemented:
+
+* ST_AddPoint(geom, point [, tolerance ]) Adds *point* to *geom* with a given *tolerance* (default 0)
+* ST_CollectionExtract(geom, dimension) Filters *geom*, returning a multi-geometry of those members with a given *dimension* (1 = point, 2 = line-string, 3 = polygon)
+* ST_Densify(geom, tolerance) Inserts extra vertices every *tolerance* along the line segments of *geom*
+* ST_FlipCoordinates(geom) Flips the X and Y coordinates of *geom*
+* ST_Holes(geom) Returns the holes in *geom* (which may be a GEOMETRYCOLLECTION)
+* ST_Normalize(geom) Converts *geom* to normal form
+* ST_RemoveDuplicatedCoordinates(geom) Removes duplicated coordinates from *geom*
+* ST_RemoveHoles(geom) Removes a *geom*'s holes
+* ST_RemovePoints(geom, poly) Removes all coordinates of *geom* located within *poly*; null if all coordinates are removed
+* ST_RemoveRepeatedPoints(geom, tolerance) Removes from *geom* all repeated points (or points within *tolerance* of another point)
+* ST_Reverse(geom) Reverses the vertex order of *geom*
+
+#### Geometry editing functions (3D)
+
+The following functions modify 3D geometries.
+
+Not implemented:
+
+* ST_AddZ(geom, zToAdd) Adds *zToAdd* to the z-coordinate of *geom*
+* ST_Interpolate3DLine(geom) Returns *geom* with a interpolation of z values, or null if it is not a line-string or MULTILINESTRING
+* ST_MultiplyZ(geom, zFactor) Returns *geom* with its z-values multiplied by *zFactor*
+* ST_Reverse3DLine(geom [, sortOrder ]) Potentially reverses *geom* according to the z-values of its first and last coordinates
+* ST_UpdateZ(geom, newZ [, updateCondition ]) Updates the z-values of *geom*
+* ST_ZUpdateLineExtremities(geom, startZ, endZ [, interpolate ]) Updates the start and end z-values of *geom*
+
+#### Geometry measurement functions (2D)
+
+Not implemented:
+
+* ST_Area(geom) Returns the area of *geom* (which may be a GEOMETRYCOLLECTION)
+* ST_ClosestCoordinate(geom, point) Returns the coordinate(s) of *geom* closest to *point*
+* ST_ClosestPoint(geom1, geom2) Returns the point of *geom1* closest to *geom2*
+* ST_FurthestCoordinate(geom, point) Returns the coordinate(s) of *geom* that are furthest from *point*
+* ST_Length(lineString) Returns the length of *lineString*
+* ST_LocateAlong(geom, segmentLengthFraction, offsetDistance) Returns a MULTIPOINT containing points along the line segments of *geom* at *segmentLengthFraction* and *offsetDistance*
+* ST_LongestLine(geom1, geom2) Returns the 2-dimensional longest line-string between the points of *geom1* and *geom2*
+* ST_MaxDistance(geom1, geom2) Computes the maximum distance between *geom1* and *geom2*
+* ST_Perimeter(polygon) Returns the length of the perimeter of *polygon* (which may be a MULTIPOLYGON)
+* ST_ProjectPoint(point, lineString) Projects *point* onto a *lineString* (which may be a MULTILINESTRING)
+
+#### Geometry measurement functions (3D)
+
+Not implemented:
+
+* ST_3DArea(geom) Return a polygon's 3D area
+* ST_3DLength(geom) Returns the 3D length of a line-string
+* ST_3DPerimeter(geom) Returns the 3D perimeter of a polygon or MULTIPOLYGON
+* ST_SunPosition(point [, timestamp ]) Computes the sun position at *point* and *timestamp* (now by default)
+
+#### Geometry processing functions (2D)
+
+The following functions process geometries.
+
+Not implemented:
+
+* ST_LineIntersector(geom1, geom2) Splits *geom1* (a line-string) with *geom2*
+* ST_LineMerge(geom) Merges a collection of linear components to form a line-string of maximal length
+* ST_MakeValid(geom [, preserveGeomDim [, preserveDuplicateCoord [, preserveCoordDim]]]) Makes *geom* valid
+* ST_Polygonize(geom) Creates a MULTIPOLYGON from edges of *geom*
+* ST_PrecisionReducer(geom, n) Reduces *geom*'s precision to *n* decimal places
+* ST_RingSideBuffer(geom, distance, bufferCount [, endCapStyle [, doDifference]]) Computes a ring buffer on one side
+* ST_SideBuffer(geom, distance [, bufferStyle ]) Compute a single buffer on one side
+* ST_Simplify(geom, distance) Simplifies *geom* using the [Douglas-Peuker algorithm](https://en.wikipedia.org/wiki/Ramer%E2%80%93Douglas%E2%80%93Peucker_algorithm) with a *distance* tolerance
+* ST_SimplifyPreserveTopology(geom) Simplifies *geom*, preserving its topology
+* ST_Snap(geom1, geom2, tolerance) Snaps *geom1* and *geom2* together
+* ST_Split(geom1, geom2 [, tolerance]) Splits *geom1* by *geom2* using *tolerance* (default 1E-6) to determine where the point splits the line
+
+#### Geometry projection functions
+
+| C | Operator syntax      | Description
+|:- |:-------------------- |:-----------
+| o | ST_SetSRID(geom, srid) | Returns a copy of *geom* with a new SRID
+| o | ST_Transform(geom, srid) | Transforms *geom* from one coordinate reference system (CRS) to the CRS specified by *srid*
+
+#### Trigonometry functions
+
+Not implemented:
+
+* ST_Azimuth(point1, point2) Return the azimuth of the segment from *point1* to *point2*
+
+#### Topography functions
+
+Not implemented:
+
+* ST_TriangleAspect(geom) Returns the aspect of a triangle
+* ST_TriangleContouring(query \[, z1, z2, z3 ]\[, varArgs]*) Splits triangles into smaller triangles according to classes
+* ST_TriangleDirection(geom) Computes the direction of steepest ascent of a triangle and returns it as a line-string
+* ST_TriangleSlope(geom) Computes the slope of a triangle as a percentage
+* ST_Voronoi(geom [, outDimension [, envelopePolygon ]]) Creates a Voronoi diagram
+
+#### Triangulation functions
+
+Not implemented:
+
+* ST_ConstrainedDelaunay(geom [, flag [, quality ]]) Computes a constrained Delaunay triangulation based on *geom*
+* ST_Delaunay(geom [, flag [, quality ]]) Computes a Delaunay triangulation based on points
+* ST_Tessellate(polygon) Tessellates *polygon* (may be MULTIPOLYGON) with adaptive triangles
+
+#### Geometry aggregate functions
+
+Not implemented:
+
+* ST_Accum(geom) Accumulates *geom* into a GEOMETRYCOLLECTION (or MULTIPOINT, MULTILINESTRING or MULTIPOLYGON if possible)
+* ST_Collect(geom) Alias for `ST_Accum`
+* ST_Union(geom) Computes the union of geometries
+
+## User-defined functions
 
 Calcite is extensible. You can define each kind of function using user code.
 For each kind of function there are often several ways to define a function,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/site/_docs/spatial.md
----------------------------------------------------------------------
diff --git a/site/_docs/spatial.md b/site/_docs/spatial.md
new file mode 100644
index 0000000..3b98464
--- /dev/null
+++ b/site/_docs/spatial.md
@@ -0,0 +1,63 @@
+---
+layout: docs
+title: Spatial
+permalink: /docs/spatial.html
+---
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+Calcite is [aiming](https://issues.apache.org/jira/browse/CALCITE-1968) to implement
+OpenGIS Simple Features Implementation Specification for SQL,
+[version 1.2.1](http://www.opengeospatial.org/standards/sfs),
+a standard implemented by spatial databases such as
+[PostGIS](http://postgis.net/)
+and [H2GIS](http://www.h2gis.org/).
+
+We also aim to add optimizer support for
+[spatial indexes](https://issues.apache.org/jira/browse/CALCITE-1861)
+and other forms of query optimization.
+
+* TOC
+{:toc}
+
+## Introduction
+
+A spatial database is a database that is optimized for storing and query data
+that represents objects defined in a geometric space.
+
+Calcite's support for spatial data includes:
+
+* A [GEOMETRY](reference.html#data-types) data type and
+  [sub-types](reference.html#spatial-types) including `POINT`, `LINESTRING`
+  and `POLYGON`
+* [Spatial functions](reference.html#spatial-functions) (prefixed `ST_`;
+  we have implemented about 35 of the 150 in the OpenGIS specification)
+
+and will at some point also include query rewrites to use spatial indexes.
+
+## Acknowledgements
+
+Calcite's OpenGIS implementation uses the
+[Esri geometry API](https://github.com/Esri/geometry-api-java). Thanks for the
+help we received from their community.
+
+While developing this feature, we made extensive use of the
+PostGIS documentation and tests,
+and the H2GIS documentation, and consulted both as reference implementations
+when the specification wasn't clear. Thank you to these awesome projects.

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/src/main/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/suppressions.xml b/src/main/config/checkstyle/suppressions.xml
index e8e839e..2f55a52 100644
--- a/src/main/config/checkstyle/suppressions.xml
+++ b/src/main/config/checkstyle/suppressions.xml
@@ -38,6 +38,9 @@ limitations under the License.
   <!-- Don't complain about field names such as cust_id -->
   <suppress checks=".*Name" files="JdbcExample.java"/>
 
+  <!-- Don't complain about method names in a class full of UDFs -->
+  <suppress checks="MethodName" files="GeoFunctions.java"/>
+
   <!-- Suppress JavadocPackage in the test packages -->
   <suppress checks="JavadocPackage" files="src[/\\]test[/\\]java[/\\]"/>
 


[08/16] calcite git commit: [CALCITE-1969] Annotate user-defined functions as strict and semi-strict

Posted by mm...@apache.org.
[CALCITE-1969] Annotate user-defined functions as strict and semi-strict

Also add an "Experimental" annotation.


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

Branch: refs/heads/branch-1.14
Commit: b2bf1ca0a40762b904f7334c92e851b21f6b2844
Parents: 4208d80
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Aug 25 16:17:53 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:15:18 2017 -0700

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/NullPolicy.java  |  2 +
 .../calcite/adapter/enumerable/RexImpTable.java | 10 ++--
 .../calcite/prepare/CalciteCatalogReader.java   |  9 +++-
 .../calcite/schema/impl/ScalarFunctionImpl.java | 44 ++++++++++++++--
 .../sql/type/SqlTypeExplicitPrecedenceList.java |  4 +-
 .../java/org/apache/calcite/test/UdfTest.java   | 53 ++++++++++++++++++++
 .../java/org/apache/calcite/util/Smalls.java    | 24 +++++++++
 .../calcite/linq4j/function/Experimental.java   | 42 ++++++++++++++++
 .../calcite/linq4j/function/SemiStrict.java     | 43 ++++++++++++++++
 .../apache/calcite/linq4j/function/Strict.java  | 39 ++++++++++++++
 10 files changed, 261 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
index dedf300..43c3431 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
@@ -27,6 +27,8 @@ package org.apache.calcite.adapter.enumerable;
 public enum NullPolicy {
   /** Returns null if and only if one of the arguments are null. */
   STRICT,
+  /** Returns null if one of the arguments is null, and possibly other times. */
+  SEMI_STRICT,
   /** If any of the arguments are null, return null. */
   ANY,
   /** If any of the arguments are false, result is false; else if any

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 00e9764..17cb540 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -475,6 +475,7 @@ public class RexImpTable {
     switch (nullPolicy) {
     case ANY:
     case STRICT:
+    case SEMI_STRICT:
       return new CallImplementor() {
         public Expression implement(
             RexToLixTranslator translator, RexCall call, NullAs nullAs) {
@@ -812,7 +813,8 @@ public class RexImpTable {
     case IS_NOT_NULL:
       // If "f" is strict, then "f(a0, a1) IS NOT NULL" is
       // equivalent to "a0 IS NOT NULL AND a1 IS NOT NULL".
-      if (nullPolicy == NullPolicy.STRICT) {
+      switch (nullPolicy) {
+      case STRICT:
         return Expressions.foldAnd(
             translator.translateList(
                 call.getOperands(), nullAs));
@@ -821,7 +823,8 @@ public class RexImpTable {
     case IS_NULL:
       // If "f" is strict, then "f(a0, a1) IS NULL" is
       // equivalent to "a0 IS NULL OR a1 IS NULL".
-      if (nullPolicy == NullPolicy.STRICT) {
+      switch (nullPolicy) {
+      case STRICT:
         return Expressions.foldOr(
             translator.translateList(
                 call.getOperands(), nullAs));
@@ -904,7 +907,8 @@ public class RexImpTable {
       // RexNode can be referred via multiple ways: RexNode itself, RexLocalRef,
       // and may be others.
       final Map<RexNode, Boolean> nullable = new HashMap<>();
-      if (nullPolicy == NullPolicy.STRICT) {
+      switch (nullPolicy) {
+      case STRICT:
         // The arguments should be not nullable if STRICT operator is computed
         // in nulls NOT_POSSIBLE mode
         for (RexNode arg : call.getOperands()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index ab1bbe4..b7f32e1 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -33,6 +33,7 @@ import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.Wrapper;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlOperator;
@@ -354,7 +355,13 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
   private SqlReturnTypeInference infer(final ScalarFunction function) {
     return new SqlReturnTypeInference() {
       public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-        final RelDataType type = function.getReturnType(typeFactory);
+        final RelDataType type;
+        if (function instanceof ScalarFunctionImpl) {
+          type = ((ScalarFunctionImpl) function).getReturnType(typeFactory,
+              opBinding);
+        } else {
+          type = function.getReturnType(typeFactory);
+        }
         return toSql(type);
       }
     };

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
index 18f398a..2429e0c 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
@@ -20,10 +20,13 @@ import org.apache.calcite.adapter.enumerable.CallImplementor;
 import org.apache.calcite.adapter.enumerable.NullPolicy;
 import org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
 import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.linq4j.function.SemiStrict;
+import org.apache.calcite.linq4j.function.Strict;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.ImplementableFunction;
 import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 import com.google.common.collect.ImmutableMultimap;
 
@@ -35,8 +38,8 @@ import static org.apache.calcite.util.Static.RESOURCE;
 /**
 * Implementation of {@link org.apache.calcite.schema.ScalarFunction}.
 */
-public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
-    ScalarFunction, ImplementableFunction {
+public class ScalarFunctionImpl extends ReflectiveFunctionBase
+    implements ScalarFunction, ImplementableFunction {
   private final CallImplementor implementor;
 
   /** Private constructor. */
@@ -112,8 +115,43 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
   }
 
   private static CallImplementor createImplementor(final Method method) {
+    final NullPolicy nullPolicy = getNullPolicy(method);
     return RexImpTable.createImplementor(
-        new ReflectiveCallNotNullImplementor(method), NullPolicy.NONE, false);
+        new ReflectiveCallNotNullImplementor(method), nullPolicy, false);
+  }
+
+  private static NullPolicy getNullPolicy(Method m) {
+    if (m.getAnnotation(Strict.class) != null) {
+      return NullPolicy.STRICT;
+    } else if (m.getAnnotation(SemiStrict.class) != null) {
+      return NullPolicy.SEMI_STRICT;
+    } else if (m.getDeclaringClass().getAnnotation(Strict.class) != null) {
+      return NullPolicy.STRICT;
+    } else if (m.getDeclaringClass().getAnnotation(SemiStrict.class) != null) {
+      return NullPolicy.SEMI_STRICT;
+    } else {
+      return NullPolicy.NONE;
+    }
+  }
+
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory,
+      SqlOperatorBinding opBinding) {
+    // Strict and semi-strict functions can return null even if their Java
+    // functions return a primitive type. Because when one of their arguments
+    // is null, they won't even be called.
+    final RelDataType returnType = getReturnType(typeFactory);
+    switch (getNullPolicy(method)) {
+    case STRICT:
+      for (RelDataType type : opBinding.collectOperandTypes()) {
+        if (type.isNullable()) {
+          return typeFactory.createTypeWithNullability(returnType, true);
+        }
+      }
+      break;
+    case SEMI_STRICT:
+      return typeFactory.createTypeWithNullability(returnType, true);
+    }
+    return returnType;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
index 7599bb7..8dcdc8b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
@@ -151,8 +151,8 @@ public class SqlTypeExplicitPrecedenceList
 
   // implement RelDataTypePrecedenceList
   public int compareTypePrecedence(RelDataType type1, RelDataType type2) {
-    assert containsType(type1);
-    assert containsType(type2);
+    assert containsType(type1) : type1;
+    assert containsType(type2) : type2;
 
     int p1 =
         getListPosition(

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/test/java/org/apache/calcite/test/UdfTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/UdfTest.java b/core/src/test/java/org/apache/calcite/test/UdfTest.java
index 2cdc449..6b5b5e3 100644
--- a/core/src/test/java/org/apache/calcite/test/UdfTest.java
+++ b/core/src/test/java/org/apache/calcite/test/UdfTest.java
@@ -22,6 +22,7 @@ import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
 import org.apache.calcite.adapter.java.ReflectiveSchema;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.SemiStrict;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.Types;
@@ -149,6 +150,18 @@ public class UdfTest {
         + "           methodName: 'abs'\n"
         + "         },\n"
         + "         {\n"
+        + "           name: 'NULL4',\n"
+        + "           className: '"
+        + Smalls.Null4Function.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'NULL8',\n"
+        + "           className: '"
+        + Smalls.Null8Function.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
         + "           className: '"
         + Smalls.MultipleFunction.class.getName()
         + "',\n"
@@ -291,6 +304,46 @@ public class UdfTest {
         .returns("");
   }
 
+  /** Tests that we generate the appropriate checks for a "semi-strict"
+   * function.
+   *
+   * <p>The difference between "strict" and "semi-strict" functions is that a
+   * "semi-strict" function might return null even if none of its arguments
+   * are null. (Both always return null if one of their arguments is null.)
+   * Thus, a nasty function is more unpredictable.
+   *
+   * @see SemiStrict */
+  @Test public void testSemiStrict() {
+    final CalciteAssert.AssertThat with = withUdf();
+    final String sql = "select\n"
+        + "  \"adhoc\".null4(upper(\"name\")) as p\n"
+        + " from \"adhoc\".EMPLOYEES";
+    with.query(sql)
+        .returnsUnordered("P=null",
+            "P=null",
+            "P=SEBASTIAN",
+            "P=THEODORE");
+    // my_str is non-strict; it must be called when args are null
+    final String sql2 = "select\n"
+        + "  \"adhoc\".my_str(upper(\"adhoc\".null4(\"name\"))) as p\n"
+        + " from \"adhoc\".EMPLOYEES";
+    with.query(sql2)
+        .returnsUnordered("P=<null>",
+            "P=<null>",
+            "P=<SEBASTIAN>",
+            "P=<THEODORE>");
+    // null8 throws NPE if its argument is null,
+    // so we had better know that null4 might return null
+    final String sql3 = "select\n"
+        + "  \"adhoc\".null8(\"adhoc\".null4(\"name\")) as p\n"
+        + " from \"adhoc\".EMPLOYEES";
+    with.query(sql3)
+        .returnsUnordered("P=null",
+            "P=null",
+            "P=Sebastian",
+            "P=null");
+  }
+
   /** Tests derived return type of user-defined function. */
   @Test public void testUdfDerivedReturnType() {
     final CalciteAssert.AssertThat with = withUdf();

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index 39f0263..6ea55bf 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -30,6 +30,7 @@ import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.function.Parameter;
+import org.apache.calcite.linq4j.function.SemiStrict;
 import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -450,6 +451,29 @@ public class Smalls {
     }
   }
 
+  /** Example of a semi-strict UDF.
+   * (Returns null if its parameter is null or if its length is 4.) */
+  public static class Null4Function {
+    @SemiStrict public static String eval(@Parameter(name = "s") String s) {
+      if (s == null || s.length() == 4) {
+        return null;
+      }
+      return s;
+    }
+  }
+
+  /** Example of a picky, semi-strict UDF.
+   * Throws {@link NullPointerException} if argument is null.
+   * Returns null if its argument's length is 8. */
+  public static class Null8Function {
+    @SemiStrict public static String eval(@Parameter(name = "s") String s) {
+      if (s.length() == 8) {
+        return null;
+      }
+      return s;
+    }
+  }
+
   /** Example of a UDF with a static {@code eval} method. Class is abstract,
    * but code-generator should not need to instantiate it. */
   public abstract static class MyDoubleFunction {

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java
new file mode 100644
index 0000000..630384d
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java
@@ -0,0 +1,42 @@
+/*
+ * 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.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.PACKAGE;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation that indicates that a class, interface, field or method
+ * is experimental, not part of the public API, and subject to change
+ * or removal.
+ *
+ * <p>And yes, it is flagged experimental. We may move it elsewhere in future,
+ * when we re-think the maturity model.
+ */
+@Target({PACKAGE, TYPE, FIELD, METHOD })
+@Retention(RetentionPolicy.SOURCE)
+@Experimental
+public @interface Experimental {
+}
+
+// End Experimental.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java
new file mode 100644
index 0000000..a9d2dce
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java
@@ -0,0 +1,43 @@
+/*
+ * 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.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation applied to a user-defined function that indicates that
+ * the function always returns null if one or more of its arguments
+ * are null but also may return null at other times.
+ *
+ * <p>Compare with {@link Strict}:
+ * <ul>
+ *   <li>A strict function returns null if and only if it has a null argument
+ *   <li>A semi-strict function returns null if it has a null argument
+ * </ul>
+ */
+@Target({METHOD, TYPE })
+@Retention(RetentionPolicy.RUNTIME)
+@Experimental
+public @interface SemiStrict {
+}
+
+// End SemiStrict.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java
new file mode 100644
index 0000000..5bddb0a
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java
@@ -0,0 +1,39 @@
+/*
+ * 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.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation applied to a user-defined function that indicates that
+ * the function returns null if and only if one or more of its arguments
+ * are null.
+ *
+ * @see SemiStrict
+ */
+@Target({METHOD, TYPE })
+@Retention(RetentionPolicy.RUNTIME)
+@Experimental
+public @interface Strict {
+}
+
+// End Strict.java


[14/16] calcite git commit: [CALCITE-1069] In Aggregate, deprecate indicators, and allow GROUPING to be used as an aggregate function

Posted by mm...@apache.org.
[CALCITE-1069] In Aggregate, deprecate indicators, and allow GROUPING to be used as an aggregate function

Deprecate the Aggregate.indicator field (strongly encouraging people
to set it to false) and to allow GROUPING (and its synonym, GROUP_ID)
to be used as an aggregate function. It will be handled at implement
time.

With indicator = false, even queries with more than one grouping set
will just output the join keys and the aggregate functions. A join key
will be nullable if it doesn't appear in all grouping sets.

The output row type of Aggregate will be more consistent, and this will
have benefits such as fewer bugs in rules.

Add RelBuilder.rename().

Remove SqlValidatorNamespace.translate() and
SqlQualified.suffixTranslated().

Mute deprecation warnings.

Remove MutableAggregate.indicator.

Close apache/calcite#470


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

Branch: refs/heads/branch-1.14
Commit: 1e7ae1c30a5e6f4335f2e4cc0059f2f2901e35fd
Parents: ddae841
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Mar 22 12:33:46 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 21:01:50 2017 -0700

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/AggContext.java  |  19 +
 .../adapter/enumerable/AggResultContext.java    |  14 +
 .../adapter/enumerable/EnumerableAggregate.java | 108 +++--
 .../adapter/enumerable/EnumerableWindow.java    |  17 +
 .../calcite/adapter/enumerable/RexImpTable.java |  60 +++
 .../enumerable/impl/AggAddContextImpl.java      |   2 +-
 .../enumerable/impl/AggResetContextImpl.java    |  14 +-
 .../enumerable/impl/AggResultContextImpl.java   |  36 +-
 .../impl/WinAggResultContextImpl.java           |   2 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  15 +-
 .../calcite/plan/SubstitutionVisitor.java       |   5 +-
 .../org/apache/calcite/rel/core/Aggregate.java  |  30 +-
 .../apache/calcite/rel/core/RelFactories.java   |   1 +
 .../calcite/rel/logical/LogicalAggregate.java   |  15 +
 .../calcite/rel/mutable/MutableAggregate.java   |  19 +-
 .../apache/calcite/rel/mutable/MutableRels.java |   7 +-
 .../rel/rules/AbstractMaterializedViewRule.java |   4 +-
 .../AggregateExpandDistinctAggregatesRule.java  | 149 +++---
 .../rel/rules/AggregateFilterTransposeRule.java |   4 +-
 .../rel/rules/AggregateJoinTransposeRule.java   |   4 +-
 .../rel/rules/AggregateProjectMergeRule.java    |   2 +-
 .../AggregateProjectPullUpConstantsRule.java    |   4 +-
 .../rel/rules/AggregateReduceFunctionsRule.java |   1 -
 .../rel/rules/AggregateUnionAggregateRule.java  |   2 +-
 .../rel/rules/AggregateUnionTransposeRule.java  |   4 +-
 .../rel/rules/FilterAggregateTransposeRule.java |   2 +-
 .../rel/rules/IntersectToDistinctRule.java      |   2 +-
 .../apache/calcite/rel/stream/StreamRules.java  |   7 +-
 .../apache/calcite/runtime/CalciteResource.java |   3 -
 .../org/apache/calcite/sql/SqlAggFunction.java  |   6 +
 .../sql/fun/SqlAbstractGroupFunction.java       |  16 +-
 .../calcite/sql/fun/SqlGroupIdFunction.java     |   5 +-
 .../calcite/sql/fun/SqlGroupingFunction.java    |   7 +
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  16 +-
 .../apache/calcite/sql/validate/AggFinder.java  |   5 +-
 .../sql/validate/AggregatingSelectScope.java    |   2 -
 .../sql/validate/DelegatingNamespace.java       |   4 -
 .../sql/validate/IdentifierNamespace.java       |   4 -
 .../calcite/sql/validate/SqlQualified.java      |  33 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  |  11 +-
 .../sql/validate/SqlValidatorNamespace.java     |   5 -
 .../apache/calcite/sql2rel/RelDecorrelator.java |  12 +-
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |   4 +-
 .../calcite/sql2rel/SqlToRelConverter.java      | 151 ++----
 .../org/apache/calcite/tools/RelBuilder.java    |  87 +++-
 .../calcite/runtime/CalciteResource.properties  |   1 -
 .../org/apache/calcite/plan/RelWriterTest.java  |   3 +-
 .../org/apache/calcite/test/RelBuilderTest.java | 114 ++++-
 .../apache/calcite/test/RelMetadataTest.java    |   4 +-
 .../apache/calcite/test/SqlValidatorTest.java   |  45 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml | 455 +++++++++----------
 .../calcite/test/SqlToRelConverterTest.xml      |  91 ++--
 core/src/test/resources/sql/agg.iq              | 295 +++++++++---
 53 files changed, 1144 insertions(+), 784 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
index 542e3a2..70e464a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
@@ -18,6 +18,7 @@ package org.apache.calcite.adapter.enumerable;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -67,6 +68,24 @@ public interface AggContext {
    * @return Parameter types of the aggregate
    */
   List<? extends Type> parameterTypes();
+
+  /** Returns the ordinals of the input fields that make up the key. */
+  List<Integer> keyOrdinals();
+
+  /**
+   * Returns the types of the group key as
+   * {@link org.apache.calcite.rel.type.RelDataType}.
+   */
+  List<? extends RelDataType> keyRelTypes();
+
+  /**
+   * Returns the types of the group key as
+   * {@link java.lang.reflect.Type}.
+   */
+  List<? extends Type> keyTypes();
+
+  /** Returns the grouping sets we are aggregating on. */
+  List<ImmutableBitSet> groupSets();
 }
 
 // End AggContext.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
index fe7ee06..f80dd2b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
@@ -16,6 +16,9 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.core.AggregateCall;
+
 /**
  * Information for a call to
  * {@link AggImplementor#implementResult(AggContext, AggResultContext)}
@@ -25,6 +28,17 @@ package org.apache.calcite.adapter.enumerable;
  * implementation MUST NOT destroy the contents of {@link #accumulator()}.
  */
 public interface AggResultContext extends NestedBlockBuilder, AggResetContext {
+  /** Expression by which to reference the key upon which the values in the
+   * accumulator were aggregated. Most aggregate functions depend on only the
+   * accumulator, but quasi-aggregate functions such as GROUPING access at the
+   * key. */
+  Expression key();
+
+  /** Returns an expression that references the {@code i}th field of the key,
+   * cast to the appropriate type. */
+  Expression keyField(int i);
+
+  AggregateCall call();
 }
 
 // End AggResultContext.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
index 2d8c497..4d6fb05 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
@@ -46,6 +46,8 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.lang.reflect.Type;
@@ -66,6 +68,8 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
       List<AggregateCall> aggCalls)
       throws InvalidRelException {
     super(cluster, traitSet, child, indicator, groupSet, groupSets, aggCalls);
+    Preconditions.checkArgument(!indicator,
+        "EnumerableAggregate no longer supports indicator fields");
     assert getConvention() instanceof EnumerableConvention;
 
     for (AggregateCall aggCall : aggCalls) {
@@ -104,7 +108,6 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
         builder.append(
             "child",
             result.block);
-    final RelDataType inputRowType = getInput().getRowType();
 
     final PhysType physType =
         PhysTypeImpl.of(
@@ -178,7 +181,6 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
         inputPhysType.project(groupSet.asList(), getGroupType() != Group.SIMPLE,
             JavaRowFormat.LIST);
     final int groupCount = getGroupCount();
-    final int indicatorCount = getIndicatorCount();
 
     final List<AggImpState> aggs = new ArrayList<>(aggCalls.size());
     for (Ord<AggregateCall> call : Ord.zip(aggCalls)) {
@@ -196,34 +198,11 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
 
     final List<Type> aggStateTypes = new ArrayList<>();
     for (final AggImpState agg : aggs) {
-      agg.context =
-          new AggContext() {
-            public SqlAggFunction aggregation() {
-              return agg.call.getAggregation();
-            }
-
-            public RelDataType returnRelType() {
-              return agg.call.type;
-            }
-
-            public Type returnType() {
-              return EnumUtils.javaClass(typeFactory, returnRelType());
-            }
-
-            public List<? extends RelDataType> parameterRelTypes() {
-              return EnumUtils.fieldRowTypes(inputRowType, null,
-                  agg.call.getArgList());
-            }
-
-            public List<? extends Type> parameterTypes() {
-              return EnumUtils.fieldTypes(typeFactory,
-                  parameterRelTypes());
-            }
-          };
-      List<Type> state =
-          agg.implementor.getStateType(agg.context);
+      agg.context = new AggContextImpl(agg, typeFactory);
+      final List<Type> state = agg.implementor.getStateType(agg.context);
 
       if (state.isEmpty()) {
+        agg.state = ImmutableList.of();
         continue;
       }
 
@@ -246,12 +225,11 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
       agg.state = decls;
       initExpressions.addAll(decls);
       agg.implementor.implementReset(agg.context,
-          new AggResultContextImpl(initBlock, decls));
+          new AggResultContextImpl(initBlock, agg.call, decls, null, null));
     }
 
     final PhysType accPhysType =
-        PhysTypeImpl.of(
-            typeFactory,
+        PhysTypeImpl.of(typeFactory,
             typeFactory.createSyntheticType(aggStateTypes));
 
 
@@ -369,15 +347,24 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
     } else {
       final Type keyType = keyPhysType.getJavaRowType();
       key_ = Expressions.parameter(keyType, "key");
-      for (int j = 0; j < groupCount + indicatorCount; j++) {
-        results.add(
-            keyPhysType.fieldReference(key_, j));
+      for (int j = 0; j < groupCount; j++) {
+        final Expression ref = keyPhysType.fieldReference(key_, j);
+        if (getGroupType() == Group.SIMPLE) {
+          results.add(ref);
+        } else {
+          results.add(
+              Expressions.condition(
+                  keyPhysType.fieldReference(key_, groupCount + j),
+                  Expressions.constant(null),
+                  Expressions.box(ref)));
+        }
       }
     }
     for (final AggImpState agg : aggs) {
       results.add(
           agg.implementor.implementResult(agg.context,
-              new AggResultContextImpl(resultBlock, agg.state)));
+              new AggResultContextImpl(resultBlock, agg.call, agg.state, key_,
+                  keyPhysType)));
     }
     resultBlock.add(physType.record(results));
     if (getGroupType() != Group.SIMPLE) {
@@ -461,6 +448,57 @@ public class EnumerableAggregate extends Aggregate implements EnumerableRel {
     }
     return implementor.result(physType, builder.toBlock());
   }
+
+  /** An implementation of {@link AggContext}. */
+  private class AggContextImpl implements AggContext {
+    private final AggImpState agg;
+    private final JavaTypeFactory typeFactory;
+
+    AggContextImpl(AggImpState agg, JavaTypeFactory typeFactory) {
+      this.agg = agg;
+      this.typeFactory = typeFactory;
+    }
+
+    public SqlAggFunction aggregation() {
+      return agg.call.getAggregation();
+    }
+
+    public RelDataType returnRelType() {
+      return agg.call.type;
+    }
+
+    public Type returnType() {
+      return EnumUtils.javaClass(typeFactory, returnRelType());
+    }
+
+    public List<? extends RelDataType> parameterRelTypes() {
+      return EnumUtils.fieldRowTypes(getInput().getRowType(), null,
+          agg.call.getArgList());
+    }
+
+    public List<? extends Type> parameterTypes() {
+      return EnumUtils.fieldTypes(
+          typeFactory,
+          parameterRelTypes());
+    }
+
+    public List<ImmutableBitSet> groupSets() {
+      return groupSets;
+    }
+
+    public List<Integer> keyOrdinals() {
+      return groupSet.asList();
+    }
+
+    public List<? extends RelDataType> keyRelTypes() {
+      return EnumUtils.fieldRowTypes(getInput().getRowType(), null,
+          groupSet.asList());
+    }
+
+    public List<? extends Type> keyTypes() {
+      return EnumUtils.fieldTypes(typeFactory, keyRelTypes());
+    }
+  }
 }
 
 // End EnumerableAggregate.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
index 9afee80..0e32225 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
@@ -49,6 +49,7 @@ import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.runtime.SortedMultiMap;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -788,6 +789,22 @@ public class EnumerableWindow extends Window implements EnumerableRel {
               return EnumUtils.fieldRowTypes(result.physType.getRowType(),
                   constants, agg.call.getArgList());
             }
+
+            public List<ImmutableBitSet> groupSets() {
+              throw new UnsupportedOperationException();
+            }
+
+            public List<Integer> keyOrdinals() {
+              throw new UnsupportedOperationException();
+            }
+
+            public List<? extends RelDataType> keyRelTypes() {
+              throw new UnsupportedOperationException();
+            }
+
+            public List<? extends Type> keyTypes() {
+              throw new UnsupportedOperationException();
+            }
           };
       String aggName = "a" + agg.aggIdx;
       if (CalcitePrepareImpl.DEBUG) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 17cb540..0c1f542 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -53,6 +53,7 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
 import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Supplier;
@@ -129,6 +130,9 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FIRST_VALUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FLOOR;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GROUPING;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GROUPING_ID;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GROUP_ID;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.INITCAP;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_FALSE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_FALSE;
@@ -410,6 +414,11 @@ public class RexImpTable {
     aggMap.put(MAX, minMax);
     aggMap.put(SINGLE_VALUE, constructorSupplier(SingleValueImplementor.class));
     aggMap.put(COLLECT, constructorSupplier(CollectImplementor.class));
+    final Supplier<GroupingImplementor> grouping =
+        constructorSupplier(GroupingImplementor.class);
+    aggMap.put(GROUPING, grouping);
+    aggMap.put(GROUP_ID, grouping);
+    aggMap.put(GROUPING_ID, grouping);
     winAggMap.put(RANK, constructorSupplier(RankImplementor.class));
     winAggMap.put(DENSE_RANK, constructorSupplier(DenseRankImplementor.class));
     winAggMap.put(ROW_NUMBER, constructorSupplier(RowNumberImplementor.class));
@@ -1229,6 +1238,57 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code GROUPING} aggregate function. */
+  static class GroupingImplementor implements AggImplementor {
+    public List<Type> getStateType(AggContext info) {
+      return ImmutableList.of();
+    }
+
+    public void implementReset(AggContext info, AggResetContext reset) {
+    }
+
+    public void implementAdd(AggContext info, AggAddContext add) {
+    }
+
+    public Expression implementResult(AggContext info,
+        AggResultContext result) {
+      final List<Integer> keys;
+      switch (info.aggregation().kind) {
+      case GROUPING: // "GROUPING(e, ...)", also "GROUPING_ID(e, ...)"
+        keys = result.call().getArgList();
+        break;
+      case GROUP_ID: // "GROUP_ID()"
+        // We don't implement GROUP_ID properly. In most circumstances, it
+        // returns 0, so we always return 0. Logged
+        // [CALCITE-1824] GROUP_ID returns wrong result
+        keys = ImmutableIntList.of();
+        break;
+      default:
+        throw new AssertionError();
+      }
+      Expression e = null;
+      if (info.groupSets().size() > 1) {
+        final List<Integer> keyOrdinals = info.keyOrdinals();
+        long x = 1L << (keys.size() - 1);
+        for (int k : keys) {
+          final int i = keyOrdinals.indexOf(k);
+          assert i >= 0;
+          final Expression e2 =
+              Expressions.condition(result.keyField(keyOrdinals.size() + i),
+                  Expressions.constant(x),
+                  Expressions.constant(0L));
+          if (e == null) {
+            e = e2;
+          } else {
+            e = Expressions.add(e, e2);
+          }
+          x >>= 1;
+        }
+      }
+      return e != null ? e : Expressions.constant(0, info.returnType());
+    }
+  }
+
   /** Implementor for user-defined aggregate functions. */
   public static class UserDefinedAggReflectiveImplementor
       extends StrictAggImplementor {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
index 18ba8cc..8d575bf 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
@@ -29,7 +29,7 @@ import java.util.List;
 public abstract class AggAddContextImpl extends AggResultContextImpl
     implements AggAddContext {
   public AggAddContextImpl(BlockBuilder block, List<Expression> accumulator) {
-    super(block, accumulator);
+    super(block, null, accumulator, null, null);
   }
 
   public final List<Expression> arguments() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
index 3e66705..c159073 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.enumerable.AggResetContext;
 import org.apache.calcite.adapter.enumerable.NestedBlockBuilderImpl;
 import org.apache.calcite.linq4j.tree.BlockBuilder;
 import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.core.AggregateCall;
 
 import java.util.List;
 
@@ -27,14 +28,15 @@ import java.util.List;
  * Implementation of
  * {@link org.apache.calcite.adapter.enumerable.AggResetContext}
  */
-public class AggResetContextImpl extends NestedBlockBuilderImpl
+public abstract class AggResetContextImpl extends NestedBlockBuilderImpl
     implements AggResetContext {
   private final List<Expression> accumulator;
 
   /**
-   * Creates aggregate reset context
-   * @param block code block that will contain the added initialization
-   * @param accumulator accumulator variables that store the intermediate
+   * Creates aggregate reset context.
+   *
+   * @param block Code block that will contain the added initialization
+   * @param accumulator Accumulator variables that store the intermediate
    *                    aggregate state
    */
   public AggResetContextImpl(BlockBuilder block, List<Expression> accumulator) {
@@ -45,6 +47,10 @@ public class AggResetContextImpl extends NestedBlockBuilderImpl
   public List<Expression> accumulator() {
     return accumulator;
   }
+
+  public AggregateCall call() {
+    throw new UnsupportedOperationException();
+  }
 }
 
 // End AggResetContextImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
index 1da28c6..6e02756 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
@@ -17,8 +17,11 @@
 package org.apache.calcite.adapter.enumerable.impl;
 
 import org.apache.calcite.adapter.enumerable.AggResultContext;
+import org.apache.calcite.adapter.enumerable.PhysType;
 import org.apache.calcite.linq4j.tree.BlockBuilder;
 import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.core.AggregateCall;
 
 import java.util.List;
 
@@ -28,15 +31,38 @@ import java.util.List;
  */
 public class AggResultContextImpl extends AggResetContextImpl
     implements AggResultContext {
+  private final AggregateCall call;
+  private final ParameterExpression key;
+  private final PhysType keyPhysType;
+
   /**
-   * Creates aggregate result context
-   * @param block code block that will contain the result calculation statements
-   * @param accumulator accumulator variables that store the intermediate
+   * Creates aggregate result context.
+   *
+   * @param block Code block that will contain the result calculation statements
+   * @param call Aggregate call
+   * @param accumulator Accumulator variables that store the intermediate
    *                    aggregate state
+   * @param key Key
    */
-  public AggResultContextImpl(BlockBuilder block,
-      List<Expression> accumulator) {
+  public AggResultContextImpl(BlockBuilder block, AggregateCall call,
+      List<Expression> accumulator, ParameterExpression key,
+      PhysType keyPhysType) {
     super(block, accumulator);
+    this.call = call;
+    this.key = key;
+    this.keyPhysType = keyPhysType;
+  }
+
+  public Expression key() {
+    return key;
+  }
+
+  public Expression keyField(int i) {
+    return keyPhysType.fieldReference(key, i);
+  }
+
+  @Override public AggregateCall call() {
+    return call;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
index d4a5c6f..df0d639 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
@@ -45,7 +45,7 @@ public abstract class WinAggResultContextImpl extends AggResultContextImpl
   public WinAggResultContextImpl(BlockBuilder block,
       List<Expression> accumulator,
       Function<BlockBuilder, WinAggFrameResultContext> frameContextBuilder) {
-    super(block, accumulator);
+    super(block, null, accumulator, null, null);
     this.frame = frameContextBuilder;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index 5c4aa0b..238cf3c 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -508,7 +508,7 @@ public abstract class RelOptUtil {
               extraName);
 
       ret =
-          LogicalAggregate.create(ret, false,
+          LogicalAggregate.create(ret,
               ImmutableBitSet.of(), null, ImmutableList.of(aggCall));
     }
 
@@ -557,8 +557,7 @@ public abstract class RelOptUtil {
         || logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
     if (!outerJoin) {
       final LogicalAggregate aggregate =
-          LogicalAggregate.create(ret, false,
-              ImmutableBitSet.range(keyCount), null,
+          LogicalAggregate.create(ret, ImmutableBitSet.range(keyCount), null,
               ImmutableList.<AggregateCall>of());
       return new Exists(aggregate, false, false);
     }
@@ -586,9 +585,8 @@ public abstract class RelOptUtil {
             null,
             null);
 
-    ret = LogicalAggregate.create(ret, false,
-        ImmutableBitSet.range(projectedKeyCount), null,
-        ImmutableList.of(aggCall));
+    ret = LogicalAggregate.create(ret, ImmutableBitSet.range(projectedKeyCount),
+        null, ImmutableList.of(aggCall));
 
     switch (logic) {
     case TRUE_FALSE_UNKNOWN:
@@ -787,14 +785,13 @@ public abstract class RelOptUtil {
               0, rel, null, null));
     }
 
-    return LogicalAggregate.create(rel, false,
-        ImmutableBitSet.of(), null, aggCalls);
+    return LogicalAggregate.create(rel, ImmutableBitSet.of(), null, aggCalls);
   }
 
   /** @deprecated Use {@link RelBuilder#distinct()}. */
   @Deprecated // to be removed before 2.0
   public static RelNode createDistinctRel(RelNode rel) {
-    return LogicalAggregate.create(rel, false,
+    return LogicalAggregate.create(rel,
         ImmutableBitSet.range(rel.getRowType().getFieldCount()), null,
         ImmutableList.<AggregateCall>of());
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
index 15cedcb..5fd8f6e 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -1204,8 +1204,7 @@ public class SubstitutionVisitor {
         Mappings.apply2(mapping, aggregate.groupSets);
     List<AggregateCall> aggregateCalls =
         apply(mapping, aggregate.aggCalls);
-    return MutableAggregate.of(input, aggregate.indicator, groupSet, groupSets,
-        aggregateCalls);
+    return MutableAggregate.of(input, groupSet, groupSets, aggregateCalls);
   }
 
   private static List<AggregateCall> apply(final Mapping mapping,
@@ -1267,7 +1266,7 @@ public class SubstitutionVisitor {
                 ImmutableList.of(target.groupSet.cardinality() + i), -1,
                 aggregateCall.type, aggregateCall.name));
       }
-      result = MutableAggregate.of(target, false, groupSet.build(), null,
+      result = MutableAggregate.of(target, groupSet.build(), null,
           aggregateCalls);
     }
     return MutableRels.createCastRel(result, query.rowType, true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
index 7388713..93f9948 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
@@ -80,6 +80,13 @@ public abstract class Aggregate extends SingleRel {
         }
       };
 
+  public static final Predicate<Aggregate> NO_INDICATOR =
+      new PredicateImpl<Aggregate>() {
+        public boolean test(Aggregate input) {
+          return !input.indicator;
+        }
+      };
+
   public static final Predicate<Aggregate> IS_NOT_GRAND_TOTAL =
       new PredicateImpl<Aggregate>() {
         public boolean test(Aggregate input) {
@@ -89,6 +96,11 @@ public abstract class Aggregate extends SingleRel {
 
   //~ Instance fields --------------------------------------------------------
 
+  /** Whether there are indicator fields.
+   *
+   * <p>We strongly discourage the use indicator fields, because they cause the
+   * output row type of GROUPING SETS queries to be different from regular GROUP
+   * BY queries, and recommend that you set this field to {@code false}. */
   public final boolean indicator;
   protected final List<AggregateCall> aggCalls;
   protected final ImmutableBitSet groupSet;
@@ -118,8 +130,7 @@ public abstract class Aggregate extends SingleRel {
    * @param traits   Traits
    * @param child    Child
    * @param indicator Whether row type should include indicator fields to
-   *                 indicate which grouping set is active; must be true if
-   *                 aggregate is not simple
+   *                 indicate which grouping set is active; true is deprecated
    * @param groupSet Bit set of grouping fields
    * @param groupSets List of all grouping sets; null for just {@code groupSet}
    * @param aggCalls Collection of calls to aggregate functions
@@ -133,7 +144,7 @@ public abstract class Aggregate extends SingleRel {
       List<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls) {
     super(cluster, traits, child);
-    this.indicator = indicator;
+    this.indicator = indicator; // true is allowed, but discouraged
     this.aggCalls = ImmutableList.copyOf(aggCalls);
     this.groupSet = Preconditions.checkNotNull(groupSet);
     if (groupSets == null) {
@@ -346,6 +357,9 @@ public abstract class Aggregate extends SingleRel {
       final RelDataTypeField field = fieldList.get(groupKey);
       containedNames.add(field.getName());
       builder.add(field);
+      if (groupSets != null && !allContain(groupSets, groupKey)) {
+        builder.nullable(true);
+      }
     }
     if (indicator) {
       for (int groupKey : groupList) {
@@ -380,6 +394,16 @@ public abstract class Aggregate extends SingleRel {
     return builder.build();
   }
 
+  private static boolean allContain(List<ImmutableBitSet> groupSets,
+      int groupKey) {
+    for (ImmutableBitSet groupSet : groupSets) {
+      if (!groupSet.get(groupKey)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   public boolean isValid(Litmus litmus, Context context) {
     return super.isValid(litmus, context)
         && litmus.check(Util.isDistinct(getRowType().getFieldNames()),

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index c98a109..477bbd4 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -206,6 +206,7 @@ public class RelFactories {
    * that returns a vanilla {@link LogicalAggregate}.
    */
   private static class AggregateFactoryImpl implements AggregateFactory {
+    @SuppressWarnings("deprecation")
     public RelNode createAggregate(RelNode input, boolean indicator,
         ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
         List<AggregateCall> aggCalls) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
index d6b9e57..720491d 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
@@ -86,6 +86,21 @@ public final class LogicalAggregate extends Aggregate {
 
   /** Creates a LogicalAggregate. */
   public static LogicalAggregate create(final RelNode input,
+      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets,
+      List<AggregateCall> aggCalls) {
+    return create_(input, false, groupSet, groupSets, aggCalls);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public static LogicalAggregate create(final RelNode input,
+      boolean indicator,
+      ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets,
+      List<AggregateCall> aggCalls) {
+    return create_(input, indicator, groupSet, groupSets, aggCalls);
+  }
+
+  private static LogicalAggregate create_(final RelNode input,
       boolean indicator,
       ImmutableBitSet groupSet,
       List<ImmutableBitSet> groupSets,

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/mutable/MutableAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/mutable/MutableAggregate.java b/core/src/main/java/org/apache/calcite/rel/mutable/MutableAggregate.java
index 546b62a..83ae7b9 100644
--- a/core/src/main/java/org/apache/calcite/rel/mutable/MutableAggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/mutable/MutableAggregate.java
@@ -28,16 +28,14 @@ import java.util.Objects;
 
 /** Mutable equivalent of {@link org.apache.calcite.rel.core.Aggregate}. */
 public class MutableAggregate extends MutableSingleRel {
-  public final boolean indicator;
   public final ImmutableBitSet groupSet;
   public final ImmutableList<ImmutableBitSet> groupSets;
   public final List<AggregateCall> aggCalls;
 
   private MutableAggregate(MutableRel input, RelDataType rowType,
-      boolean indicator, ImmutableBitSet groupSet,
+      ImmutableBitSet groupSet,
       List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
     super(MutableRelType.AGGREGATE, rowType, input);
-    this.indicator = indicator;
     this.groupSet = groupSet;
     this.groupSets = groupSets == null
         ? ImmutableList.of(groupSet)
@@ -49,20 +47,16 @@ public class MutableAggregate extends MutableSingleRel {
    * Creates a MutableAggregate.
    *
    * @param input     Input relational expression
-   * @param indicator Whether row type should include indicator fields to
-   *                  indicate which grouping set is active; must be true if
-   *                  aggregate is not simple
    * @param groupSet  Bit set of grouping fields
    * @param groupSets List of all grouping sets; null for just {@code groupSet}
    * @param aggCalls  Collection of calls to aggregate functions
    */
-  public static MutableAggregate of(MutableRel input, boolean indicator,
-      ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
-      List<AggregateCall> aggCalls) {
+  public static MutableAggregate of(MutableRel input, ImmutableBitSet groupSet,
+      ImmutableList<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
     RelDataType rowType =
         Aggregate.deriveRowType(input.cluster.getTypeFactory(),
-            input.rowType, indicator, groupSet, groupSets, aggCalls);
-    return new MutableAggregate(input, rowType, indicator, groupSet,
+            input.rowType, false, groupSet, groupSets, aggCalls);
+    return new MutableAggregate(input, rowType, groupSet,
         groupSets, aggCalls);
   }
 
@@ -89,8 +83,7 @@ public class MutableAggregate extends MutableSingleRel {
   }
 
   @Override public MutableRel clone() {
-    return MutableAggregate.of(input.clone(),
-        indicator, groupSet, groupSets, aggCalls);
+    return MutableAggregate.of(input.clone(), groupSet, groupSets, aggCalls);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
index b07da0e..8470dc6 100644
--- a/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
+++ b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
@@ -194,7 +194,7 @@ public abstract class MutableRels {
       final MutableAggregate aggregate = (MutableAggregate) node;
       relBuilder.push(fromMutable(aggregate.input, relBuilder));
       relBuilder.aggregate(
-          relBuilder.groupKey(aggregate.groupSet, aggregate.indicator, aggregate.groupSets),
+          relBuilder.groupKey(aggregate.groupSet, aggregate.groupSets),
           aggregate.aggCalls);
       return relBuilder.build();
     case SORT:
@@ -312,9 +312,8 @@ public abstract class MutableRels {
     if (rel instanceof Aggregate) {
       final Aggregate aggregate = (Aggregate) rel;
       final MutableRel input = toMutable(aggregate.getInput());
-      return MutableAggregate.of(input, aggregate.indicator,
-          aggregate.getGroupSet(), aggregate.getGroupSets(),
-          aggregate.getAggCallList());
+      return MutableAggregate.of(input, aggregate.getGroupSet(),
+          aggregate.getGroupSets(), aggregate.getAggCallList());
     }
     if (rel instanceof Sort) {
       final Sort sort = (Sort) rel;

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
index 665bccf..7202b81 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
@@ -1053,7 +1053,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
                         relBuilder.peek(), aggregate.getGroupCount() + i))));
       }
       RelNode result = relBuilder
-          .aggregate(relBuilder.groupKey(groupSet, false, null), aggregateCalls)
+          .aggregate(relBuilder.groupKey(groupSet, null), aggregateCalls)
           .build();
       if (topProject != null) {
         // Top project
@@ -1274,7 +1274,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
         }
         result = relBuilder
             .push(result)
-            .aggregate(relBuilder.groupKey(groupSet, false, null), aggregateCalls)
+            .aggregate(relBuilder.groupKey(groupSet, null), aggregateCalls)
             .build();
         // We introduce a project on top, as group by columns order is lost
         List<RexNode> projects = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index dee822d..ca95bcc 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.rel.rules;
 
+import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
@@ -26,8 +26,6 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalAggregate;
-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.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
@@ -36,7 +34,6 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
-import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -49,10 +46,10 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
-import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -90,8 +87,6 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
       new AggregateExpandDistinctAggregatesRule(LogicalAggregate.class, false,
           RelFactories.LOGICAL_BUILDER);
 
-  private static final BigDecimal TWO = BigDecimal.valueOf(2L);
-
   public final boolean useGroupingSets;
 
   //~ Constructors -----------------------------------------------------------
@@ -161,7 +156,9 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
 
     // If all of the agg expressions are distinct and have the same
     // arguments then we can use a more efficient form.
-    if (nonDistinctAggCallCount == 0 && argLists.size() == 1) {
+    if (nonDistinctAggCallCount == 0
+        && argLists.size() == 1
+        && aggregate.getGroupSets().size() == 1) {
       final Pair<List<Integer>, Integer> pair =
           Iterables.getOnlyElement(argLists);
       final RelBuilder relBuilder = call.builder();
@@ -171,7 +168,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     }
 
     if (useGroupingSets) {
-      rewriteUsingGroupingSets(call, aggregate, argLists);
+      rewriteUsingGroupingSets(call, aggregate);
       return;
     }
 
@@ -224,7 +221,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     int n = 0;
     if (!newAggCallList.isEmpty()) {
       final RelBuilder.GroupKey groupKey =
-          relBuilder.groupKey(groupSet, aggregate.indicator, aggregate.getGroupSets());
+          relBuilder.groupKey(groupSet, aggregate.getGroupSets());
       relBuilder.aggregate(groupKey, newAggCallList);
       ++n;
     }
@@ -370,17 +367,19 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     return relBuilder;
   }
 
-  @SuppressWarnings("DanglingJavadoc")
   private void rewriteUsingGroupingSets(RelOptRuleCall call,
-      Aggregate aggregate, Set<Pair<List<Integer>, Integer>> argLists) {
+      Aggregate aggregate) {
     final Set<ImmutableBitSet> groupSetTreeSet =
         new TreeSet<>(ImmutableBitSet.ORDERING);
-    groupSetTreeSet.add(aggregate.getGroupSet());
-    for (Pair<List<Integer>, Integer> argList : argLists) {
-      groupSetTreeSet.add(
-          ImmutableBitSet.of(argList.left)
-              .setIf(argList.right, argList.right >= 0)
-              .union(aggregate.getGroupSet()));
+    for (AggregateCall aggCall : aggregate.getAggCallList()) {
+      if (!aggCall.isDistinct()) {
+        groupSetTreeSet.add(aggregate.getGroupSet());
+      } else {
+        groupSetTreeSet.add(
+            ImmutableBitSet.of(aggCall.getArgList())
+                .setIf(aggCall.filterArg, aggCall.filterArg >= 0)
+                .union(aggregate.getGroupSet()));
+      }
     }
 
     final ImmutableList<ImmutableBitSet> groupSets =
@@ -396,87 +395,38 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
 
     final RelBuilder relBuilder = call.builder();
     relBuilder.push(aggregate.getInput());
-    final boolean indicator = groupSets.size() > 1;
-    relBuilder.aggregate(
-        relBuilder.groupKey(fullGroupSet, indicator, groupSets),
-        distinctAggCalls);
-    final RelNode distinct = relBuilder.peek();
     final int groupCount = fullGroupSet.cardinality();
-    final int indicatorCount = indicator ? groupCount : 0;
-
-    final RelOptCluster cluster = aggregate.getCluster();
-    final RexBuilder rexBuilder = cluster.getRexBuilder();
-    final RelDataTypeFactory typeFactory = cluster.getTypeFactory();
-    final RelDataType booleanType =
-        typeFactory.createTypeWithNullability(
-            typeFactory.createSqlType(SqlTypeName.BOOLEAN), false);
-    final List<Pair<RexNode, String>> predicates = new ArrayList<>();
-    final Map<ImmutableBitSet, Integer> filters = new HashMap<>();
-
-    /** Function to register a filter for a group set. */
-    class Registrar {
-      RexNode group = null;
-
-      private int register(ImmutableBitSet groupSet) {
-        if (group == null) {
-          group = makeGroup(groupCount - 1);
-        }
-        final RexNode node =
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, group,
-                rexBuilder.makeExactLiteral(
-                    toNumber(remap(fullGroupSet, groupSet))));
-        predicates.add(Pair.of(node, toString(groupSet)));
-        return groupCount + indicatorCount + distinctAggCalls.size()
-            + predicates.size() - 1;
-      }
 
-      private RexNode makeGroup(int i) {
-        final RexInputRef ref =
-            rexBuilder.makeInputRef(booleanType, groupCount + i);
-        final RexNode kase =
-            rexBuilder.makeCall(SqlStdOperatorTable.CASE, ref,
-                rexBuilder.makeExactLiteral(BigDecimal.ZERO),
-                rexBuilder.makeExactLiteral(TWO.pow(i)));
-        if (i == 0) {
-          return kase;
-        } else {
-          return rexBuilder.makeCall(SqlStdOperatorTable.PLUS,
-              makeGroup(i - 1), kase);
-        }
-      }
-
-      private BigDecimal toNumber(ImmutableBitSet bitSet) {
-        BigDecimal n = BigDecimal.ZERO;
-        for (int key : bitSet) {
-          n = n.add(TWO.pow(key));
-        }
-        return n;
-      }
-
-      private String toString(ImmutableBitSet bitSet) {
-        final StringBuilder buf = new StringBuilder("$i");
-        for (int key : bitSet) {
-          buf.append(key).append('_');
-        }
-        return buf.substring(0, buf.length() - 1);
-      }
-    }
-    final Registrar registrar = new Registrar();
-    for (ImmutableBitSet groupSet : groupSets) {
-      filters.put(groupSet, indicator ? registrar.register(groupSet) : -1);
+    final Map<ImmutableBitSet, Integer> filters = new LinkedHashMap<>();
+    final int z = groupCount + distinctAggCalls.size();
+    distinctAggCalls.add(
+        AggregateCall.create(SqlStdOperatorTable.GROUPING, false,
+            ImmutableIntList.copyOf(fullGroupSet), -1, groupSets.size(),
+            relBuilder.peek(), null, "$g"));
+    for (Ord<ImmutableBitSet> groupSet : Ord.zip(groupSets)) {
+      filters.put(groupSet.e, z + groupSet.i);
     }
 
-    if (!predicates.isEmpty()) {
-      List<Pair<RexNode, String>> nodes = new ArrayList<>();
-      for (RelDataTypeField f : relBuilder.peek().getRowType().getFieldList()) {
-        final RexNode node = rexBuilder.makeInputRef(f.getType(), f.getIndex());
-        nodes.add(Pair.of(node, f.getName()));
+    relBuilder.aggregate(relBuilder.groupKey(fullGroupSet, groupSets),
+        distinctAggCalls);
+    final RelNode distinct = relBuilder.peek();
+
+    // GROUPING returns an integer (0 or 1). Add a project to convert those
+    // values to BOOLEAN.
+    if (!filters.isEmpty()) {
+      final List<RexNode> nodes = new ArrayList<>(relBuilder.fields());
+      final RexNode nodeZ = nodes.remove(nodes.size() - 1);
+      for (Map.Entry<ImmutableBitSet, Integer> entry : filters.entrySet()) {
+        final long v = groupValue(fullGroupSet, entry.getKey());
+        nodes.add(
+            relBuilder.alias(
+                relBuilder.equals(nodeZ, relBuilder.literal(v)),
+                "$g_" + v));
       }
-      nodes.addAll(predicates);
-      relBuilder.project(Pair.left(nodes), Pair.right(nodes));
+      relBuilder.project(nodes);
     }
 
-    int x = groupCount + indicatorCount;
+    int x = groupCount;
     final List<AggregateCall> newCalls = new ArrayList<>();
     for (AggregateCall aggCall : aggregate.getAggCallList()) {
       final int newFilterArg;
@@ -504,13 +454,26 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     relBuilder.aggregate(
         relBuilder.groupKey(
             remap(fullGroupSet, aggregate.getGroupSet()),
-            aggregate.indicator,
             remap(fullGroupSet, aggregate.getGroupSets())),
         newCalls);
     relBuilder.convert(aggregate.getRowType(), true);
     call.transformTo(relBuilder.build());
   }
 
+  private static long groupValue(ImmutableBitSet fullGroupSet,
+      ImmutableBitSet groupSet) {
+    long v = 0;
+    long x = 1L << (fullGroupSet.cardinality() - 1);
+    assert fullGroupSet.contains(groupSet);
+    for (int i : fullGroupSet) {
+      if (!groupSet.get(i)) {
+        v |= x;
+      }
+      x >>= 1;
+    }
+    return v;
+  }
+
   private static ImmutableBitSet remap(ImmutableBitSet groupSet,
       ImmutableBitSet bitSet) {
     final ImmutableBitSet.Builder builder = ImmutableBitSet.builder();

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
index 932a18f..dfe28f5 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
@@ -106,7 +106,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
         RexUtil.apply(mapping, filter.getCondition());
     final Filter newFilter = filter.copy(filter.getTraitSet(),
         newAggregate, newCondition);
-    if (allColumnsInAggregate && !aggregate.indicator) {
+    if (allColumnsInAggregate && aggregate.getGroupSets().size() == 1) {
       // Everything needed by the filter is returned by the aggregate.
       assert newGroupSet.equals(aggregate.getGroupSet());
       call.transformTo(newFilter);
@@ -119,7 +119,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
         topGroupSet.set(newGroupSet.indexOf(c));
       }
       ImmutableList<ImmutableBitSet> newGroupingSets = null;
-      if (aggregate.indicator) {
+      if (aggregate.groupSets.size() > 1) {
         ImmutableList.Builder<ImmutableBitSet> newGroupingSetsBuilder =
                 ImmutableList.builder();
         for (ImmutableBitSet groupingSet : aggregate.getGroupSets()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
index 6544db0..3806f07 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
@@ -253,7 +253,7 @@ public class AggregateJoinTransposeRule extends RelOptRule {
           }
         }
         side.newInput = relBuilder.push(joinInput)
-            .aggregate(relBuilder.groupKey(belowAggregateKey, false, null),
+            .aggregate(relBuilder.groupKey(belowAggregateKey, null),
                 belowAggCalls)
             .build();
       }
@@ -336,7 +336,7 @@ public class AggregateJoinTransposeRule extends RelOptRule {
     if (!aggConvertedToProjects) {
       relBuilder.aggregate(
           relBuilder.groupKey(Mappings.apply(mapping, aggregate.getGroupSet()),
-              aggregate.indicator, Mappings.apply2(mapping, aggregate.getGroupSets())),
+              Mappings.apply2(mapping, aggregate.getGroupSets())),
           newAggCalls);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
index 5b486eb..432a891 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
@@ -88,7 +88,7 @@ public class AggregateProjectMergeRule extends RelOptRule {
 
     final ImmutableBitSet newGroupSet = aggregate.getGroupSet().permute(map);
     ImmutableList<ImmutableBitSet> newGroupingSets = null;
-    if (aggregate.indicator) {
+    if (aggregate.getGroupSets().size() > 1) {
       newGroupingSets =
           ImmutableBitSet.ORDERING.immutableSortedCopy(
               ImmutableBitSet.permute(aggregate.getGroupSets(), map));

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index 6bb1d3b..75af27d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -153,9 +153,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
           aggCall.adaptTo(input, aggCall.getArgList(), aggCall.filterArg,
               groupCount, newGroupCount));
     }
-    relBuilder.aggregate(
-        relBuilder.groupKey(newGroupSet, false, null),
-        newAggCalls);
+    relBuilder.aggregate(relBuilder.groupKey(newGroupSet, null), newAggCalls);
 
     // Create a projection back again.
     List<Pair<RexNode, String>> projects = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index 7e6e4a1..a9bdb84 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -560,7 +560,6 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
       List<AggregateCall> newCalls) {
     relBuilder.aggregate(
         relBuilder.groupKey(oldAggregate.getGroupSet(),
-            oldAggregate.indicator,
             oldAggregate.getGroupSets()),
         newCalls);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
index a99a29e..8eeaaa7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
@@ -104,7 +104,7 @@ public class AggregateUnionAggregateRule extends RelOptRule {
     }
 
     relBuilder.union(true);
-    relBuilder.aggregate(relBuilder.groupKey(topAggRel.getGroupSet(), false, null),
+    relBuilder.aggregate(relBuilder.groupKey(topAggRel.getGroupSet(), null),
         topAggRel.getAggCallList());
     call.transformTo(relBuilder.build());
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
index 2d8a7e0..0e07256 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
@@ -125,7 +125,7 @@ public class AggregateUnionTransposeRule extends RelOptRule {
       relBuilder.push(input);
       if (!alreadyUnique) {
         ++transformCount;
-        relBuilder.aggregate(relBuilder.groupKey(aggRel.getGroupSet(), false, null),
+        relBuilder.aggregate(relBuilder.groupKey(aggRel.getGroupSet(), null),
             aggRel.getAggCallList());
       }
     }
@@ -140,7 +140,7 @@ public class AggregateUnionTransposeRule extends RelOptRule {
     // create a new union whose children are the aggregates created above
     relBuilder.union(true, union.getInputs().size());
     relBuilder.aggregate(
-        relBuilder.groupKey(aggRel.getGroupSet(), aggRel.indicator, aggRel.getGroupSets()),
+        relBuilder.groupKey(aggRel.getGroupSet(), aggRel.getGroupSets()),
         transformedAggCalls);
     call.transformTo(relBuilder.build());
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
index 118e637..f971be3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
@@ -137,7 +137,7 @@ public class FilterAggregateTransposeRule extends RelOptRule {
       return false;
     }
 
-    if (aggregate.indicator) {
+    if (aggregate.getGroupSets().size() > 1) {
       // If grouping sets are used, the filter can be pushed if
       // the columns referenced in the predicate are present in
       // all the grouping sets.

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java b/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
index f105c53..2b4bfbb 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
@@ -107,7 +107,7 @@ public class IntersectToDistinctRule extends RelOptRule {
 
     final ImmutableBitSet groupSet =
         ImmutableBitSet.range(fieldCount - 1);
-    relBuilder.aggregate(relBuilder.groupKey(groupSet, false, null),
+    relBuilder.aggregate(relBuilder.groupKey(groupSet, null),
         relBuilder.countStar(null));
 
     // add a filter count(c) = #branches

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/rel/stream/StreamRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/stream/StreamRules.java b/core/src/main/java/org/apache/calcite/rel/stream/StreamRules.java
index 538261c..80ee68f 100644
--- a/core/src/main/java/org/apache/calcite/rel/stream/StreamRules.java
+++ b/core/src/main/java/org/apache/calcite/rel/stream/StreamRules.java
@@ -108,7 +108,7 @@ public class StreamRules {
     private DeltaAggregateTransposeRule() {
       super(
           operand(Delta.class,
-              operand(Aggregate.class, any())));
+              operand(Aggregate.class, null, Aggregate.NO_INDICATOR, any())));
     }
 
     @Override public void onMatch(RelOptRuleCall call) {
@@ -118,9 +118,8 @@ public class StreamRules {
       final LogicalDelta newDelta =
           LogicalDelta.create(aggregate.getInput());
       final LogicalAggregate newAggregate =
-          LogicalAggregate.create(newDelta, aggregate.indicator,
-              aggregate.getGroupSet(), aggregate.groupSets,
-              aggregate.getAggCallList());
+          LogicalAggregate.create(newDelta, aggregate.getGroupSet(),
+              aggregate.groupSets, aggregate.getAggCallList());
       call.transformTo(newAggregate);
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index b42b1a0..1f34ec7 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -320,9 +320,6 @@ public interface CalciteResource {
   @BaseMessage("Windowed aggregate expression is illegal in {0} clause")
   ExInst<SqlValidatorException> windowedAggregateIllegalInClause(String a0);
 
-  @BaseMessage("Aggregate expression is illegal in GROUP BY clause")
-  ExInst<SqlValidatorException> aggregateIllegalInGroupBy();
-
   @BaseMessage("Aggregate expressions cannot be nested")
   ExInst<SqlValidatorException> nestedAggIllegal();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
index e2875c0..6c1f648 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -125,6 +125,12 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
   public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
     throw new UnsupportedOperationException("remove before calcite-2.0");
   }
+
+  /** Whether this aggregate function allows a {@code FILTER (WHERE ...)}
+   * clause. */
+  public boolean allowsFilter() {
+    return true;
+  }
 }
 
 // End SqlAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
index 443bd4f..d1001dd 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
@@ -37,7 +37,7 @@ import org.apache.calcite.util.Static;
  * Base class for grouping functions {@code GROUP_ID}, {@code GROUPING_ID},
  * {@code GROUPING}.
  */
-public class SqlAbstractGroupFunction extends SqlFunction {
+public class SqlAbstractGroupFunction extends SqlAggFunction {
   /**
    * Creates a SqlAbstractGroupFunction.
    *
@@ -54,8 +54,8 @@ public class SqlAbstractGroupFunction extends SqlFunction {
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
       SqlFunctionCategory category) {
-    super(name, kind, returnTypeInference, operandTypeInference,
-        operandTypeChecker, category);
+    super(name, null, kind, returnTypeInference, operandTypeInference,
+        operandTypeChecker, category, false, false);
   }
 
   @Override public void validateCall(SqlCall call, SqlValidator validator,
@@ -88,6 +88,14 @@ public class SqlAbstractGroupFunction extends SqlFunction {
       }
     }
   }
+
+  @Override public boolean isQuantifierAllowed() {
+    return false;
+  }
+
+  @Override public boolean allowsFilter() {
+    return false;
+  }
 }
 
 // End SqlAbstractGroupFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupIdFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupIdFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupIdFunction.java
index 8f74d36..6b1138b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupIdFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupIdFunction.java
@@ -22,7 +22,10 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
- * The {@code GROUP_ID} function.
+ * The {@code GROUP_ID()} function.
+ *
+ * <p>Accepts no arguments. If the query has {@code GROUP BY x, y, z} then
+ * {@code GROUP_ID()} is the same as {@code GROUPING(x, y, z)}.
  *
  * <p>This function is not defined in the SQL standard; our implementation is
  * consistent with Oracle.

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
index 9abbd60..5504464 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
@@ -24,6 +24,13 @@ import org.apache.calcite.sql.type.ReturnTypes;
 /**
  * The {@code GROUPING} function.
  *
+ * <p>Accepts 1 or more arguments.
+ * Example: {@code GROUPING(deptno, gender)} returns
+ * 3 if both deptno and gender are being grouped,
+ * 2 if only deptno is being grouped,
+ * 1 if only gender is being groped,
+ * 0 if neither deptno nor gender are being grouped.
+ *
  * <p>This function is defined in the SQL standard.
  * {@code GROUPING_ID} is a non-standard synonym.
  *

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/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 39a45b3..7b6b771 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
@@ -195,26 +195,28 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlInternalOperator GROUPING_SETS =
       new SqlRollupOperator("GROUPING SETS", SqlKind.GROUPING_SETS);
 
-  /** {@code GROUPING} function. Occurs in similar places to an aggregate
+  /** {@code GROUPING(c1 [, c2, ...])} function.
+   *
+   * <p>Occurs in similar places to an aggregate
    * function ({@code SELECT}, {@code HAVING} clause, etc. of an aggregate
    * query), but not technically an aggregate function. */
   public static final SqlGroupingFunction GROUPING =
       new SqlGroupingFunction("GROUPING");
 
-  /** {@code GROUP_ID} function. */
+  /** {@code GROUP_ID()} function. (Oracle-specific.) */
   public static final SqlGroupIdFunction GROUP_ID =
       new SqlGroupIdFunction();
 
-  /** {@code GROUP_ID} function is a synonym for {@code GROUPING}.
+  /** {@code GROUPING_ID} function is a synonym for {@code GROUPING}.
    *
    * <p>Some history. The {@code GROUPING} function is in the SQL standard,
-   * and originally supported only one argument. The {@code GROUP_ID} is not
-   * standard (though supported in Oracle and SQL Server) and supports zero or
+   * and originally supported only one argument. {@code GROUPING_ID} is not
+   * standard (though supported in Oracle and SQL Server) and supports one or
    * more arguments.
    *
    * <p>The SQL standard has changed to allow {@code GROUPING} to have multiple
-   * arguments. It is now equivalent to {@code GROUP_ID}, so we made
-   * {@code GROUP_ID} a synonym for {@code GROUPING}. */
+   * arguments. It is now equivalent to {@code GROUPING_ID}, so we made
+   * {@code GROUPING_ID} a synonym for {@code GROUPING}. */
   public static final SqlGroupingFunction GROUPING_ID =
       new SqlGroupingFunction("GROUPING_ID");
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
index d8d51f0..0dd7f1d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.fun.SqlAbstractGroupFunction;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.util.Util;
 
@@ -105,7 +106,9 @@ class AggFinder extends SqlBasicVisitor<Void> {
   public Void visit(SqlCall call) {
     final SqlOperator operator = call.getOperator();
     // If nested aggregates disallowed or found an aggregate at invalid level
-    if (operator.isAggregator() && !operator.requiresOver()) {
+    if (operator.isAggregator()
+        && !(operator instanceof SqlAbstractGroupFunction)
+        && !operator.requiresOver()) {
       if (delegate != null) {
         return operator.acceptCall(delegate, call);
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
index 656f8d6..5df9374 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
@@ -248,7 +248,6 @@ public class AggregatingSelectScope
     public final ImmutableList<SqlNode> groupExprList;
     public final ImmutableBitSet groupSet;
     public final ImmutableList<ImmutableBitSet> groupSets;
-    public final boolean indicator;
     public final Map<Integer, Integer> groupExprProjection;
 
     Resolved(List<SqlNode> extraExprList, List<SqlNode> groupExprList,
@@ -258,7 +257,6 @@ public class AggregatingSelectScope
       this.groupExprList = ImmutableList.copyOf(groupExprList);
       this.groupSet = ImmutableBitSet.range(groupExprList.size());
       this.groupSets = ImmutableList.copyOf(groupSets);
-      this.indicator = !this.groupSets.equals(ImmutableList.of(groupSet));
       this.groupExprProjection = ImmutableMap.copyOf(groupExprProjection);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
index c5754e8..b8c2af1 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
@@ -101,10 +101,6 @@ public abstract class DelegatingNamespace implements SqlValidatorNamespace {
   public void makeNullable() {
   }
 
-  public String translate(String name) {
-    return namespace.translate(name);
-  }
-
   public <T> T unwrap(Class<T> clazz) {
     if (clazz.isInstance(this)) {
       return clazz.cast(this);

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/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 e9e12e6..e3395db 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
@@ -246,10 +246,6 @@ public class IdentifierNamespace extends AbstractNamespace {
     return resolvedNamespace.resolve();
   }
 
-  @Override public String translate(String name) {
-    return resolvedNamespace.translate(name);
-  }
-
   @Override public SqlValidatorTable getTable() {
     return resolvedNamespace == null ? null : resolve().getTable();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
index 52d4e1b..6303cb8 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
@@ -19,8 +19,6 @@ package org.apache.calcite.sql.validate;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.util.Util;
 
-import com.google.common.collect.ImmutableList;
-
 import java.util.List;
 
 /**
@@ -34,14 +32,13 @@ import java.util.List;
  * <p>It is immutable.
  */
 public class SqlQualified {
-  private final SqlValidatorScope scope;
   public final int prefixLength;
   public final SqlValidatorNamespace namespace;
   public final SqlIdentifier identifier;
 
   private SqlQualified(SqlValidatorScope scope, int prefixLength,
       SqlValidatorNamespace namespace, SqlIdentifier identifier) {
-    this.scope = scope;
+    Util.discard(scope);
     this.prefixLength = prefixLength;
     this.namespace = namespace;
     this.identifier = identifier;
@@ -56,30 +53,6 @@ public class SqlQualified {
     return new SqlQualified(scope, prefixLength, namespace, identifier);
   }
 
-  public List<String> translatedNames() {
-    if (scope == null) {
-      return identifier.names;
-    }
-    final SqlNameMatcher nameMatcher =
-        scope.getValidator().getCatalogReader().nameMatcher();
-    final ImmutableList.Builder<String> builder = ImmutableList.builder();
-    final SqlValidatorScope.ResolvedImpl resolved =
-        new SqlValidatorScope.ResolvedImpl();
-    final List<String> prefix = Util.skipLast(identifier.names);
-    scope.resolve(prefix, nameMatcher, false, resolved);
-    SqlValidatorNamespace namespace =
-        resolved.count() == 1 ? resolved.only().namespace : null;
-    builder.add(identifier.names.get(0));
-    for (String name : Util.skip(identifier.names)) {
-      if (namespace != null) {
-        name = namespace.translate(name);
-        namespace = null;
-      }
-      builder.add(name);
-    }
-    return builder.build();
-  }
-
   public final List<String> prefix() {
     return identifier.names.subList(0, prefixLength);
   }
@@ -87,10 +60,6 @@ public class SqlQualified {
   public final List<String> suffix() {
     return Util.skip(identifier.names, prefixLength);
   }
-
-  public final List<String> suffixTranslated() {
-    return Util.skip(translatedNames(), prefixLength);
-  }
 }
 
 // End SqlQualified.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/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 68106a9..cd83186 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
@@ -3793,7 +3793,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     if (groupList == null) {
       return;
     }
-    validateNoAggs(aggOrOverFinder, groupList, "GROUP BY");
+    final String clause = "GROUP BY";
+    validateNoAggs(aggOrOverFinder, groupList, clause);
     final SqlValidatorScope groupScope = getGroupScope(select);
     inferUnknownTypes(unknownType, groupScope, groupList);
 
@@ -3842,7 +3843,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
     SqlNode agg = aggFinder.findAgg(groupList);
     if (agg != null) {
-      throw newValidationError(agg, RESOURCE.aggregateIllegalInGroupBy());
+      throw newValidationError(agg, RESOURCE.aggregateIllegalInClause(clause));
     }
   }
 
@@ -3886,8 +3887,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   protected void validateWhereOrOn(
       SqlValidatorScope scope,
       SqlNode condition,
-      String keyword) {
-    validateNoAggs(aggOrOverOrGroupFinder, condition, keyword);
+      String clause) {
+    validateNoAggs(aggOrOverOrGroupFinder, condition, clause);
     inferUnknownTypes(
         booleanType,
         scope,
@@ -3896,7 +3897,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
     final RelDataType type = deriveType(scope, condition);
     if (!SqlTypeUtil.inBooleanFamily(type)) {
-      throw newValidationError(condition, RESOURCE.condMustBeBoolean(keyword));
+      throw newValidationError(condition, RESOURCE.condMustBeBoolean(clause));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
index da8d031..3c0b5f1 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
@@ -162,11 +162,6 @@ public interface SqlValidatorNamespace {
   void makeNullable();
 
   /**
-   * Translates a field name to the name in the underlying namespace.
-   */
-  String translate(String name);
-
-  /**
    * Returns this namespace, or a wrapped namespace, cast to a particular
    * class.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index c21b7c7..cb71879 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -578,11 +578,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     relBuilder.push(
-        LogicalAggregate.create(newProject,
-            false,
-            newGroupSet,
-            null,
-            newAggCalls));
+        LogicalAggregate.create(newProject, newGroupSet, null, newAggCalls));
 
     if (!omittedConstants.isEmpty()) {
       final List<RexNode> postProjects = new ArrayList<>(relBuilder.fields());
@@ -2311,12 +2307,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
       ImmutableBitSet groupSet =
           ImmutableBitSet.range(groupCount);
       LogicalAggregate newAggregate =
-          LogicalAggregate.create(joinOutputProject,
-              false,
-              groupSet,
-              null,
+          LogicalAggregate.create(joinOutputProject, groupSet, null,
               newAggCalls);
-
       List<RexNode> newAggOutputProjectList = Lists.newArrayList();
       for (int i : groupSet) {
         newAggOutputProjectList.add(

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
index ff0529d..0b7af6f 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -873,8 +873,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       ++j;
     }
 
-    final RelBuilder.GroupKey groupKey = relBuilder.groupKey(newGroupSet,
-        aggregate.indicator, newGroupSets);
+    final RelBuilder.GroupKey groupKey =
+        relBuilder.groupKey(newGroupSet, newGroupSets);
     relBuilder.aggregate(groupKey, newAggCallList);
 
     return result(relBuilder.build(), mapping);


[12/16] calcite git commit: [CALCITE-1069] In Aggregate, deprecate indicators, and allow GROUPING to be used as an aggregate function

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/test/resources/sql/agg.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/agg.iq b/core/src/test/resources/sql/agg.iq
index 28e8b4b..ba97052 100755
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -483,17 +483,52 @@ group by deptno;
 +---+---+
 | C | G |
 +---+---+
-| 1 | 1 |
-| 1 | 1 |
-| 1 | 1 |
-| 2 | 1 |
-| 2 | 1 |
-| 2 | 1 |
+| 1 | 0 |
+| 1 | 0 |
+| 1 | 0 |
+| 2 | 0 |
+| 2 | 0 |
+| 2 | 0 |
 +---+---+
 (6 rows)
 
 !ok
 
+!use scott
+
+# GROUPING in SELECT clause of CUBE query
+select deptno, job, count(*) as c, grouping(deptno) as d,
+  grouping(job) j, grouping(deptno, job) as x
+from "scott".emp
+group by cube(deptno, job);
++--------+-----------+----+---+---+---+
+| DEPTNO | JOB       | C  | D | J | X |
++--------+-----------+----+---+---+---+
+|     10 | CLERK     |  1 | 0 | 0 | 0 |
+|     10 | MANAGER   |  1 | 0 | 0 | 0 |
+|     10 | PRESIDENT |  1 | 0 | 0 | 0 |
+|     10 |           |  3 | 0 | 1 | 1 |
+|     20 | ANALYST   |  2 | 0 | 0 | 0 |
+|     20 | CLERK     |  2 | 0 | 0 | 0 |
+|     20 | MANAGER   |  1 | 0 | 0 | 0 |
+|     20 |           |  5 | 0 | 1 | 1 |
+|     30 | CLERK     |  1 | 0 | 0 | 0 |
+|     30 | MANAGER   |  1 | 0 | 0 | 0 |
+|     30 | SALESMAN  |  4 | 0 | 0 | 0 |
+|     30 |           |  6 | 0 | 1 | 1 |
+|        | ANALYST   |  2 | 1 | 0 | 2 |
+|        | CLERK     |  4 | 1 | 0 | 2 |
+|        | MANAGER   |  3 | 1 | 0 | 2 |
+|        | PRESIDENT |  1 | 1 | 0 | 2 |
+|        | SALESMAN  |  4 | 1 | 0 | 2 |
+|        |           | 14 | 1 | 1 | 3 |
++--------+-----------+----+---+---+---+
+(18 rows)
+
+!ok
+
+!use post
+
 # GROUPING, GROUP_ID, GROUPING_ID in SELECT clause of GROUP BY query
 select count(*) as c,
   grouping(deptno) as g,
@@ -501,22 +536,29 @@ select count(*) as c,
   grouping_id(deptno) as gd,
   grouping_id(gender) as gg,
   grouping_id(gender, deptno) as ggd,
-  grouping_id(gender, deptno) as gdg
+  grouping_id(deptno, gender) as gdg
 from emp
-group by deptno, gender;
+group by rollup(deptno, gender);
 +---+---+-----+----+----+-----+-----+
 | C | G | GID | GD | GG | GGD | GDG |
 +---+---+-----+----+----+-----+-----+
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 1 | 1 |   3 |  1 |  1 |   3 |   3 |
-| 2 | 1 |   3 |  1 |  1 |   3 |   3 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 1 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 2 | 0 |   0 |  0 |  0 |   0 |   0 |
+| 9 | 1 |   0 |  1 |  1 |   3 |   3 |
+| 1 | 0 |   0 |  0 |  1 |   2 |   1 |
+| 1 | 0 |   0 |  0 |  1 |   2 |   1 |
+| 1 | 0 |   0 |  0 |  1 |   2 |   1 |
+| 2 | 0 |   0 |  0 |  1 |   2 |   1 |
+| 2 | 0 |   0 |  0 |  1 |   2 |   1 |
+| 2 | 0 |   0 |  0 |  1 |   2 |   1 |
 +---+---+-----+----+----+-----+-----+
-(8 rows)
+(15 rows)
 
 !ok
 
@@ -527,40 +569,48 @@ select count(*) as c,
   grouping(deptno, gender, deptno) as gdgd,
   grouping_id(deptno, gender, deptno) as gidgd
 from emp
-group by deptno, gender
+group by rollup(deptno, gender)
 having grouping(deptno) <= grouping_id(deptno, gender, deptno);
 +---+----+-----+------+-------+
 | C | GD | GID | GDGD | GIDGD |
 +---+----+-----+------+-------+
-| 1 |  1 |   1 |    7 |     7 |
-| 1 |  1 |   1 |    7 |     7 |
-| 1 |  1 |   1 |    7 |     7 |
-| 1 |  1 |   1 |    7 |     7 |
-| 1 |  1 |   1 |    7 |     7 |
-| 1 |  1 |   1 |    7 |     7 |
-| 1 |  1 |   1 |    7 |     7 |
-| 2 |  1 |   1 |    7 |     7 |
+| 1 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    0 |     0 |
+| 2 |  0 |   0 |    0 |     0 |
+| 1 |  0 |   0 |    2 |     2 |
+| 1 |  0 |   0 |    2 |     2 |
+| 1 |  0 |   0 |    2 |     2 |
+| 2 |  0 |   0 |    2 |     2 |
+| 2 |  0 |   0 |    2 |     2 |
+| 2 |  0 |   0 |    2 |     2 |
+| 9 |  1 |   1 |    7 |     7 |
 +---+----+-----+------+-------+
-(8 rows)
+(15 rows)
 
 !ok
 
 # GROUPING in ORDER BY clause
 select count(*) as c
 from emp
-group by deptno
-order by grouping(deptno);
+group by rollup(deptno)
+order by grouping(deptno), c;
 +---+
 | C |
 +---+
 | 1 |
-| 2 |
 | 1 |
-| 2 |
 | 1 |
 | 2 |
+| 2 |
+| 2 |
+| 9 |
 +---+
-(6 rows)
+(7 rows)
 
 !ok
 
@@ -605,27 +655,28 @@ group by rollup(deptno);
 select deptno, gender, grouping(deptno) gd, grouping(gender) gg,
   grouping_id(deptno, gender) dg, grouping_id(gender, deptno) gd,
   group_id() gid, count(*) c
-from emp group by cube(deptno, gender);
+from emp
+group by cube(deptno, gender);
 +--------+--------+----+----+----+----+-----+---+
 | DEPTNO | GENDER | GD | GG | DG | GD | GID | C |
 +--------+--------+----+----+----+----+-----+---+
 |     10 | F      |  0 |  0 |  0 |  0 |   0 | 1 |
 |     10 | M      |  0 |  0 |  0 |  0 |   0 | 1 |
-|     10 |        |  0 |  1 |  1 |  2 |   1 | 2 |
 |     20 | M      |  0 |  0 |  0 |  0 |   0 | 1 |
-|     20 |        |  0 |  1 |  1 |  2 |   1 | 1 |
 |     30 | F      |  0 |  0 |  0 |  0 |   0 | 2 |
-|     30 |        |  0 |  1 |  1 |  2 |   1 | 2 |
 |     50 | F      |  0 |  0 |  0 |  0 |   0 | 1 |
 |     50 | M      |  0 |  0 |  0 |  0 |   0 | 1 |
-|     50 |        |  0 |  1 |  1 |  2 |   1 | 2 |
 |     60 | F      |  0 |  0 |  0 |  0 |   0 | 1 |
-|     60 |        |  0 |  1 |  1 |  2 |   1 | 1 |
 |        | F      |  0 |  0 |  0 |  0 |   0 | 1 |
-|        | F      |  1 |  0 |  2 |  1 |   2 | 6 |
-|        | M      |  1 |  0 |  2 |  1 |   2 | 3 |
-|        |        |  0 |  1 |  1 |  2 |   1 | 1 |
-|        |        |  1 |  1 |  3 |  3 |   3 | 9 |
+|        |        |  1 |  1 |  3 |  3 |   0 | 9 |
+|     10 |        |  0 |  1 |  1 |  2 |   0 | 2 |
+|     20 |        |  0 |  1 |  1 |  2 |   0 | 1 |
+|     30 |        |  0 |  1 |  1 |  2 |   0 | 2 |
+|     50 |        |  0 |  1 |  1 |  2 |   0 | 2 |
+|     60 |        |  0 |  1 |  1 |  2 |   0 | 1 |
+|        | F      |  1 |  0 |  2 |  1 |   0 | 6 |
+|        | M      |  1 |  0 |  2 |  1 |   0 | 3 |
+|        |        |  0 |  1 |  1 |  2 |   0 | 1 |
 +--------+--------+----+----+----+----+-----+---+
 (17 rows)
 
@@ -720,6 +771,142 @@ group by rollup(1);
 
 !use scott
 
+# When
+#   [CALCITE-1824] GROUP_ID returns wrong result
+# is fixed, there will be an extra row (null, 1, 14).
+select deptno, group_id() as g, count(*) as c
+from "scott".emp
+group by grouping sets (deptno, (), ());
+
++--------+---+----+
+| DEPTNO | G | C  |
++--------+---+----+
+|     10 | 0 |  3 |
+|     20 | 0 |  5 |
+|     30 | 0 |  6 |
+|        | 0 | 14 |
++--------+---+----+
+(4 rows)
+
+!ok
+
+# From http://rwijk.blogspot.com/2008/12/groupid.html
+select deptno
+       , job
+       , empno
+       , ename
+       , sum(sal) sumsal
+       , case grouping_id(deptno,job,empno)
+           when 0 then 'grouped by deptno,job,empno,ename'
+           when 1 then 'grouped by deptno,job'
+           when 3 then 'grouped by deptno'
+           when 7 then 'grouped by ()'
+         end gr_text
+    from "scott".emp
+   group by rollup(deptno,job,(empno,ename))
+   order by deptno
+       , job
+       , empno;
+
++--------+-----------+-------+--------+----------+-----------------------------------+
+| DEPTNO | JOB       | EMPNO | ENAME  | SUMSAL   | GR_TEXT                           |
++--------+-----------+-------+--------+----------+-----------------------------------+
+|     10 | CLERK     |  7934 | MILLER |  1300.00 | grouped by deptno,job,empno,ename |
+|     10 | CLERK     |       |        |  1300.00 | grouped by deptno,job             |
+|     10 | MANAGER   |  7782 | CLARK  |  2450.00 | grouped by deptno,job,empno,ename |
+|     10 | MANAGER   |       |        |  2450.00 | grouped by deptno,job             |
+|     10 | PRESIDENT |  7839 | KING   |  5000.00 | grouped by deptno,job,empno,ename |
+|     10 | PRESIDENT |       |        |  5000.00 | grouped by deptno,job             |
+|     10 |           |       |        |  8750.00 | grouped by deptno                 |
+|     20 | ANALYST   |  7788 | SCOTT  |  3000.00 | grouped by deptno,job,empno,ename |
+|     20 | ANALYST   |  7902 | FORD   |  3000.00 | grouped by deptno,job,empno,ename |
+|     20 | ANALYST   |       |        |  6000.00 | grouped by deptno,job             |
+|     20 | CLERK     |  7369 | SMITH  |   800.00 | grouped by deptno,job,empno,ename |
+|     20 | CLERK     |  7876 | ADAMS  |  1100.00 | grouped by deptno,job,empno,ename |
+|     20 | CLERK     |       |        |  1900.00 | grouped by deptno,job             |
+|     20 | MANAGER   |  7566 | JONES  |  2975.00 | grouped by deptno,job,empno,ename |
+|     20 | MANAGER   |       |        |  2975.00 | grouped by deptno,job             |
+|     20 |           |       |        | 10875.00 | grouped by deptno                 |
+|     30 | CLERK     |  7900 | JAMES  |   950.00 | grouped by deptno,job,empno,ename |
+|     30 | CLERK     |       |        |   950.00 | grouped by deptno,job             |
+|     30 | MANAGER   |  7698 | BLAKE  |  2850.00 | grouped by deptno,job,empno,ename |
+|     30 | MANAGER   |       |        |  2850.00 | grouped by deptno,job             |
+|     30 | SALESMAN  |  7499 | ALLEN  |  1600.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |  7521 | WARD   |  1250.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |  7654 | MARTIN |  1250.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |  7844 | TURNER |  1500.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |       |        |  5600.00 | grouped by deptno,job             |
+|     30 |           |       |        |  9400.00 | grouped by deptno                 |
+|        |           |       |        | 29025.00 | grouped by ()                     |
++--------+-----------+-------+--------+----------+-----------------------------------+
+(27 rows)
+
+!ok
+
+# From http://rwijk.blogspot.com/2008/12/groupid.html
+# (replacing "to_char(...)" with "cast(... as varchar)")
+# The current results are incorrect. When
+#   [CALCITE-1824] GROUP_ID returns wrong result
+# is fixed, there will be 4 more rows.
+select deptno
+       , job
+       , empno
+       , ename
+       , sum(sal) sumsal
+       , case grouping_id(deptno,job,empno)
+           when 0 then 'grouped by deptno,job,empno,ename'
+           when 1 then 'grouped by deptno,job'
+           when 3 then 'grouped by deptno, grouping set ' || cast(3+group_id() as varchar)
+           when 7 then 'grouped by (), grouping set ' || cast(5+group_id() as varchar)
+         end gr_text
+    from "scott".emp
+   group by grouping sets
+         ( (deptno,job,empno,ename)
+         , (deptno,job)
+         , deptno
+         , deptno
+         , ()
+         , ()
+         )
+   order by deptno
+       , job
+       , empno;
+
++--------+-----------+-------+--------+----------+-----------------------------------+
+| DEPTNO | JOB       | EMPNO | ENAME  | SUMSAL   | GR_TEXT                           |
++--------+-----------+-------+--------+----------+-----------------------------------+
+|     10 | CLERK     |  7934 | MILLER |  1300.00 | grouped by deptno,job,empno,ename |
+|     10 | CLERK     |       |        |  1300.00 | grouped by deptno,job             |
+|     10 | MANAGER   |  7782 | CLARK  |  2450.00 | grouped by deptno,job,empno,ename |
+|     10 | MANAGER   |       |        |  2450.00 | grouped by deptno,job             |
+|     10 | PRESIDENT |  7839 | KING   |  5000.00 | grouped by deptno,job,empno,ename |
+|     10 | PRESIDENT |       |        |  5000.00 | grouped by deptno,job             |
+|     10 |           |       |        |  8750.00 | grouped by deptno, grouping set 3 |
+|     20 | ANALYST   |  7788 | SCOTT  |  3000.00 | grouped by deptno,job,empno,ename |
+|     20 | ANALYST   |  7902 | FORD   |  3000.00 | grouped by deptno,job,empno,ename |
+|     20 | ANALYST   |       |        |  6000.00 | grouped by deptno,job             |
+|     20 | CLERK     |  7369 | SMITH  |   800.00 | grouped by deptno,job,empno,ename |
+|     20 | CLERK     |  7876 | ADAMS  |  1100.00 | grouped by deptno,job,empno,ename |
+|     20 | CLERK     |       |        |  1900.00 | grouped by deptno,job             |
+|     20 | MANAGER   |  7566 | JONES  |  2975.00 | grouped by deptno,job,empno,ename |
+|     20 | MANAGER   |       |        |  2975.00 | grouped by deptno,job             |
+|     20 |           |       |        | 10875.00 | grouped by deptno, grouping set 3 |
+|     30 | CLERK     |  7900 | JAMES  |   950.00 | grouped by deptno,job,empno,ename |
+|     30 | CLERK     |       |        |   950.00 | grouped by deptno,job             |
+|     30 | MANAGER   |  7698 | BLAKE  |  2850.00 | grouped by deptno,job,empno,ename |
+|     30 | MANAGER   |       |        |  2850.00 | grouped by deptno,job             |
+|     30 | SALESMAN  |  7499 | ALLEN  |  1600.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |  7521 | WARD   |  1250.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |  7654 | MARTIN |  1250.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |  7844 | TURNER |  1500.00 | grouped by deptno,job,empno,ename |
+|     30 | SALESMAN  |       |        |  5600.00 | grouped by deptno,job             |
+|     30 |           |       |        |  9400.00 | grouped by deptno, grouping set 3 |
+|        |           |       |        | 29025.00 | grouped by (), grouping set 5     |
++--------+-----------+-------+--------+----------+-----------------------------------+
+(27 rows)
+
+!ok
+
 # [KYLIN-751] Max on negative double values is not working
 # [CALCITE-735] Primitive.DOUBLE.min should be large and negative
 select max(v) as x, min(v) as n
@@ -1834,9 +2021,9 @@ select count(distinct EMPNO), COUNT(SAL), MIN(SAL), MAX(SAL) from "scott".emp;
 !ok
 
 EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):BIGINT NOT NULL], EXPR$0=[$t0], EXPR$1=[$t4], EXPR$2=[$t2], EXPR$3=[$t3])
-  EnumerableAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $5], EXPR$1=[MIN($2) FILTER $6], EXPR$2=[MIN($3) FILTER $6], EXPR$3=[MIN($4) FILTER $6])
-    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[1], expr#7=[CASE($t1, $t5, $t6)], expr#8=[=($t7, $t6)], expr#9=[=($t7, $t5)], proj#0..4=[{exprs}], $i0=[$t8], $=[$t9])
-      EnumerableAggregate(group=[{0}], groups=[[{0}, {}]], indicator=[true], EXPR$1=[COUNT($5)], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)])
+  EnumerableAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $4], EXPR$1=[MIN($1) FILTER $5], EXPR$2=[MIN($2) FILTER $5], EXPR$3=[MIN($3) FILTER $5])
+    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t4, $t5)], expr#7=[1], expr#8=[=($t4, $t7)], proj#0..3=[{exprs}], $g_0=[$t6], $g_1=[$t8])
+      EnumerableAggregate(group=[{0}], groups=[[{0}, {}]], EXPR$1=[COUNT($5)], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)], $g=[GROUPING($0)])
         EnumerableTableScan(table=[[scott, EMP]])
 !plan
 
@@ -1852,9 +2039,9 @@ select count(distinct DEPTNO), COUNT(JOB), MIN(SAL), MAX(SAL) from "scott".emp;
 !ok
 
 EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):BIGINT NOT NULL], EXPR$0=[$t0], EXPR$1=[$t4], EXPR$2=[$t2], EXPR$3=[$t3])
-  EnumerableAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $5], EXPR$1=[MIN($2) FILTER $6], EXPR$2=[MIN($3) FILTER $6], EXPR$3=[MIN($4) FILTER $6])
-    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[1], expr#7=[CASE($t1, $t5, $t6)], expr#8=[=($t7, $t6)], expr#9=[=($t7, $t5)], proj#0..4=[{exprs}], $i0=[$t8], $=[$t9])
-      EnumerableAggregate(group=[{7}], groups=[[{7}, {}]], indicator=[true], EXPR$1=[COUNT($2)], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)])
+  EnumerableAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $4], EXPR$1=[MIN($1) FILTER $5], EXPR$2=[MIN($2) FILTER $5], EXPR$3=[MIN($3) FILTER $5])
+    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t4, $t5)], expr#7=[1], expr#8=[=($t4, $t7)], proj#0..3=[{exprs}], $g_0=[$t6], $g_1=[$t8])
+      EnumerableAggregate(group=[{7}], groups=[[{7}, {}]], EXPR$1=[COUNT($2)], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)], $g=[GROUPING($7)])
         EnumerableTableScan(table=[[scott, EMP]])
 !plan
 
@@ -1876,9 +2063,9 @@ select MGR, count(distinct DEPTNO), COUNT(JOB), MIN(SAL), MAX(SAL) from "scott".
 !ok
 
 EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):BIGINT NOT NULL], proj#0..1=[{exprs}], EXPR$2=[$t5], EXPR$3=[$t3], EXPR$4=[$t4])
-  EnumerableAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $7], EXPR$2=[MIN($4) FILTER $8], EXPR$3=[MIN($5) FILTER $8], EXPR$4=[MIN($6) FILTER $8])
-    EnumerableCalc(expr#0..6=[{inputs}], expr#7=[0], expr#8=[1], expr#9=[CASE($t2, $t7, $t8)], expr#10=[2], expr#11=[CASE($t3, $t7, $t10)], expr#12=[+($t9, $t11)], expr#13=[3], expr#14=[=($t12, $t13)], expr#15=[=($t12, $t8)], proj#0..6=[{exprs}], $i0_1=[$t14], $i0=[$t15])
-      EnumerableAggregate(group=[{3, 7}], groups=[[{3, 7}, {3}]], indicator=[true], EXPR$2=[COUNT($2)], EXPR$3=[MIN($5)], EXPR$4=[MAX($5)])
+  EnumerableAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $5], EXPR$2=[MIN($2) FILTER $6], EXPR$3=[MIN($3) FILTER $6], EXPR$4=[MIN($4) FILTER $6])
+    EnumerableCalc(expr#0..5=[{inputs}], expr#6=[0], expr#7=[=($t5, $t6)], expr#8=[1], expr#9=[=($t5, $t8)], proj#0..4=[{exprs}], $g_0=[$t7], $g_1=[$t9])
+      EnumerableAggregate(group=[{3, 7}], groups=[[{3, 7}, {3}]], EXPR$2=[COUNT($2)], EXPR$3=[MIN($5)], EXPR$4=[MAX($5)], $g=[GROUPING($3, $7)])
         EnumerableTableScan(table=[[scott, EMP]])
 !plan
 
@@ -1899,9 +2086,9 @@ select MGR, count(distinct DEPTNO, JOB), MIN(SAL), MAX(SAL) from "scott".emp gro
 
 !ok
 
-EnumerableAggregate(group=[{1}], EXPR$1=[COUNT($2, $0) FILTER $8], EXPR$2=[MIN($6) FILTER $9], EXPR$3=[MIN($7) FILTER $9])
-  EnumerableCalc(expr#0..7=[{inputs}], expr#8=[0], expr#9=[1], expr#10=[CASE($t3, $t8, $t9)], expr#11=[2], expr#12=[CASE($t4, $t8, $t11)], expr#13=[+($t10, $t12)], expr#14=[4], expr#15=[CASE($t5, $t8, $t14)], expr#16=[+($t13, $t15)], expr#17=[7], expr#18=[=($t16, $t17)], expr#19=[=($t16, $t11)], proj#0..7=[{exprs}], $i2_3_7=[$t18], $i3=[$t19])
-    EnumerableAggregate(group=[{2, 3, 7}], groups=[[{2, 3, 7}, {3}]], indicator=[true], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)])
+EnumerableAggregate(group=[{1}], EXPR$1=[COUNT($2, $0) FILTER $5], EXPR$2=[MIN($3) FILTER $6], EXPR$3=[MIN($4) FILTER $6])
+  EnumerableCalc(expr#0..5=[{inputs}], expr#6=[0], expr#7=[=($t5, $t6)], expr#8=[5], expr#9=[=($t5, $t8)], proj#0..4=[{exprs}], $g_0=[$t7], $g_5=[$t9])
+    EnumerableAggregate(group=[{2, 3, 7}], groups=[[{2, 3, 7}, {3}]], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)], $g=[GROUPING($2, $3, $7)])
       EnumerableTableScan(table=[[scott, EMP]])
 !plan
 


[03/16] calcite git commit: [CALCITE-1967] Elasticsearch 5 adapter (Christian Beikov)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
new file mode 100644
index 0000000..636aa5f
--- /dev/null
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
@@ -0,0 +1,66 @@
+/*
+ * 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.adapter.elasticsearch2;
+
+import org.apache.calcite.adapter.elasticsearch.AbstractElasticsearchTable;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.apache.calcite.util.Util;
+
+import org.elasticsearch.client.Client;
+import org.elasticsearch.search.SearchHit;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Table based on an Elasticsearch2 type.
+ */
+public class Elasticsearch2Table extends AbstractElasticsearchTable {
+  private final Client client;
+
+  /**
+   * Creates an Elasticsearch2Table.
+   */
+  public Elasticsearch2Table(Client client, String indexName, String typeName) {
+    super(indexName, typeName);
+    this.client = client;
+  }
+
+  @Override protected Enumerable<Object> find(String index, List<String> ops,
+      List<Map.Entry<String, Class>> fields) {
+    final String dbName = index;
+
+    final String queryString = "{" + Util.toString(ops, "", ", ", "") + "}";
+
+    final Function1<SearchHit, Object> getter = Elasticsearch2Enumerator.getter(fields);
+
+    return new AbstractEnumerable<Object>() {
+      public Enumerator<Object> enumerator() {
+        final Iterator<SearchHit> cursor = client.prepareSearch(dbName).setTypes(typeName)
+            .setSource(queryString).execute().actionGet().getHits().iterator();
+        return new Elasticsearch2Enumerator(cursor, getter);
+      }
+    };
+  }
+}
+
+// End Elasticsearch2Table.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/package-info.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/package-info.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/package-info.java
new file mode 100644
index 0000000..a1c10c9
--- /dev/null
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * Query provider based on an Elasticsearch2 DB.
+ */
+@PackageMarker
+package org.apache.calcite.adapter.elasticsearch2;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java b/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java
new file mode 100644
index 0000000..4e0c2b6
--- /dev/null
+++ b/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableMap;
+
+import org.junit.Test;
+
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Tests for the {@code org.apache.calcite.adapter.elasticsearch2} package.
+ *
+ * <p>Before calling this test, you need to populate Elasticsearch, as follows:
+ *
+ * <blockquote><code>
+ * git clone https://github.com/vlsi/calcite-test-dataset<br>
+ * cd calcite-test-dataset<br>
+ * mvn install
+ * </code></blockquote>
+ *
+ * <p>This will create a virtual machine with Elasticsearch and the "zips" test
+ * dataset.
+ */
+public class Elasticsearch2AdapterIT {
+  /**
+   * Whether to run Elasticsearch tests. Enabled by default, however test is only
+   * included if "it" profile is activated ({@code -Pit}). To disable,
+   * specify {@code -Dcalcite.test.elasticsearch=false} on the Java command line.
+   */
+  private static final boolean ENABLED = Util.getBooleanProperty("calcite.test.elasticsearch",
+      true);
+
+  /** Connection factory based on the "zips-es" model. */
+  private static final ImmutableMap<String, String> ZIPS = ImmutableMap.of("model",
+      Elasticsearch2AdapterIT.class.getResource("/elasticsearch-zips-model.json").getPath());
+
+  /** Whether to run this test. */
+  private boolean enabled() {
+    return ENABLED;
+  }
+
+  /** Returns a function that checks that a particular Elasticsearch pipeline is
+   * generated to implement a query. */
+  private static Function<List, Void> elasticsearchChecker(final String... strings) {
+    return new Function<List, Void>() {
+      @Nullable
+      @Override public Void apply(@Nullable List actual) {
+        Object[] actualArray = actual == null || actual.isEmpty() ? null
+            : ((List) actual.get(0)).toArray();
+        CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
+            actualArray);
+        return null;
+      }
+    };
+  }
+
+  @Test public void testSort() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchSort(sort0=[$4], dir0=[ASC])\n"
+        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select * from zips order by \"state\"")
+        .returnsCount(10)
+        .explainContains(explain);
+  }
+
+  @Test public void testSortLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "order by \"state\", \"id\" offset 2 rows fetch next 3 rows only";
+    CalciteAssert.that()
+        .with(ZIPS)
+        .query(sql)
+        .returnsUnordered("state=AK; id=99503",
+            "state=AK; id=99504",
+            "state=AK; id=99505")
+        .queryContains(
+            elasticsearchChecker(
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}",
+                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]",
+                "\"from\": 2",
+                "\"size\": 3"));
+  }
+
+  @Test public void testOffsetLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "offset 2 fetch next 3 rows only";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(
+                "\"from\": 2",
+                "\"size\": 3",
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
+  }
+
+  @Test public void testLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "fetch next 3 rows only";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(
+                "\"size\": 3",
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
+  }
+
+  @Test public void testFilterSort() {
+    final String sql = "select * from zips\n"
+        + "where \"city\" = 'SPRINGFIELD' and \"id\" >= '70000'\n"
+        + "order by \"state\", \"id\"";
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
+        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "      ElasticsearchFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
+        + "        ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .returnsOrdered(
+            "city=SPRINGFIELD; longitude=-92.54567; latitude=35.274879; pop=752; state=AR; id=72157",
+            "city=SPRINGFIELD; longitude=-102.617322; latitude=37.406727; pop=1992; state=CO; id=81073",
+            "city=SPRINGFIELD; longitude=-90.577479; latitude=30.415738; pop=5597; state=LA; id=70462",
+            "city=SPRINGFIELD; longitude=-123.015259; latitude=44.06106; pop=32384; state=OR; id=97477",
+            "city=SPRINGFIELD; longitude=-122.917108; latitude=44.056056; pop=27521; state=OR; id=97478")
+        .queryContains(
+            elasticsearchChecker("\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                    + "{\"must\":[{\"term\":{\"city\":\"springfield\"}},{\"range\":{\"id\":{\"gte\":\"70000\"}}}]}}}}",
+                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}",
+                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]"))
+        .explainContains(explain);
+  }
+
+  @Test public void testFilterSortDesc() {
+    final String sql = "select * from zips\n"
+        + "where \"pop\" BETWEEN 20000 AND 20100\n"
+        + "order by \"state\" desc, \"pop\"";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .limit(4)
+        .returnsOrdered(
+            "city=SHERIDAN; longitude=-106.964795; latitude=44.78486; pop=20025; state=WY; id=82801",
+            "city=MOUNTLAKE TERRAC; longitude=-122.304036; latitude=47.793061; pop=20059; state=WA; id=98043",
+            "city=FALMOUTH; longitude=-77.404537; latitude=38.314557; pop=20039; state=VA; id=22405",
+            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104");
+  }
+
+  @Test public void testFilterRedundant() {
+    final String sql = "select * from zips\n"
+        + "where \"state\" > 'CA' and \"state\" < 'AZ' and \"state\" = 'OK'";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(""
+                + "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                + "{\"must\":[{\"term\":{\"state\":\"ok\"}}]}}}}",
+                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"));
+  }
+
+  @Test public void testInPlan() {
+    final String[] searches = {
+        "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":{\"should\":"
+          + "[{\"bool\":{\"must\":[{\"term\":{\"pop\":20012}}]}},{\"bool\":{\"must\":[{\"term\":"
+          + "{\"pop\":15590}}]}}]}}}}",
+        "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"
+    };
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select * from zips where \"pop\" in (20012, 15590)")
+        .returnsUnordered(
+            "city=COVINA; longitude=-117.884285; latitude=34.08596; pop=15590; state=CA; id=91723",
+            "city=ARLINGTON; longitude=-97.091987; latitude=32.654752; pop=15590; state=TX; id=76018",
+            "city=CROFTON; longitude=-76.680166; latitude=39.011163; pop=15590; state=MD; id=21114",
+            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104",
+            "city=DINUBA; longitude=-119.39087; latitude=36.534931; pop=20012; state=CA; id=93618")
+        .queryContains(elasticsearchChecker(searches));
+  }
+
+  @Test public void testZips() {
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips")
+        .returnsCount(10);
+  }
+
+  @Test public void testProject() {
+    final String sql = "select \"state\", \"city\", 0 as \"zero\"\n"
+        + "from zips\n"
+        + "order by \"state\", \"city\"";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .limit(2)
+        .returnsUnordered("state=AK; city=ELMENDORF AFB; zero=0",
+            "state=AK; city=EIELSON AFB; zero=0")
+        .queryContains(
+            elasticsearchChecker("\"sort\": [ {\"state\": \"asc\"}, {\"city\": \"asc\"}]",
+                "\"fields\" : [\"state\", \"city\"], \"script_fields\": {\"zero\":{\"script\": \"0\"}}"));
+  }
+
+  @Test public void testFilter() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchProject(state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "    ElasticsearchFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
+        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where \"state\" = 'CA'")
+        .limit(2)
+        .returnsUnordered("state=CA; city=LOS ANGELES",
+            "state=CA; city=LOS ANGELES")
+        .explainContains(explain);
+  }
+
+  @Test public void testFilterReversed() {
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where 'WI' < \"state\"")
+        .limit(2)
+        .returnsUnordered("state=WV; city=WELCH",
+            "state=WV; city=HANOVER");
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where \"state\" > 'WI'")
+        .limit(2)
+        .returnsUnordered("state=WV; city=WELCH",
+            "state=WV; city=HANOVER");
+  }
+}
+
+// End Elasticsearch2AdapterIT.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/test/resources/elasticsearch-zips-model.json
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/resources/elasticsearch-zips-model.json b/elasticsearch2/src/test/resources/elasticsearch-zips-model.json
new file mode 100644
index 0000000..6c00a9c
--- /dev/null
+++ b/elasticsearch2/src/test/resources/elasticsearch-zips-model.json
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+{
+  "version": "1.0",
+  "defaultSchema": "elasticsearch",
+  "schemas": [
+    {
+      "type": "custom",
+      "name": "elasticsearch_raw",
+      "factory": "org.apache.calcite.adapter.elasticsearch2.Elasticsearch2SchemaFactory",
+      "operand": {
+        "coordinates": "{'127.0.0.1': 9300}",
+        "userConfig": "{'bulk.flush.max.actions': 10, 'bulk.flush.max.size.mb': 1}",
+        "index": "usa"
+      }
+    },
+    {
+      "name": "elasticsearch",
+      "tables": [
+        {
+          "name": "ZIPS",
+          "type": "view",
+          "sql": [
+            "select cast(_MAP['city'] AS varchar(20)) AS \"city\",\n",
+            " cast(_MAP['loc'][0] AS float) AS \"longitude\",\n",
+            " cast(_MAP['loc'][1] AS float) AS \"latitude\",\n",
+            " cast(_MAP['pop'] AS integer) AS \"pop\",\n",
+            " cast(_MAP['state'] AS varchar(2)) AS \"state\",\n",
+            " cast(_MAP['id'] AS varchar(5)) AS \"id\"\n",
+            "from \"elasticsearch_raw\".\"zips\""
+          ]
+        }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/resources/log4j.properties b/elasticsearch2/src/test/resources/log4j.properties
new file mode 100644
index 0000000..834e2db
--- /dev/null
+++ b/elasticsearch2/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+# 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.
+
+# Root logger is configured at INFO and is sent to A1
+log4j.rootLogger=INFO, A1
+
+# A1 goes to the console
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# Set the pattern for each log message
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%d [%t] %-5p - %m%n

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/pom.xml
----------------------------------------------------------------------
diff --git a/elasticsearch5/pom.xml b/elasticsearch5/pom.xml
new file mode 100644
index 0000000..a599bac
--- /dev/null
+++ b/elasticsearch5/pom.xml
@@ -0,0 +1,153 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.calcite</groupId>
+    <artifactId>calcite</artifactId>
+    <version>1.14.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>calcite-elasticsearch5</artifactId>
+  <packaging>jar</packaging>
+  <version>1.14.0-SNAPSHOT</version>
+  <name>Calcite Elasticsearch5</name>
+  <description>Elasticsearch5 adapter for Calcite</description>
+
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <type>jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-linq4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.elasticsearch.client</groupId>
+      <artifactId>transport</artifactId>
+      <version>${elasticsearch5-java-driver.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.elasticsearch</groupId>
+      <artifactId>elasticsearch</artifactId>
+      <version>${elasticsearch5-java-driver.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.carrotsearch</groupId>
+      <artifactId>hppc</artifactId>
+      <version>${hppc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>${maven-dependency-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>org.apache.calcite.avatica:avatica</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-api</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-log4j12</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-release-plugin</artifactId>
+      </plugin>
+      <!-- Parent module has the same plugin and does the work of
+          generating -sources.jar for each project. But without the
+          plugin declared here, IDEs don't know the sources are
+          available. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>attach-sources</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>jar-no-fork</goal>
+              <goal>test-jar-no-fork</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Enumerator.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Enumerator.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Enumerator.java
new file mode 100644
index 0000000..262058a
--- /dev/null
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Enumerator.java
@@ -0,0 +1,154 @@
+/*
+ * 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.adapter.elasticsearch5;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.Primitive;
+
+import org.elasticsearch.search.SearchHit;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Enumerator that reads from an Elasticsearch type.
+ */
+public class Elasticsearch5Enumerator implements Enumerator<Object> {
+  private final Iterator<SearchHit> cursor;
+  private final Function1<SearchHit, Object> getter;
+  private Object current;
+
+  /**
+   * Creates an Elasticsearch5Enumerator.
+   *
+   * @param cursor Iterator over Elasticsearch {@link SearchHit} objects
+   * @param getter Converts an object into a list of fields
+   */
+  public Elasticsearch5Enumerator(Iterator<SearchHit> cursor,
+      Function1<SearchHit, Object> getter) {
+    this.cursor = cursor;
+    this.getter = getter;
+  }
+
+  public Object current() {
+    return current;
+  }
+
+  public boolean moveNext() {
+    if (cursor.hasNext()) {
+      SearchHit map = cursor.next();
+      current = getter.apply(map);
+      return true;
+    } else {
+      current = null;
+      return false;
+    }
+  }
+
+  public void reset() {
+    throw new UnsupportedOperationException();
+  }
+
+  public void close() {
+    // nothing to do
+  }
+
+  private static Function1<SearchHit, Map> mapGetter() {
+    return new Function1<SearchHit, Map>() {
+      public Map apply(SearchHit searchHitFields) {
+        return (Map) searchHitFields.getFields();
+      }
+    };
+  }
+
+  private static Function1<SearchHit, Object> singletonGetter(final String fieldName,
+      final Class fieldClass) {
+    return new Function1<SearchHit, Object>() {
+      public Object apply(SearchHit searchHitFields) {
+        if (searchHitFields.getFields().isEmpty()) {
+          return convert(searchHitFields.getSource(), fieldClass);
+        } else {
+          return convert(searchHitFields.getFields(), fieldClass);
+        }
+      }
+    };
+  }
+
+  /**
+   * Function that extracts a given set of fields from {@link SearchHit}
+   * objects.
+   *
+   * @param fields List of fields to project
+   */
+  private static Function1<SearchHit, Object[]> listGetter(
+      final List<Map.Entry<String, Class>> fields) {
+    return new Function1<SearchHit, Object[]>() {
+      public Object[] apply(SearchHit searchHitFields) {
+        Object[] objects = new Object[fields.size()];
+        for (int i = 0; i < fields.size(); i++) {
+          final Map.Entry<String, Class> field = fields.get(i);
+          final String name = field.getKey();
+          if (searchHitFields.getFields().isEmpty()) {
+            objects[i] = convert(searchHitFields.getSource().get(name),
+                field.getValue());
+          } else {
+            objects[i] = convert(searchHitFields.getField(name).getValue(),
+                field.getValue());
+          }
+        }
+        return objects;
+      }
+    };
+  }
+
+  static Function1<SearchHit, Object> getter(List<Map.Entry<String, Class>> fields) {
+    //noinspection unchecked
+    return fields == null
+      ? (Function1) mapGetter()
+      : fields.size() == 1
+      ? singletonGetter(fields.get(0).getKey(), fields.get(0).getValue())
+      : (Function1) listGetter(fields);
+  }
+
+  private static Object convert(Object o, Class clazz) {
+    if (o == null) {
+      return null;
+    }
+    Primitive primitive = Primitive.of(clazz);
+    if (primitive != null) {
+      clazz = primitive.boxClass;
+    } else {
+      primitive = Primitive.ofBox(clazz);
+    }
+    if (clazz.isInstance(o)) {
+      return o;
+    }
+    if (o instanceof Date && primitive != null) {
+      o = ((Date) o).getTime() / DateTimeUtils.MILLIS_PER_DAY;
+    }
+    if (o instanceof Number && primitive != null) {
+      return primitive.number((Number) o);
+    }
+    return o;
+  }
+}
+
+// End Elasticsearch5Enumerator.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
new file mode 100644
index 0000000..6e16f2f
--- /dev/null
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
@@ -0,0 +1,138 @@
+/*
+ * 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.adapter.elasticsearch5;
+
+import org.apache.calcite.adapter.elasticsearch.ElasticsearchSchema;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
+import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
+import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.cluster.metadata.MappingMetaData;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.collect.ImmutableOpenMap;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.transport.client.PreBuiltTransportClient;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Schema mapped onto an index of ELASTICSEARCH types.
+ *
+ * <p>Each table in the schema is an ELASTICSEARCH type in that index.
+ */
+public class Elasticsearch5Schema extends AbstractSchema
+    implements ElasticsearchSchema {
+  final String index;
+
+  private transient Client client;
+
+  /**
+   * Creates an Elasticsearch5 schema.
+   *
+   * @param coordinates Map of Elasticsearch node locations (host, port)
+   * @param userConfig Map of user-specified configurations
+   * @param indexName Elasticsearch database name, e.g. "usa".
+   */
+  Elasticsearch5Schema(Map<String, Integer> coordinates,
+      Map<String, String> userConfig, String indexName) {
+    super();
+
+    final List<InetSocketAddress> transportAddresses = new ArrayList<>();
+    for (Map.Entry<String, Integer> coordinate: coordinates.entrySet()) {
+      transportAddresses.add(
+          new InetSocketAddress(coordinate.getKey(), coordinate.getValue()));
+    }
+
+    open(transportAddresses, userConfig);
+
+    if (client != null) {
+      final String[] indices = client.admin().indices()
+          .getIndex(new GetIndexRequest().indices(indexName))
+          .actionGet().getIndices();
+      if (indices.length == 1) {
+        index = indices[0];
+      } else {
+        index = null;
+      }
+    } else {
+      index = null;
+    }
+  }
+
+  @Override protected Map<String, Table> getTableMap() {
+    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+
+    try {
+      final GetMappingsResponse response = client.admin().indices()
+          .getMappings(new GetMappingsRequest().indices(index))
+          .get();
+      ImmutableOpenMap<String, MappingMetaData> mapping =
+          response.getMappings().get(index);
+      for (ObjectObjectCursor<String, MappingMetaData> c : mapping) {
+        builder.put(c.key, new Elasticsearch5Table(client, index, c.key));
+      }
+    } catch (RuntimeException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return builder.build();
+  }
+
+  private void open(List<InetSocketAddress> transportAddresses,
+      Map<String, String> userConfig) {
+    final List<TransportAddress> transportNodes = new ArrayList<>(transportAddresses.size());
+    for (InetSocketAddress address : transportAddresses) {
+      transportNodes.add(new InetSocketTransportAddress(address));
+    }
+
+    Settings settings = Settings.builder().put(userConfig).build();
+
+    final TransportClient transportClient = new PreBuiltTransportClient(settings);
+    for (TransportAddress transport : transportNodes) {
+      transportClient.addTransportAddress(transport);
+    }
+
+    final List<DiscoveryNode> nodes =
+        ImmutableList.copyOf(transportClient.connectedNodes());
+    if (nodes.isEmpty()) {
+      throw new RuntimeException("Cannot connect to any elasticsearch nodes");
+    }
+
+    client = transportClient;
+  }
+
+  @Override public String getIndex() {
+    return index;
+  }
+}
+
+// End Elasticsearch5Schema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5SchemaFactory.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5SchemaFactory.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5SchemaFactory.java
new file mode 100644
index 0000000..b573d37
--- /dev/null
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5SchemaFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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.adapter.elasticsearch5;
+
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Factory that creates an {@link Elasticsearch5Schema}.
+ *
+ * <p>Allows a custom schema to be included in a model.json file.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public class Elasticsearch5SchemaFactory implements SchemaFactory {
+
+  public Elasticsearch5SchemaFactory() {
+  }
+
+  @Override public Schema create(SchemaPlus parentSchema, String name,
+      Map<String, Object> operand) {
+    final Map map = (Map) operand;
+
+    final ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true);
+
+    try {
+      final Map<String, Integer> coordinates =
+          mapper.readValue((String) map.get("coordinates"),
+              new TypeReference<Map<String, Integer>>() { });
+      final Map<String, String> userConfig =
+          mapper.readValue((String) map.get("userConfig"),
+              new TypeReference<Map<String, String>>() { });
+      final String index = (String) map.get("index");
+      return new Elasticsearch5Schema(coordinates, userConfig, index);
+    } catch (IOException e) {
+      throw new RuntimeException("Cannot parse values from json", e);
+    }
+  }
+}
+
+// End Elasticsearch5SchemaFactory.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
new file mode 100644
index 0000000..175e3cc
--- /dev/null
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
@@ -0,0 +1,86 @@
+/*
+ * 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.adapter.elasticsearch5;
+
+import org.apache.calcite.adapter.elasticsearch.AbstractElasticsearchTable;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.apache.calcite.util.Util;
+
+import org.elasticsearch.client.Client;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.common.xcontent.XContent;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.common.xcontent.json.JsonXContent;
+import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Table based on an Elasticsearch5 type.
+ */
+public class Elasticsearch5Table extends AbstractElasticsearchTable {
+  private final Client client;
+
+  /**
+   * Creates an Elasticsearch5Table.
+   */
+  public Elasticsearch5Table(Client client, String indexName, String typeName) {
+    super(indexName, typeName);
+    this.client = client;
+  }
+
+  @Override protected Enumerable<Object> find(String index, List<String> ops,
+      List<Map.Entry<String, Class>> fields) {
+    final String dbName = index;
+
+    final SearchSourceBuilder searchSourceBuilder;
+    if (ops.isEmpty()) {
+      searchSourceBuilder = new SearchSourceBuilder();
+    } else {
+      String queryString = "{" + Util.toString(ops, "", ", ", "") + "}";
+      NamedXContentRegistry xContentRegistry = NamedXContentRegistry.EMPTY;
+      XContent xContent = JsonXContent.jsonXContent;
+      try (XContentParser parser = xContent.createParser(xContentRegistry, queryString)) {
+        final QueryParseContext queryParseContext = new QueryParseContext(parser);
+        searchSourceBuilder = SearchSourceBuilder.fromXContent(queryParseContext);
+      } catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    }
+    final Function1<SearchHit, Object> getter = Elasticsearch5Enumerator.getter(fields);
+
+    return new AbstractEnumerable<Object>() {
+      public Enumerator<Object> enumerator() {
+        final Iterator<SearchHit> cursor = client.prepareSearch(dbName).setTypes(typeName)
+            .setSource(searchSourceBuilder)
+            .execute().actionGet().getHits().iterator();
+        return new Elasticsearch5Enumerator(cursor, getter);
+      }
+    };
+  }
+}
+
+// End Elasticsearch5Table.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/package-info.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/package-info.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/package-info.java
new file mode 100644
index 0000000..70c4695
--- /dev/null
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * Query provider based on an Elasticsearch5 DB.
+ */
+@PackageMarker
+package org.apache.calcite.adapter.elasticsearch5;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java b/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java
new file mode 100644
index 0000000..1cf52fe
--- /dev/null
+++ b/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableMap;
+
+import org.junit.Test;
+
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Tests for the {@code org.apache.calcite.adapter.elasticsearch} package.
+ *
+ * <p>Before calling this test, you need to populate Elasticsearch, as follows:
+ *
+ * <blockquote><code>
+ * git clone https://github.com/vlsi/calcite-test-dataset<br>
+ * cd calcite-test-dataset<br>
+ * mvn install
+ * </code></blockquote>
+ *
+ * <p>This will create a virtual machine with Elasticsearch and the "zips" test
+ * dataset.
+ */
+public class Elasticsearch5AdapterIT {
+  /**
+   * Whether to run Elasticsearch tests. Enabled by default, however test is only
+   * included if "it" profile is activated ({@code -Pit}). To disable,
+   * specify {@code -Dcalcite.test.elasticsearch=false} on the Java command line.
+   */
+  private static final boolean ENABLED = Util.getBooleanProperty("calcite.test.elasticsearch",
+      true);
+
+  /** Connection factory based on the "zips-es" model. */
+  private static final ImmutableMap<String, String> ZIPS = ImmutableMap.of("model",
+      Elasticsearch5AdapterIT.class.getResource("/elasticsearch-zips-model.json").getPath());
+
+  /** Whether to run this test. */
+  private boolean enabled() {
+    return ENABLED;
+  }
+
+  /** Returns a function that checks that a particular Elasticsearch pipeline is
+   * generated to implement a query. */
+  private static Function<List, Void> elasticsearchChecker(final String... strings) {
+    return new Function<List, Void>() {
+      @Nullable
+      @Override public Void apply(@Nullable List actual) {
+        Object[] actualArray = actual == null || actual.isEmpty() ? null
+            : ((List) actual.get(0)).toArray();
+        CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
+            actualArray);
+        return null;
+      }
+    };
+  }
+
+  @Test public void testSort() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchSort(sort0=[$4], dir0=[ASC])\n"
+        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select * from zips order by \"state\"")
+        .returnsCount(10)
+        .explainContains(explain);
+  }
+
+  @Test public void testSortLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "order by \"state\", \"id\" offset 2 rows fetch next 3 rows only";
+    CalciteAssert.that()
+        .with(ZIPS)
+        .query(sql)
+        .returnsUnordered("state=AK; id=99503",
+            "state=AK; id=99504",
+            "state=AK; id=99505")
+        .queryContains(
+            elasticsearchChecker(
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}",
+                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]",
+                "\"from\": 2",
+                "\"size\": 3"));
+  }
+
+  @Test public void testOffsetLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "offset 2 fetch next 3 rows only";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(
+                "\"from\": 2",
+                "\"size\": 3",
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
+  }
+
+  @Test public void testLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "fetch next 3 rows only";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(
+                "\"size\": 3",
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
+  }
+
+  @Test public void testFilterSort() {
+    final String sql = "select * from zips\n"
+        + "where \"city\" = 'SPRINGFIELD' and \"id\" >= '70000'\n"
+        + "order by \"state\", \"id\"";
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
+        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "      ElasticsearchFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
+        + "        ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .returnsOrdered(
+            "city=SPRINGFIELD; longitude=-92.54567; latitude=35.274879; pop=752; state=AR; id=72157",
+            "city=SPRINGFIELD; longitude=-102.617322; latitude=37.406727; pop=1992; state=CO; id=81073",
+            "city=SPRINGFIELD; longitude=-90.577479; latitude=30.415738; pop=5597; state=LA; id=70462",
+            "city=SPRINGFIELD; longitude=-123.015259; latitude=44.06106; pop=32384; state=OR; id=97477",
+            "city=SPRINGFIELD; longitude=-122.917108; latitude=44.056056; pop=27521; state=OR; id=97478")
+        .queryContains(
+            elasticsearchChecker("\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                    + "{\"must\":[{\"term\":{\"city\":\"springfield\"}},{\"range\":{\"id\":{\"gte\":\"70000\"}}}]}}}}",
+                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}",
+                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]"))
+        .explainContains(explain);
+  }
+
+  @Test public void testFilterSortDesc() {
+    final String sql = "select * from zips\n"
+        + "where \"pop\" BETWEEN 20000 AND 20100\n"
+        + "order by \"state\" desc, \"pop\"";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .limit(4)
+        .returnsOrdered(
+            "city=SHERIDAN; longitude=-106.964795; latitude=44.78486; pop=20025; state=WY; id=82801",
+            "city=MOUNTLAKE TERRAC; longitude=-122.304036; latitude=47.793061; pop=20059; state=WA; id=98043",
+            "city=FALMOUTH; longitude=-77.404537; latitude=38.314557; pop=20039; state=VA; id=22405",
+            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104");
+  }
+
+  @Test public void testFilterRedundant() {
+    final String sql = "select * from zips\n"
+        + "where \"state\" > 'CA' and \"state\" < 'AZ' and \"state\" = 'OK'";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(""
+                + "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                + "{\"must\":[{\"term\":{\"state\":\"ok\"}}]}}}}",
+                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"));
+  }
+
+  @Test public void testInPlan() {
+    final String[] searches = {
+        "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":{\"should\":"
+          + "[{\"bool\":{\"must\":[{\"term\":{\"pop\":20012}}]}},{\"bool\":{\"must\":[{\"term\":"
+          + "{\"pop\":15590}}]}}]}}}}",
+        "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"
+    };
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select * from zips where \"pop\" in (20012, 15590)")
+        .returnsUnordered(
+            "city=COVINA; longitude=-117.884285; latitude=34.08596; pop=15590; state=CA; id=91723",
+            "city=ARLINGTON; longitude=-97.091987; latitude=32.654752; pop=15590; state=TX; id=76018",
+            "city=CROFTON; longitude=-76.680166; latitude=39.011163; pop=15590; state=MD; id=21114",
+            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104",
+            "city=DINUBA; longitude=-119.39087; latitude=36.534931; pop=20012; state=CA; id=93618")
+        .queryContains(elasticsearchChecker(searches));
+  }
+
+  @Test public void testZips() {
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips")
+        .returnsCount(10);
+  }
+
+  @Test public void testProject() {
+    final String sql = "select \"state\", \"city\", 0 as \"zero\"\n"
+        + "from zips\n"
+        + "order by \"state\", \"city\"";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .limit(2)
+        .returnsUnordered("state=AK; city=ELMENDORF AFB; zero=0",
+            "state=AK; city=EIELSON AFB; zero=0")
+        .queryContains(
+            elasticsearchChecker("\"sort\": [ {\"state\": \"asc\"}, {\"city\": \"asc\"}]",
+                "\"fields\" : [\"state\", \"city\"], \"script_fields\": {\"zero\":{\"script\": \"0\"}}"));
+  }
+
+  @Test public void testFilter() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchProject(state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "    ElasticsearchFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
+        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where \"state\" = 'CA'")
+        .limit(2)
+        .returnsUnordered("state=CA; city=LOS ANGELES",
+            "state=CA; city=LOS ANGELES")
+        .explainContains(explain);
+  }
+
+  @Test public void testFilterReversed() {
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where 'WI' < \"state\"")
+        .limit(2)
+        .returnsUnordered("state=WV; city=WELCH",
+            "state=WV; city=HANOVER");
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where \"state\" > 'WI'")
+        .limit(2)
+        .returnsUnordered("state=WV; city=WELCH",
+            "state=WV; city=HANOVER");
+  }
+}
+
+// End Elasticsearch5AdapterIT.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/test/resources/elasticsearch-zips-model.json
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/test/resources/elasticsearch-zips-model.json b/elasticsearch5/src/test/resources/elasticsearch-zips-model.json
new file mode 100644
index 0000000..626923e
--- /dev/null
+++ b/elasticsearch5/src/test/resources/elasticsearch-zips-model.json
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+{
+  "version": "1.0",
+  "defaultSchema": "elasticsearch",
+  "schemas": [
+    {
+      "type": "custom",
+      "name": "elasticsearch_raw",
+      "factory": "org.apache.calcite.adapter.elasticsearch5.Elasticsearch5SchemaFactory",
+      "operand": {
+        "coordinates": "{'127.0.0.1': 9300}",
+        "userConfig": "{'bulk.flush.max.actions': 10, 'bulk.flush.max.size.mb': 1}",
+        "index": "usa"
+      }
+    },
+    {
+      "name": "elasticsearch",
+      "tables": [
+        {
+          "name": "ZIPS",
+          "type": "view",
+          "sql": [
+            "select cast(_MAP['city'] AS varchar(20)) AS \"city\",\n",
+            " cast(_MAP['loc'][0] AS float) AS \"longitude\",\n",
+            " cast(_MAP['loc'][1] AS float) AS \"latitude\",\n",
+            " cast(_MAP['pop'] AS integer) AS \"pop\",\n",
+            " cast(_MAP['state'] AS varchar(2)) AS \"state\",\n",
+            " cast(_MAP['id'] AS varchar(5)) AS \"id\"\n",
+            "from \"elasticsearch_raw\".\"zips\""
+          ]
+        }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch5/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/test/resources/log4j.properties b/elasticsearch5/src/test/resources/log4j.properties
new file mode 100644
index 0000000..834e2db
--- /dev/null
+++ b/elasticsearch5/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+# 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.
+
+# Root logger is configured at INFO and is sent to A1
+log4j.rootLogger=INFO, A1
+
+# A1 goes to the console
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# Set the pattern for each log message
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%d [%t] %-5p - %m%n

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3372f99..a8f4df3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -61,6 +61,7 @@ limitations under the License.
     <commons-lang3.version>3.2</commons-lang3.version>
     <commons-logging.version>1.1.3</commons-logging.version>
     <elasticsearch-java-driver.version>2.3.2</elasticsearch-java-driver.version>
+    <elasticsearch5-java-driver.version>5.5.2</elasticsearch5-java-driver.version>
     <findbugs.version>3.0.1</findbugs.version>
     <fmpp-maven-plugin.version>1.0</fmpp-maven-plugin.version>
     <foodmart-data-hsqldb.version>0.3</foodmart-data-hsqldb.version>
@@ -139,7 +140,8 @@ limitations under the License.
     <module>cassandra</module>
     <module>core</module>
     <module>druid</module>
-    <module>elasticsearch</module>
+    <module>elasticsearch2</module>
+    <module>elasticsearch5</module>
     <module>example</module>
     <module>file</module>
     <module>linq4j</module>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/site/_docs/adapter.md
----------------------------------------------------------------------
diff --git a/site/_docs/adapter.md b/site/_docs/adapter.md
index ef8e761..3987804 100644
--- a/site/_docs/adapter.md
+++ b/site/_docs/adapter.md
@@ -30,7 +30,10 @@ presenting the data as tables within a schema.
 * [Cassandra adapter](cassandra_adapter.html) (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/cassandra/package-summary.html">calcite-cassandra</a>)
 * CSV adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/csv/package-summary.html">example/csv</a>)
 * [Druid adapter](druid_adapter.html) (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/druid/package-summary.html">calcite-druid</a>)
-* [Elasticsearch adapter](elasticsearch_adapter.html) (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/elasticsearch/package-summary.html">calcite-elasticsearch</a>)
+* [Elasticsearch adapter](elasticsearch_adapter.html)
+  (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/elasticsearch2/package-summary.html">calcite-elasticsearch2</a>
+  and
+  <a href="{{ site.apiRoot }}/org/apache/calcite/adapter/elasticsearch5/package-summary.html">calcite-elasticsearch5</a>)
 * [File adapter](file_adapter.html) (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/file/package-summary.html">calcite-file</a>)
 * JDBC adapter (part of <a href="{{ site.apiRoot }}/org/apache/calcite/adapter/jdbc/package-summary.html">calcite-core</a>)
 * MongoDB adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/mongodb/package-summary.html">calcite-mongodb</a>)

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/site/_docs/elasticsearch_adapter.md
----------------------------------------------------------------------
diff --git a/site/_docs/elasticsearch_adapter.md b/site/_docs/elasticsearch_adapter.md
index d87d9e3..21f2459 100644
--- a/site/_docs/elasticsearch_adapter.md
+++ b/site/_docs/elasticsearch_adapter.md
@@ -47,7 +47,7 @@ A basic example of a model file is given below:
     {
       "type": "custom",
       "name": "elasticsearch",
-      "factory": "org.apache.calcite.adapter.elasticsearch.ElasticsearchSchemaFactory",
+      "factory": "org.apache.calcite.adapter.elasticsearch2.Elasticsearch2SchemaFactory",
       "operand": {
         "coordinates": "{'127.0.0.1': 9300}",
         "userConfig": "{'bulk.flush.max.actions': 10, 'bulk.flush.max.size.mb': 1}",
@@ -58,6 +58,9 @@ A basic example of a model file is given below:
 }
 {% endhighlight %}
 
+This adapter is targeted for Elasticsearch 2.x. To use Calcite with Elasticsearch 5.x+ you can use the factory
+of the adapter targeted for Elasticsearch 5.x: `org.apache.calcite.adapter.elasticsearch5.Elasticsearch5SchemaFactory`
+
 Assuming this file is stored as `model.json`, you can connect to
 Elasticsearch via [`sqlline`](https://github.com/julianhyde/sqlline) as
 follows:

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/sqlline
----------------------------------------------------------------------
diff --git a/sqlline b/sqlline
index 6e4b4e9..b0fc489 100755
--- a/sqlline
+++ b/sqlline
@@ -37,7 +37,7 @@ if [ ! -f target/fullclasspath.txt ]; then
 fi
 
 CP=
-for module in core cassandra druid elasticsearch file mongodb spark splunk example/csv example/function; do
+for module in core cassandra druid elasticsearch2 elasticsearch5 file mongodb spark splunk example/csv example/function; do
   CP=${CP}${module}/target/classes:
   CP=${CP}${module}/target/test-classes:
 done

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/sqlline.bat
----------------------------------------------------------------------
diff --git a/sqlline.bat b/sqlline.bat
index 99548de..53f7159 100644
--- a/sqlline.bat
+++ b/sqlline.bat
@@ -23,6 +23,6 @@
 :: Copy dependency jars on first call. (To force jar refresh, remove target\dependencies)
 if not exist target\dependencies (call mvn -B dependency:copy-dependencies -DoverWriteReleases=false -DoverWriteSnapshots=false -DoverWriteIfNewer=true -DoutputDirectory=target\dependencies)
 
-java -Xmx1G -cp ".\target\dependencies\*;core\target\dependencies\*;cassandra\target\dependencies\*;druid\target\dependencies\*;elasticsearch\target\dependencies\*;file\target\dependencies\*;mongodb\target\dependencies\*;spark\target\dependencies\*;splunk\target\dependencies\*" sqlline.SqlLine --verbose=true %*
+java -Xmx1G -cp ".\target\dependencies\*;core\target\dependencies\*;cassandra\target\dependencies\*;druid\target\dependencies\*;elasticsearch2\target\dependencies\*;elasticsearch5\target\dependencies\*;file\target\dependencies\*;mongodb\target\dependencies\*;spark\target\dependencies\*;splunk\target\dependencies\*" sqlline.SqlLine --verbose=true %*
 
 :: End sqlline.bat


[11/16] calcite git commit: Fix data race in RexCall.toString() (Christian Beikov)

Posted by mm...@apache.org.
Fix data race in RexCall.toString() (Christian Beikov)

Close apache/calcite#531


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

Branch: refs/heads/branch-1.14
Commit: ddae8411cf3fc92360da7e856dce9d3c6b1f7524
Parents: cc20ca1
Author: Christian Beikov <ch...@gmail.com>
Authored: Mon Aug 28 16:20:43 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 17:13:32 2017 -0700

----------------------------------------------------------------------
 core/src/main/java/org/apache/calcite/rex/RexCall.java | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ddae8411/core/src/main/java/org/apache/calcite/rex/RexCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCall.java b/core/src/main/java/org/apache/calcite/rex/RexCall.java
index 5a1f112..ac9e09e 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -93,11 +93,14 @@ public class RexCall extends RexNode {
   }
 
   public String toString() {
-    if (digest == null) {
-      digest = computeDigest(
+    // This data race is intentional
+    String localDigest = digest;
+    if (localDigest == null) {
+      localDigest = computeDigest(
           isA(SqlKind.CAST) || isA(SqlKind.NEW_SPECIFICATION));
+      digest = localDigest;
     }
-    return digest;
+    return localDigest;
   }
 
   public <R> R accept(RexVisitor<R> visitor) {


[15/16] calcite git commit: [CALCITE-1941] Refine interface Schema#snapshot()

Posted by mm...@apache.org.
[CALCITE-1941] Refine interface Schema#snapshot()


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

Branch: refs/heads/branch-1.14
Commit: 35209136c5b21b83b1d2a3d4180a121f58e3c2f6
Parents: 1e7ae1c
Author: maryannxue <ma...@gmail.com>
Authored: Wed Aug 30 13:13:48 2017 -0700
Committer: maryannxue <ma...@gmail.com>
Committed: Wed Aug 30 13:13:48 2017 -0700

----------------------------------------------------------------------
 .../apache/calcite/adapter/jdbc/JdbcSchema.java |  7 +--
 .../calcite/jdbc/CachingCalciteSchema.java      | 16 +++---
 .../apache/calcite/jdbc/CalciteConnection.java  |  4 ++
 .../calcite/jdbc/CalciteConnectionImpl.java     | 16 ++++--
 .../org/apache/calcite/jdbc/CalcitePrepare.java |  2 +-
 .../org/apache/calcite/jdbc/CalciteSchema.java  | 21 +++----
 .../apache/calcite/jdbc/CalciteStatement.java   |  4 --
 .../calcite/jdbc/SimpleCalciteSchema.java       |  7 ++-
 .../org/apache/calcite/model/JsonSchema.java    |  7 +--
 .../apache/calcite/prepare/RelOptTableImpl.java |  8 +--
 .../java/org/apache/calcite/schema/Schema.java  | 26 +--------
 .../apache/calcite/schema/SchemaVersion.java    | 42 ++++++++++++++
 .../calcite/schema/impl/AbstractSchema.java     |  7 +--
 .../calcite/schema/impl/DelegatingSchema.java   |  9 +--
 .../calcite/schema/impl/LongSchemaVersion.java  | 60 ++++++++++++++++++++
 .../java/org/apache/calcite/test/JdbcTest.java  | 22 ++-----
 16 files changed, 156 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
index 69a411c..31e262e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
@@ -28,6 +28,7 @@ import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaFactory;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlDialect;
@@ -163,11 +164,7 @@ public class JdbcSchema implements Schema {
     return false;
   }
 
-  public boolean contentsHaveChangedSince(long lastCheck, long now) {
-    return false;
-  }
-
-  public Schema snapshot(long now) {
+  public Schema snapshot(SchemaVersion version) {
     return new JdbcSchema(dataSource, dialect, convention, catalog, schema,
         tableMap);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java
index 3e64838..4c61a21 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java
@@ -18,6 +18,7 @@ package org.apache.calcite.jdbc;
 
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.util.NameMap;
@@ -214,12 +215,12 @@ class CachingCalciteSchema extends CalciteSchema {
     return null;
   }
 
-  protected CalciteSchema snapshot(CalciteSchema parent, long now) {
+  protected CalciteSchema snapshot(CalciteSchema parent, SchemaVersion version) {
     CalciteSchema snapshot = new CachingCalciteSchema(parent,
-        schema.snapshot(now), name, null, tableMap, latticeMap,
+        schema.snapshot(version), name, null, tableMap, latticeMap,
         functionMap, functionNames, nullaryFunctionMap, getPath());
     for (CalciteSchema subSchema : subSchemaMap.map().values()) {
-      CalciteSchema subSchemaSnapshot = subSchema.snapshot(snapshot, now);
+      CalciteSchema subSchemaSnapshot = subSchema.snapshot(snapshot, version);
       snapshot.subSchemaMap.put(subSchema.name, subSchemaSnapshot);
     }
     return snapshot;
@@ -247,17 +248,16 @@ class CachingCalciteSchema extends CalciteSchema {
    * @param <T> element type */
   private abstract class AbstractCached<T> implements Cached<T> {
     T t;
-    long checked = Long.MIN_VALUE;
+    boolean built = false;
 
     public T get(long now) {
       if (!CachingCalciteSchema.this.cache) {
         return build();
       }
-      if (checked == Long.MIN_VALUE
-          || schema.contentsHaveChangedSince(checked, now)) {
+      if (!built) {
         t = build();
       }
-      checked = now;
+      built = true;
       return t;
     }
 
@@ -265,7 +265,7 @@ class CachingCalciteSchema extends CalciteSchema {
       if (!enabled) {
         t = null;
       }
-      checked = Long.MIN_VALUE;
+      built = false;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
index 5301b3b..6fbc564 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
@@ -18,6 +18,7 @@ package org.apache.calcite.jdbc;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.jdbc.CalcitePrepare.Context;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.schema.SchemaPlus;
 
@@ -76,6 +77,9 @@ public interface CalciteConnection extends Connection, QueryProvider {
   String getSchema() throws SQLException;
 
   CalciteConnectionConfig config();
+
+  /** Creates a context for preparing a statement for execution. */
+  Context createPrepareContext();
 }
 
 // End CalciteConnection.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
index 94c0d19..85dfc65 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -31,6 +31,7 @@ import org.apache.calcite.avatica.UnregisteredDriver;
 import org.apache.calcite.avatica.remote.TypedValue;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.jdbc.CalcitePrepare.Context;
 import org.apache.calcite.linq4j.BaseQueryable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
@@ -47,8 +48,10 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.LongSchemaVersion;
 import org.apache.calcite.server.CalciteServer;
 import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.advise.SqlAdvisor;
@@ -139,6 +142,10 @@ abstract class CalciteConnectionImpl
     return new CalciteConnectionConfigImpl(info);
   }
 
+  public Context createPrepareContext() {
+    return new ContextImpl(this);
+  }
+
   /** Called after the constructor has completed and the model has been
    * loaded. */
   void init() {
@@ -193,7 +200,7 @@ abstract class CalciteConnectionImpl
       int resultSetHoldability) throws SQLException {
     try {
       final Meta.Signature signature =
-          parseQuery(query, new ContextImpl(this), -1);
+          parseQuery(query, createPrepareContext(), -1);
       final CalcitePreparedStatement calcitePreparedStatement =
           (CalcitePreparedStatement) factory.newPreparedStatement(this, null,
               signature, resultSetType, resultSetConcurrency, resultSetHoldability);
@@ -463,7 +470,8 @@ abstract class CalciteConnectionImpl
     ContextImpl(CalciteConnectionImpl connection) {
       this.connection = Preconditions.checkNotNull(connection);
       long now = System.currentTimeMillis();
-      this.rootSchema = connection.rootSchema.createSnapshot(now);
+      SchemaVersion schemaVersion = new LongSchemaVersion(now);
+      this.rootSchema = connection.rootSchema.createSnapshot(schemaVersion);
     }
 
     public JavaTypeFactory getTypeFactory() {
@@ -532,8 +540,8 @@ abstract class CalciteConnectionImpl
       this.connection = Preconditions.checkNotNull(connection);
     }
 
-    public ContextImpl createPrepareContext() {
-      return new ContextImpl(connection);
+    public Context createPrepareContext() {
+      return connection.createPrepareContext();
     }
 
     public CalciteConnection getConnection() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
index 790e5da..56c5385 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
@@ -124,7 +124,7 @@ public interface CalcitePrepare {
      *
      * <p>The object is being analyzed is typically a view. If it is already
      * being analyzed further up the stack, the view definition can be deduced
-     * to be cylic. */
+     * to be cyclic. */
     List<String> getObjectPath();
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
index dcd93a0..f120ec7 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
@@ -21,6 +21,7 @@ import org.apache.calcite.materialize.Lattice;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.impl.MaterializedViewTable;
@@ -144,7 +145,8 @@ public abstract class CalciteSchema {
       ImmutableSortedMap.Builder<String, Table> builder);
 
   /** Returns a snapshot representation of this CalciteSchema. */
-  protected abstract CalciteSchema snapshot(CalciteSchema parent, long now);
+  protected abstract CalciteSchema snapshot(
+      CalciteSchema parent, SchemaVersion version);
 
   protected abstract boolean isCacheEnabled();
 
@@ -380,21 +382,20 @@ public abstract class CalciteSchema {
   /** Creates a snapshot of this CalciteSchema as of the specified time. All
    * explicit objects in this CalciteSchema will be copied into the snapshot
    * CalciteSchema, while the contents of the snapshot of the underlying schema
-   * should not change as specified in {@link Schema#snapshot(long)}. Snapshots
-   * of explicit sub schemas will be created and copied recursively.
+   * should not change as specified in {@link Schema#snapshot(SchemaVersion)}.
+   * Snapshots of explicit sub schemas will be created and copied recursively.
    *
    * <p>Currently, to accommodate the requirement of creating tables on the fly
    * for materializations, the snapshot will still use the same table map and
    * lattice map as in the original CalciteSchema instead of making copies.</p>
    *
-   * @param now The current time in millis, as returned by
-   *   {@link System#currentTimeMillis()}
+   * @param version The current schema version
    *
    * @return the schema snapshot.
    */
-  public CalciteSchema createSnapshot(long now) {
+  public CalciteSchema createSnapshot(SchemaVersion version) {
     Preconditions.checkArgument(this.isRoot(), "must be root schema");
-    return snapshot(null, now);
+    return snapshot(null, version);
   }
 
   /** Returns a subset of a map whose keys match the given string
@@ -548,11 +549,7 @@ public abstract class CalciteSchema {
       return CalciteSchema.this.isCacheEnabled();
     }
 
-    public boolean contentsHaveChangedSince(long lastCheck, long now) {
-      return schema.contentsHaveChangedSince(lastCheck, now);
-    }
-
-    public Schema snapshot(long now) {
+    public Schema snapshot(SchemaVersion version) {
       throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
index 2cb70e8..e759944 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
@@ -64,10 +64,6 @@ public abstract class CalciteStatement extends AvaticaStatement {
     return (CalciteConnectionImpl) connection;
   }
 
-  public CalciteConnectionImpl.ContextImpl createPrepareContext() {
-    return new CalciteConnectionImpl.ContextImpl(getConnection());
-  }
-
   protected <T> CalcitePrepare.CalciteSignature<T> prepare(
       Queryable<T> queryable) {
     final CalciteConnectionImpl calciteConnection = getConnection();

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
index 64da625..df004c7 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
@@ -18,6 +18,7 @@ package org.apache.calcite.jdbc;
 
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.util.NameMap;
@@ -152,12 +153,12 @@ class SimpleCalciteSchema extends CalciteSchema {
     return null;
   }
 
-  protected CalciteSchema snapshot(CalciteSchema parent, long now) {
+  protected CalciteSchema snapshot(CalciteSchema parent, SchemaVersion version) {
     CalciteSchema snapshot = new SimpleCalciteSchema(parent,
-        schema.snapshot(now), name, null, tableMap, latticeMap,
+        schema.snapshot(version), name, null, tableMap, latticeMap,
         functionMap, functionNames, nullaryFunctionMap, getPath());
     for (CalciteSchema subSchema : subSchemaMap.map().values()) {
-      CalciteSchema subSchemaSnapshot = subSchema.snapshot(snapshot, now);
+      CalciteSchema subSchemaSnapshot = subSchema.snapshot(snapshot, version);
       snapshot.subSchemaMap.put(subSchema.name, subSchemaSnapshot);
     }
     return snapshot;

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/model/JsonSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonSchema.java b/core/src/main/java/org/apache/calcite/model/JsonSchema.java
index 9a643c5..eb2ab3d 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonSchema.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonSchema.java
@@ -80,12 +80,7 @@ public abstract class JsonSchema {
    * <p>If {@code true}, Calcite will cache the metadata the first time it reads
    * it. This can lead to better performance, especially if name-matching is
    * case-insensitive
-   * (see {@link org.apache.calcite.config.Lex#caseSensitive}).
-   * However, it also leads to the problem of cache staleness.
-   * A particular schema implementation can override the
-   * {@link org.apache.calcite.schema.Schema#contentsHaveChangedSince(long, long)}
-   * method to tell Calcite when it should consider its cache to be out of
-   * date.</p>
+   * (see {@link org.apache.calcite.config.Lex#caseSensitive}).</p>
    *
    * <p>Tables, functions and sub-schemas explicitly created in a schema are
    * not affected by this caching mechanism. They always appear in the schema

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/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 27a4faa..d7a8526 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -41,6 +41,7 @@ import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.StreamableTable;
 import org.apache.calcite.schema.Table;
@@ -430,12 +431,7 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
       return schema.getExpression(parentSchema, name);
     }
 
-    @Override public boolean contentsHaveChangedSince(long lastCheck,
-        long now) {
-      return schema.contentsHaveChangedSince(lastCheck, now);
-    }
-
-    @Override public Schema snapshot(long now) {
+    @Override public Schema snapshot(SchemaVersion version) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/schema/Schema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Schema.java b/core/src/main/java/org/apache/calcite/schema/Schema.java
index 7514e8d..a2d53a0 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schema.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schema.java
@@ -121,36 +121,14 @@ public interface Schema {
    */
   boolean isMutable();
 
-  /** Returns whether the contents of this schema have changed since a given
-   * time. The time is a millisecond value, as returned by
-   * {@link System#currentTimeMillis()}. If this method returns true, and
-   * caching is enabled, Calcite will re-build caches.
-   *
-   * <p>The default implementation in
-   * {@link org.apache.calcite.schema.impl.AbstractSchema} always returns
-   * {@code false}.</p>
-   *
-   * <p>To control whether Calcite caches the contents of a schema, use the
-   * "cache" JSON attribute. The default value is "true".</p>
-   *
-   * @param lastCheck The last time that Calcite called this method, or
-   *   {@link Long#MIN_VALUE} if this is the first call
-   * @param now The current time in millis, as returned by
-   *   {@link System#currentTimeMillis()}
-   *
-   * @return Whether contents changed after {@code lastCheckMillis}.
-   */
-  boolean contentsHaveChangedSince(long lastCheck, long now);
-
   /** Returns the snapshot of this schema as of the specified time. The
    * contents of the schema snapshot should not change over time.
    *
-   * @param now The current time in millis, as returned by
-   *   {@link System#currentTimeMillis()}
+   * @param version The current schema version
    *
    * @return the schema snapshot.
    */
-  Schema snapshot(long now);
+  Schema snapshot(SchemaVersion version);
 
   /** Table type. */
   enum TableType {

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/schema/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/SchemaVersion.java b/core/src/main/java/org/apache/calcite/schema/SchemaVersion.java
new file mode 100644
index 0000000..892eda9
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/SchemaVersion.java
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+/**
+ * An interface to represent a version ID that can be used to create a
+ * read-consistent view of a Schema. This interface assumes a strict
+ * partial ordering contract that is:
+ * <ol>
+ * <li>irreflexive: !a.isBefore(a), which means a cannot happen before itself;
+ * <li>transitive: if a.isBefore(b) and b.isBefore(c) then a.isBefore(c);
+ * <li>antisymmetric: if a.isBefore(b) then !b.isBefore(a).
+ * </ol>
+ * Implementation classes of this interface must also override equals(Object),
+ * hashCode() and toString().
+ *
+ * @see Schema#snapshot(SchemaVersion)
+ */
+public interface SchemaVersion {
+
+  /**
+   * Returns if this Version happens before the other Version.
+   * @param other the other Version object
+   */
+  boolean isBefore(SchemaVersion other);
+}
+
+// End SchemaVersion.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java b/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
index f8ad4b0..129d3dd 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
@@ -21,6 +21,7 @@ import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaFactory;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 
@@ -61,11 +62,7 @@ public class AbstractSchema implements Schema {
     return true;
   }
 
-  public boolean contentsHaveChangedSince(long lastCheck, long now) {
-    return false;
-  }
-
-  public Schema snapshot(long now) {
+  public Schema snapshot(SchemaVersion version) {
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java b/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
index 5be33e7..e222100 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
@@ -20,6 +20,7 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Table;
 
 import java.util.Collection;
@@ -49,12 +50,8 @@ public class DelegatingSchema implements Schema {
     return schema.isMutable();
   }
 
-  public boolean contentsHaveChangedSince(long lastCheck, long now) {
-    return schema.contentsHaveChangedSince(lastCheck, now);
-  }
-
-  public Schema snapshot(long now) {
-    return schema.snapshot(now);
+  public Schema snapshot(SchemaVersion version) {
+    return schema.snapshot(version);
   }
 
   public Expression getExpression(SchemaPlus parentSchema, String name) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/main/java/org/apache/calcite/schema/impl/LongSchemaVersion.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/LongSchemaVersion.java b/core/src/main/java/org/apache/calcite/schema/impl/LongSchemaVersion.java
new file mode 100644
index 0000000..b28cd91
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/impl/LongSchemaVersion.java
@@ -0,0 +1,60 @@
+/*
+ * 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.impl;
+
+import org.apache.calcite.schema.SchemaVersion;
+
+/** Implementation of SchemaVersion that uses a long value as representation. */
+public class LongSchemaVersion implements SchemaVersion {
+  private final long value;
+
+  public LongSchemaVersion(long value) {
+    this.value = value;
+  }
+
+  public boolean isBefore(SchemaVersion other) {
+    if (!(other instanceof LongSchemaVersion)) {
+      throw new IllegalArgumentException(
+          "Cannot compare a LongSchemaVersion object with a "
+          + other.getClass() + " object.");
+    }
+
+    return this.value < ((LongSchemaVersion) other).value;
+  }
+
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof LongSchemaVersion)) {
+      return false;
+    }
+
+    return this.value == ((LongSchemaVersion) obj).value;
+  }
+
+  public int hashCode() {
+    return Long.valueOf(value).hashCode();
+  }
+
+  public String toString() {
+    return String.valueOf(value);
+  }
+}
+
+// End LongSchemaVersion.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/35209136/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 087d656..2282037 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -6192,11 +6192,6 @@ public class JdbcTest {
     aSchema.setCacheEnabled(true);
     assertThat(aSchema.getSubSchemaNames().size(), is(0));
 
-    // AbstractSchema never thinks its contents have changed; subsequent tests
-    // assume this
-    assertThat(aSchema.contentsHaveChangedSince(-1, 1), equalTo(false));
-    assertThat(aSchema.contentsHaveChangedSince(1, 1), equalTo(false));
-
     // first call, to populate the cache
     assertThat(aSchema.getSubSchemaNames().size(), is(0));
 
@@ -6226,33 +6221,24 @@ public class JdbcTest {
 
     // create schema "/a2"
     final Map<String, Schema> a2SubSchemaMap = new HashMap<>();
-    final boolean[] changed = {false};
     final SchemaPlus a2Schema = rootSchema.add("a", new AbstractSchema() {
       @Override protected Map<String, Schema> getSubSchemaMap() {
         return a2SubSchemaMap;
       }
-      @Override public boolean contentsHaveChangedSince(long lastCheck,
-          long now) {
-        return changed[0];
-      }
     });
     a2Schema.setCacheEnabled(true);
     assertThat(a2Schema.getSubSchemaNames().size(), is(0));
 
-    // create schema "/a2/b3". Appears only when we mark the schema changed.
+    // create schema "/a2/b3". Change not visible since caching is enabled.
     a2SubSchemaMap.put("b3", new AbstractSchema());
     assertThat(a2Schema.getSubSchemaNames().size(), is(0));
     Thread.sleep(1);
     assertThat(a2Schema.getSubSchemaNames().size(), is(0));
-    changed[0] = true;
-    assertThat(a2Schema.getSubSchemaNames().size(), is(1));
-    changed[0] = false;
 
-    // or if we disable caching
-    a2SubSchemaMap.put("b4", new AbstractSchema());
-    assertThat(a2Schema.getSubSchemaNames().size(), is(1));
+    // Change visible after we turn off caching.
     a2Schema.setCacheEnabled(false);
-    a2Schema.setCacheEnabled(true);
+    assertThat(a2Schema.getSubSchemaNames().size(), is(1));
+    a2SubSchemaMap.put("b4", new AbstractSchema());
     assertThat(a2Schema.getSubSchemaNames().size(), is(2));
     for (String name : aSchema.getSubSchemaNames()) {
       assertThat(aSchema.getSubSchema(name), notNullValue());


[07/16] calcite git commit: [CALCITE-1945] Make return types of AVG, VARIANCE, STDDEV and COVAR customizable via RelDataTypeSystem

Posted by mm...@apache.org.
[CALCITE-1945] Make return types of AVG, VARIANCE, STDDEV and COVAR customizable via RelDataTypeSystem

* Introduce VARIANCE and STDDEV as alias for _SAMP

Close apache/calcite#518


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

Branch: refs/heads/branch-1.14
Commit: 4208d8021b4978a3f0a259ec299fa7a62c582180
Parents: 6d2fc4e
Author: MinJi Kim <mi...@apache.org>
Authored: Sun Aug 27 16:21:22 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:15:17 2017 -0700

----------------------------------------------------------------------
 .../calcite/rel/rel2sql/SqlImplementor.java     |   6 +-
 .../rel/rules/AggregateReduceFunctionsRule.java | 127 ++++++++++++-------
 .../calcite/rel/type/RelDataTypeSystem.java     |  18 ++-
 .../calcite/rel/type/RelDataTypeSystemImpl.java |  14 +-
 .../apache/calcite/runtime/SqlFunctions.java    |   4 +
 .../java/org/apache/calcite/sql/SqlKind.java    |   9 ++
 .../calcite/sql/fun/SqlAvgAggFunction.java      |  15 ++-
 .../calcite/sql/fun/SqlCovarAggFunction.java    |   2 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  12 ++
 .../apache/calcite/sql/type/ReturnTypes.java    |  34 ++++-
 .../sql2rel/StandardConvertletTable.java        |  49 +++++--
 .../calcite/sql/test/SqlOperatorBaseTest.java   |  70 ++++++++++
 core/src/test/resources/sql/agg.iq              |  24 +++-
 13 files changed, 303 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index d227310..57155b7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -665,7 +665,11 @@ public abstract class SqlImplementor {
         }
 
         final RexCall call = (RexCall) stripCastFromString(rex);
-        final SqlOperator op = call.getOperator();
+        SqlOperator op = call.getOperator();
+        switch (op.getKind()) {
+        case SUM0:
+          op = SqlStdOperatorTable.SUM;
+        }
         final List<SqlNode> nodeList = toSql(program, call.getOperands());
         switch (call.getKind()) {
         case CAST:

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index 8fceff0..7e6e4a1 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -31,10 +31,9 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.fun.SqlSumAggFunction;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
@@ -117,8 +116,7 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
    */
   private boolean containsAvgStddevVarCall(List<AggregateCall> aggCallList) {
     for (AggregateCall call : aggCallList) {
-      if (call.getAggregation() instanceof SqlAvgAggFunction
-          || call.getAggregation() instanceof SqlSumAggFunction) {
+      if (isReducible(call.getAggregation().getKind())) {
         return true;
       }
     }
@@ -126,6 +124,20 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
   }
 
   /**
+   * Returns whether the aggregate call is a reducible function
+   */
+  private boolean isReducible(final SqlKind kind) {
+    if (SqlKind.AVG_AGG_FUNCTIONS.contains(kind)) {
+      return true;
+    }
+    switch (kind) {
+    case SUM:
+      return true;
+    }
+    return false;
+  }
+
+  /**
    * Reduces all calls to AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP in
    * the aggregates list to.
    *
@@ -187,17 +199,16 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
       List<AggregateCall> newCalls,
       Map<AggregateCall, RexNode> aggCallMapping,
       List<RexNode> inputExprs) {
-    if (oldCall.getAggregation() instanceof SqlSumAggFunction) {
-      // replace original SUM(x) with
-      // case COUNT(x) when 0 then null else SUM0(x) end
-      return reduceSum(oldAggRel, oldCall, newCalls, aggCallMapping);
-    }
-    if (oldCall.getAggregation() instanceof SqlAvgAggFunction) {
-      final SqlKind kind = oldCall.getAggregation().getKind();
+    final SqlKind kind = oldCall.getAggregation().getKind();
+    if (isReducible(kind)) {
       switch (kind) {
+      case SUM:
+        // replace original SUM(x) with
+        // case COUNT(x) when 0 then null else SUM0(x) end
+        return reduceSum(oldAggRel, oldCall, newCalls, aggCallMapping);
       case AVG:
         // replace original AVG(x) with SUM(x) / COUNT(x)
-        return reduceAvg(oldAggRel, oldCall, newCalls, aggCallMapping);
+        return reduceAvg(oldAggRel, oldCall, newCalls, aggCallMapping, inputExprs);
       case STDDEV_POP:
         // replace original STDDEV_POP(x) with
         //   SQRT(
@@ -243,19 +254,39 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
     }
   }
 
+  private AggregateCall createAggregateCallWithBinding(
+      RelDataTypeFactory typeFactory,
+      SqlAggFunction aggFunction,
+      RelDataType operandType,
+      Aggregate oldAggRel,
+      AggregateCall oldCall,
+      int argOrdinal) {
+    final Aggregate.AggCallBinding binding =
+        new Aggregate.AggCallBinding(typeFactory, aggFunction,
+            ImmutableList.of(operandType), oldAggRel.getGroupCount(),
+            oldCall.filterArg >= 0);
+    return AggregateCall.create(aggFunction,
+        oldCall.isDistinct(),
+        ImmutableIntList.of(argOrdinal),
+        oldCall.filterArg,
+        aggFunction.inferReturnType(binding),
+        null);
+  }
+
   private RexNode reduceAvg(
       Aggregate oldAggRel,
       AggregateCall oldCall,
       List<AggregateCall> newCalls,
-      Map<AggregateCall, RexNode> aggCallMapping) {
+      Map<AggregateCall, RexNode> aggCallMapping,
+      List<RexNode> inputExprs) {
     final int nGroups = oldAggRel.getGroupCount();
-    RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder();
-    int iAvgInput = oldCall.getArgList().get(0);
-    RelDataType avgInputType =
+    final RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder();
+    final int iAvgInput = oldCall.getArgList().get(0);
+    final RelDataType avgInputType =
         getFieldType(
             oldAggRel.getInput(),
             iAvgInput);
-    AggregateCall sumCall =
+    final AggregateCall sumCall =
         AggregateCall.create(
             SqlStdOperatorTable.SUM,
             oldCall.isDistinct(),
@@ -265,7 +296,7 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
             oldAggRel.getInput(),
             null,
             null);
-    AggregateCall countCall =
+    final AggregateCall countCall =
         AggregateCall.create(
             SqlStdOperatorTable.COUNT,
             oldCall.isDistinct(),
@@ -285,17 +316,20 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
             newCalls,
             aggCallMapping,
             ImmutableList.of(avgInputType));
-    RexNode denominatorRef =
+    final RexNode denominatorRef =
         rexBuilder.addAggCall(countCall,
             nGroups,
             oldAggRel.indicator,
             newCalls,
             aggCallMapping,
             ImmutableList.of(avgInputType));
+
+    final RelDataTypeFactory typeFactory = oldAggRel.getCluster().getTypeFactory();
+    final RelDataType avgType = typeFactory.createTypeWithNullability(
+        oldCall.getType(), numeratorRef.getType().isNullable());
+    numeratorRef = rexBuilder.ensureType(avgType, numeratorRef, true);
     final RexNode divideRef =
-        rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE,
-            numeratorRef,
-            denominatorRef);
+        rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, numeratorRef, denominatorRef);
     return rexBuilder.makeCast(oldCall.getType(), divideRef);
   }
 
@@ -381,36 +415,30 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
 
     assert oldCall.getArgList().size() == 1 : oldCall.getArgList();
     final int argOrdinal = oldCall.getArgList().get(0);
-    final RelDataType argType =
-        getFieldType(
-            oldAggRel.getInput(),
-            argOrdinal);
+    final RelDataType argOrdinalType = getFieldType(oldAggRel.getInput(), argOrdinal);
+    final RelDataType oldCallType =
+        typeFactory.createTypeWithNullability(oldCall.getType(),
+            argOrdinalType.isNullable());
 
-    final RexNode argRef = inputExprs.get(argOrdinal);
-    final RexNode argSquared =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.MULTIPLY, argRef, argRef);
+    final RexNode argRef =
+        rexBuilder.ensureType(oldCallType, inputExprs.get(argOrdinal), true);
+    final int argRefOrdinal = lookupOrAdd(inputExprs, argRef);
+
+    final RexNode argSquared = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY,
+        argRef, argRef);
     final int argSquaredOrdinal = lookupOrAdd(inputExprs, argSquared);
 
-    final Aggregate.AggCallBinding binding =
-        new Aggregate.AggCallBinding(typeFactory, SqlStdOperatorTable.SUM,
-            ImmutableList.of(argRef.getType()), oldAggRel.getGroupCount(),
-            oldCall.filterArg >= 0);
     final AggregateCall sumArgSquaredAggCall =
-        AggregateCall.create(
-            SqlStdOperatorTable.SUM,
-            oldCall.isDistinct(),
-            ImmutableIntList.of(argSquaredOrdinal),
-            oldCall.filterArg,
-            SqlStdOperatorTable.SUM.inferReturnType(binding),
-            null);
+        createAggregateCallWithBinding(typeFactory, SqlStdOperatorTable.SUM,
+            argSquared.getType(), oldAggRel, oldCall, argSquaredOrdinal);
+
     final RexNode sumArgSquared =
         rexBuilder.addAggCall(sumArgSquaredAggCall,
             nGroups,
             oldAggRel.indicator,
             newCalls,
             aggCallMapping,
-            ImmutableList.of(argType));
+            ImmutableList.of(sumArgSquaredAggCall.getType()));
 
     final AggregateCall sumArgAggCall =
         AggregateCall.create(
@@ -422,17 +450,18 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
             oldAggRel.getInput(),
             null,
             null);
+
     final RexNode sumArg =
         rexBuilder.addAggCall(sumArgAggCall,
             nGroups,
             oldAggRel.indicator,
             newCalls,
             aggCallMapping,
-            ImmutableList.of(argType));
-
+            ImmutableList.of(sumArgAggCall.getType()));
+    final RexNode sumArgCast = rexBuilder.ensureType(oldCallType, sumArg, true);
     final RexNode sumSquaredArg =
         rexBuilder.makeCall(
-            SqlStdOperatorTable.MULTIPLY, sumArg, sumArg);
+            SqlStdOperatorTable.MULTIPLY, sumArgCast, sumArgCast);
 
     final AggregateCall countArgAggCall =
         AggregateCall.create(
@@ -441,21 +470,21 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
             oldCall.getArgList(),
             oldCall.filterArg,
             oldAggRel.getGroupCount(),
-            oldAggRel.getInput(),
+            oldAggRel,
             null,
             null);
+
     final RexNode countArg =
         rexBuilder.addAggCall(countArgAggCall,
             nGroups,
             oldAggRel.indicator,
             newCalls,
             aggCallMapping,
-            ImmutableList.of(argType));
+            ImmutableList.of(argOrdinalType));
 
     final RexNode avgSumSquaredArg =
         rexBuilder.makeCall(
-            SqlStdOperatorTable.DIVIDE,
-            sumSquaredArg, countArg);
+            SqlStdOperatorTable.DIVIDE, sumSquaredArg, countArg);
 
     final RexNode diff =
         rexBuilder.makeCall(

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
index 858567c..b8a8088 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
@@ -69,11 +69,21 @@ public interface RelDataTypeSystem {
    * 0 means "not applicable". */
   int getNumTypeRadix(SqlTypeName typeName);
 
-  /**
-   * Returns the return type of a call to the {@code SUM} aggregate function
-   * inferred from its argument type.
+  /** Returns the return type of a call to the {@code SUM} aggregate function,
+   * inferred from its argument type. */
+  RelDataType deriveSumType(RelDataTypeFactory typeFactory,
+      RelDataType argumentType);
+
+  /** Returns the return type of a call to the {@code AVG}, {@code STDDEV} or
+   * {@code VAR} aggregate functions, inferred from its argument type.
    */
-  RelDataType deriveSumType(RelDataTypeFactory typeFactory, RelDataType argumentType);
+  RelDataType deriveAvgAggType(RelDataTypeFactory typeFactory,
+      RelDataType argumentType);
+
+  /** Returns the return type of a call to the {@code COVAR} aggregate function,
+   * inferred from its argument types. */
+  RelDataType deriveCovarType(RelDataTypeFactory typeFactory,
+      RelDataType arg0Type, RelDataType arg1Type);
 
   /** Returns the return type of the {@code CUME_DIST} and {@code PERCENT_RANK}
    * aggregate functions. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
index ef89895..3e0eebd 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
@@ -207,11 +207,21 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     return 0;
   }
 
-  @Override public RelDataType deriveSumType(
-      RelDataTypeFactory typeFactory, RelDataType argumentType) {
+  @Override public RelDataType deriveSumType(RelDataTypeFactory typeFactory,
+      RelDataType argumentType) {
     return argumentType;
   }
 
+  @Override public RelDataType deriveAvgAggType(RelDataTypeFactory typeFactory,
+      RelDataType argumentType) {
+    return argumentType;
+  }
+
+  @Override public RelDataType deriveCovarType(RelDataTypeFactory typeFactory,
+      RelDataType arg0Type, RelDataType arg1Type) {
+    return arg0Type;
+  }
+
   @Override public RelDataType deriveFractionalRankType(RelDataTypeFactory typeFactory) {
     return typeFactory.createTypeWithNullability(
         typeFactory.createSqlType(SqlTypeName.DOUBLE), false);

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 69c6154..6832ee4 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -823,6 +823,10 @@ public class SqlFunctions {
     return Math.pow(b0, b1);
   }
 
+  public static double power(double b0, BigDecimal b1) {
+    return Math.pow(b0, b1.doubleValue());
+  }
+
   public static double power(long b0, long b1) {
     return Math.pow(b0, b1);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/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 ad7c4e2..8d7c8aa 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1119,6 +1119,15 @@ public enum SqlKind {
       EnumSet.of(OTHER_FUNCTION, ROW, TRIM, LTRIM, RTRIM, CAST, JDBC_FN);
 
   /**
+   * Category of SqlAvgAggFunction.
+   *
+   * <p>Consists of {@link #AVG}, {@link #STDDEV_POP}, {@link #STDDEV_SAMP},
+   * {@link #VAR_POP}, {@link #VAR_SAMP}.
+   */
+  public static final Set<SqlKind> AVG_AGG_FUNCTIONS =
+      EnumSet.of(AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP);
+
+  /**
    * Category of comparison operators.
    *
    * <p>Consists of:

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
index 95f8049..6be1ce9 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
@@ -32,26 +32,27 @@ import com.google.common.base.Preconditions;
  * double</code>), and the result is the same type.
  */
 public class SqlAvgAggFunction extends SqlAggFunction {
+
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a SqlAvgAggFunction.
    */
   public SqlAvgAggFunction(SqlKind kind) {
-    super(kind.name(),
+    this(kind.name(), kind);
+  }
+
+  SqlAvgAggFunction(String name, SqlKind kind) {
+    super(name,
         null,
         kind,
-        ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
+        ReturnTypes.AVG_AGG_FUNCTION,
         null,
         OperandTypes.NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    Preconditions.checkArgument(kind == SqlKind.AVG
-        || kind == SqlKind.STDDEV_POP
-        || kind == SqlKind.STDDEV_SAMP
-        || kind == SqlKind.VAR_POP
-        || kind == SqlKind.VAR_SAMP);
+    Preconditions.checkArgument(SqlKind.AVG_AGG_FUNCTIONS.contains(kind), "unsupported sql kind");
   }
 
   @Deprecated // to be removed before 2.0

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
index ea23300..8c62290 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
@@ -43,7 +43,7 @@ public class SqlCovarAggFunction extends SqlAggFunction {
     super(kind.name(),
         null,
         kind,
-        ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
+        ReturnTypes.COVAR_FUNCTION,
         null,
         OperandTypes.NUMERIC_NUMERIC,
         SqlFunctionCategory.NUMERIC,

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/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 3f125bd..39a45b3 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
@@ -918,6 +918,12 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
       new SqlAvgAggFunction(SqlKind.STDDEV_SAMP);
 
   /**
+   * <code>STDDEV</code> aggregate function.
+   */
+  public static final SqlAggFunction STDDEV =
+      new SqlAvgAggFunction("STDDEV", SqlKind.STDDEV_SAMP);
+
+  /**
    * <code>VAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_POP =
@@ -929,6 +935,12 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlAggFunction VAR_SAMP =
       new SqlAvgAggFunction(SqlKind.VAR_SAMP);
 
+  /**
+   * <code>VARIANCE</code> aggregate function.
+   */
+  public static final SqlAggFunction VARIANCE =
+      new SqlAvgAggFunction("VARIANCE", SqlKind.VAR_SAMP);
+
   //-------------------------------------------------------------
   // WINDOW Aggregate Functions
   //-------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
index 73e99f8..15ca544 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
@@ -779,8 +779,10 @@ public abstract class ReturnTypes {
         @Override public RelDataType
         inferReturnType(SqlOperatorBinding opBinding) {
           final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
-          return typeFactory.getTypeSystem()
+          final RelDataType sumType = typeFactory.getTypeSystem()
               .deriveSumType(typeFactory, opBinding.getOperandType(0));
+          // SUM0 should not return null.
+          return typeFactory.createTypeWithNullability(sumType, false);
         }
       };
 
@@ -809,6 +811,36 @@ public abstract class ReturnTypes {
           return typeFactory.getTypeSystem().deriveRankType(typeFactory);
         }
       };
+
+  public static final SqlReturnTypeInference AVG_AGG_FUNCTION =
+      new SqlReturnTypeInference() {
+        @Override public RelDataType
+        inferReturnType(SqlOperatorBinding opBinding) {
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          final RelDataType relDataType = typeFactory.getTypeSystem().deriveAvgAggType(
+              typeFactory, opBinding.getOperandType(0));
+          if (opBinding.getGroupCount() == 0 || opBinding.hasFilter()) {
+            return typeFactory.createTypeWithNullability(relDataType, true);
+          } else {
+            return relDataType;
+          }
+        }
+      };
+
+  public static final SqlReturnTypeInference COVAR_FUNCTION =
+      new SqlReturnTypeInference() {
+        @Override public RelDataType
+        inferReturnType(SqlOperatorBinding opBinding) {
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          final RelDataType relDataType = typeFactory.getTypeSystem().deriveCovarType(
+              typeFactory, opBinding.getOperandType(0), opBinding.getOperandType(1));
+          if (opBinding.getGroupCount() == 0 || opBinding.hasFilter()) {
+            return typeFactory.createTypeWithNullability(relDataType, true);
+          } else {
+            return relDataType;
+          }
+        }
+      };
 }
 
 // End ReturnTypes.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 0d62f9f..8940629 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -299,11 +299,17 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
         SqlStdOperatorTable.STDDEV_SAMP,
         new AvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
     registerOp(
+        SqlStdOperatorTable.STDDEV,
+        new AvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
+    registerOp(
         SqlStdOperatorTable.VAR_POP,
         new AvgVarianceConvertlet(SqlKind.VAR_POP));
     registerOp(
         SqlStdOperatorTable.VAR_SAMP,
         new AvgVarianceConvertlet(SqlKind.VAR_SAMP));
+    registerOp(
+        SqlStdOperatorTable.VARIANCE,
+        new AvgVarianceConvertlet(SqlKind.VAR_SAMP));
 
     final SqlRexConvertlet floorCeilConvertlet = new FloorCeilConvertlet();
     registerOp(SqlStdOperatorTable.FLOOR, floorCeilConvertlet);
@@ -1272,44 +1278,56 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       assert call.operandCount() == 1;
       final SqlNode arg = call.operand(0);
       final SqlNode expr;
+      final RelDataType type =
+          cx.getValidator().getValidatedNodeType(call);
       switch (kind) {
       case AVG:
-        expr = expandAvg(arg);
+        expr = expandAvg(arg, type, cx);
         break;
       case STDDEV_POP:
-        expr = expandVariance(arg, true, true);
+        expr = expandVariance(arg, type, cx, true, true);
         break;
       case STDDEV_SAMP:
-        expr = expandVariance(arg, false, true);
+        expr = expandVariance(arg, type, cx, false, true);
         break;
       case VAR_POP:
-        expr = expandVariance(arg, true, false);
+        expr = expandVariance(arg, type, cx, true, false);
         break;
       case VAR_SAMP:
-        expr = expandVariance(arg, false, false);
+        expr = expandVariance(arg, type, cx, false, false);
         break;
       default:
         throw Util.unexpected(kind);
       }
-      RelDataType type =
-          cx.getValidator().getValidatedNodeType(call);
       RexNode rex = cx.convertExpression(expr);
       return cx.getRexBuilder().ensureType(type, rex, true);
     }
 
     private SqlNode expandAvg(
-        final SqlNode arg) {
+        final SqlNode arg, final RelDataType avgType, final SqlRexContext cx) {
       final SqlParserPos pos = SqlParserPos.ZERO;
       final SqlNode sum =
           SqlStdOperatorTable.SUM.createCall(pos, arg);
+      final RexNode sumRex = cx.convertExpression(sum);
+      final SqlNode sumCast;
+      if (!sumRex.getType().equals(avgType)) {
+        sumCast = SqlStdOperatorTable.CAST.createCall(pos,
+            new SqlDataTypeSpec(
+                new SqlIdentifier(avgType.getSqlTypeName().getName(), pos),
+                avgType.getPrecision(), avgType.getScale(), null, null, pos));
+      } else {
+        sumCast = sum;
+      }
       final SqlNode count =
           SqlStdOperatorTable.COUNT.createCall(pos, arg);
       return SqlStdOperatorTable.DIVIDE.createCall(
-          pos, sum, count);
+          pos, sumCast, count);
     }
 
     private SqlNode expandVariance(
-        final SqlNode arg,
+        final SqlNode argInput,
+        final RelDataType varType,
+        final SqlRexContext cx,
         boolean biased,
         boolean sqrt) {
       // stddev_pop(x) ==>
@@ -1332,6 +1350,17 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       //     (sum(x * x) - sum(x) * sum(x) / count(x))
       //     / (count(x) - 1)
       final SqlParserPos pos = SqlParserPos.ZERO;
+
+      final RexNode argRex = cx.convertExpression(argInput);
+      final SqlNode arg;
+      if (!argRex.getType().equals(varType)) {
+        arg = SqlStdOperatorTable.CAST.createCall(pos,
+            new SqlDataTypeSpec(new SqlIdentifier(varType.getSqlTypeName().getName(), pos),
+                varType.getPrecision(), varType.getScale(), null, null, pos));
+      } else {
+        arg = argInput;
+      }
+
       final SqlNode argSquared =
           SqlStdOperatorTable.MULTIPLY.createCall(pos, arg, arg);
       final SqlNode sumArgSquared =

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index f73921f..15ddb13 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -6417,6 +6417,33 @@ public abstract class SqlOperatorBaseTest {
         0d);
   }
 
+  @Test public void testStddevFunc() {
+    tester.setFor(SqlStdOperatorTable.STDDEV, VM_EXPAND);
+    tester.checkFails(
+        "stddev(^*^)",
+        "Unknown identifier '\\*'",
+        false);
+    tester.checkFails(
+        "^stddev(cast(null as varchar(2)))^",
+        "(?s)Cannot apply 'STDDEV' to arguments of type 'STDDEV\\(<VARCHAR\\(2\\)>\\)'\\. Supported form\\(s\\): 'STDDEV\\(<NUMERIC>\\)'.*",
+        false);
+    tester.checkType("stddev(CAST(NULL AS INTEGER))", "INTEGER");
+    checkAggType(tester, "stddev(DISTINCT 1.5)", "DECIMAL(2, 1) NOT NULL");
+    final String[] values = {"0", "CAST(null AS FLOAT)", "3", "3"};
+    // with one value
+    tester.checkAgg(
+        "stddev(x)",
+        new String[]{"5"},
+        null,
+        0d);
+    // with zero values
+    tester.checkAgg(
+        "stddev(x)",
+        new String[]{},
+        null,
+        0d);
+  }
+
   @Test public void testVarPopFunc() {
     tester.setFor(SqlStdOperatorTable.VAR_POP, VM_EXPAND);
     tester.checkFails(
@@ -6505,6 +6532,49 @@ public abstract class SqlOperatorBaseTest {
         0d);
   }
 
+  @Test public void testVarFunc() {
+    tester.setFor(SqlStdOperatorTable.VARIANCE, VM_EXPAND);
+    tester.checkFails(
+        "variance(^*^)",
+        "Unknown identifier '\\*'",
+        false);
+    tester.checkFails(
+        "^variance(cast(null as varchar(2)))^",
+        "(?s)Cannot apply 'VARIANCE' to arguments of type 'VARIANCE\\(<VARCHAR\\(2\\)>\\)'\\. Supported form\\(s\\): 'VARIANCE\\(<NUMERIC>\\)'.*",
+        false);
+    tester.checkType("variance(CAST(NULL AS INTEGER))", "INTEGER");
+    checkAggType(tester, "variance(DISTINCT 1.5)", "DECIMAL(2, 1) NOT NULL");
+    final String[] values = {"0", "CAST(null AS FLOAT)", "3", "3"};
+    if (!enable) {
+      return;
+    }
+    tester.checkAgg(
+        "variance(x)", values, 3d, // verified on Oracle 10g
+        0d);
+    tester.checkAgg(
+        "variance(DISTINCT x)", // Oracle does not allow distinct
+        values,
+        4.5d,
+        0.0001d);
+    tester.checkAgg(
+        "variance(DISTINCT CASE x WHEN 0 THEN NULL ELSE -1 END)",
+        values,
+        null,
+        0d);
+    // with one value
+    tester.checkAgg(
+        "variance(x)",
+        new String[]{"5"},
+        null,
+        0d);
+    // with zero values
+    tester.checkAgg(
+        "variance(x)",
+        new String[]{},
+        null,
+        0d);
+  }
+
   @Test public void testMinFunc() {
     tester.setFor(SqlStdOperatorTable.MIN, VM_EXPAND);
     tester.checkFails(

http://git-wip-us.apache.org/repos/asf/calcite/blob/4208d802/core/src/test/resources/sql/agg.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/agg.iq b/core/src/test/resources/sql/agg.iq
index e4ec228..28e8b4b 100755
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -85,19 +85,31 @@ select stddev_pop(deptno) as s from emp;
 
 !ok
 
+# stddev
+select stddev(deptno) as s from emp;
++----+
+| S  |
++----+
+| 19 |
++----+
+(1 row)
+
+!ok
+
 # both
 select gender,
   stddev_pop(deptno) as p,
   stddev_samp(deptno) as s,
+  stddev(deptno) as ss,
   count(deptno) as c
 from emp
 group by gender;
-+--------+----+----+---+
-| GENDER | P  | S  | C |
-+--------+----+----+---+
-| F      | 17 | 19 | 5 |
-| M      | 17 | 20 | 3 |
-+--------+----+----+---+
++--------+----+----+----+---+
+| GENDER | P  | S  | SS | C |
++--------+----+----+----+---+
+| F      | 17 | 19 | 19 | 5 |
+| M      | 17 | 20 | 20 | 3 |
++--------+----+----+----+---+
 (2 rows)
 
 !ok


[06/16] calcite git commit: [CALCITE-1966] Allow normal views to act as materialization table (Christian Beikov)

Posted by mm...@apache.org.
[CALCITE-1966] Allow normal views to act as materialization table (Christian Beikov)

Close apache/calcite#527


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

Branch: refs/heads/branch-1.14
Commit: 6d2fc4ecf1f2640b6ccfc0dc890737503dbc0734
Parents: e152592
Author: Christian Beikov <ch...@gmail.com>
Authored: Thu Aug 24 16:21:40 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:14:31 2017 -0700

----------------------------------------------------------------------
 .../materialize/MaterializationService.java     |  6 ++++
 .../calcite/plan/RelOptMaterialization.java     |  6 ++--
 .../calcite/plan/RelOptMaterializations.java    | 13 ++++---
 .../calcite/prepare/CalcitePrepareImpl.java     |  3 +-
 .../org/apache/calcite/prepare/Prepare.java     |  4 ++-
 .../org/apache/calcite/test/CalciteAssert.java  | 25 +++++++++-----
 .../calcite/test/MaterializationTest.java       | 36 +++++++++++++++++---
 .../java/org/apache/calcite/util/Smalls.java    | 11 ++++++
 8 files changed, 80 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java b/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
index 1b9fd88..3625a02 100644
--- a/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
+++ b/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
@@ -118,14 +118,20 @@ public class MaterializationService {
     final CalciteConnection connection =
         CalciteMetaImpl.connect(schema.root(), null);
     CalciteSchema.TableEntry tableEntry;
+    // If the user says the materialization exists, first try to find a table
+    // with the name and if none can be found, lookup a view in the schema
     if (existing) {
       tableEntry = schema.getTable(suggestedTableName, true);
+      if (tableEntry == null) {
+        tableEntry = schema.getTableBasedOnNullaryFunction(suggestedTableName, true);
+      }
     } else {
       tableEntry = null;
     }
     if (tableEntry == null) {
       tableEntry = schema.getTableBySql(viewSql);
     }
+
     RelDataType rowType = null;
     if (tableEntry == null) {
       Table table = tableFactory.createTable(schema, viewSql, viewSchemaPath);

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java b/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
index b65af48..6e1af24 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
@@ -56,14 +56,14 @@ public class RelOptMaterialization {
   public final RelNode tableRel;
   public final RelOptTable starRelOptTable;
   public final StarTable starTable;
-  public final RelOptTable table;
+  public final List<String> qualifiedTableName;
   public final RelNode queryRel;
 
   /**
    * Creates a RelOptMaterialization.
    */
   public RelOptMaterialization(RelNode tableRel, RelNode queryRel,
-      RelOptTable starRelOptTable) {
+      RelOptTable starRelOptTable, List<String> qualifiedTableName) {
     this.tableRel =
         RelOptUtil.createCastRel(tableRel, queryRel.getRowType(), false);
     this.starRelOptTable = starRelOptTable;
@@ -73,7 +73,7 @@ public class RelOptMaterialization {
       this.starTable = starRelOptTable.unwrap(StarTable.class);
       assert starTable != null;
     }
-    this.table = tableRel.getTable();
+    this.qualifiedTableName = qualifiedTableName;
     this.queryRel = queryRel;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/main/java/org/apache/calcite/plan/RelOptMaterializations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptMaterializations.java b/core/src/main/java/org/apache/calcite/plan/RelOptMaterializations.java
index 1e3a9e9..ce0c033 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptMaterializations.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptMaterializations.java
@@ -138,9 +138,9 @@ public abstract class RelOptMaterializations {
     final Map<List<String>, RelOptMaterialization> qnameMap = new HashMap<>();
     for (RelOptMaterialization materialization : materializations) {
       // If materialization is a tile in a lattice, we will deal with it shortly.
-      if (materialization.table != null
+      if (materialization.qualifiedTableName != null
           && materialization.starTable == null) {
-        final List<String> qname = materialization.table.getQualifiedName();
+        final List<String> qname = materialization.qualifiedTableName;
         qnameMap.put(qname, materialization);
         for (RelOptTable usedTable
             : RelOptUtil.findTables(materialization.queryRel)) {
@@ -167,7 +167,7 @@ public abstract class RelOptMaterializations {
     for (List<String> qname : TopologicalOrderIterator.of(usesGraph)) {
       RelOptMaterialization materialization = qnameMap.get(qname);
       if (materialization != null
-          && usesTable(materialization.table, queryTablesUsed, frozenGraph)) {
+          && usesTable(materialization.qualifiedTableName, queryTablesUsed, frozenGraph)) {
         applicableMaterializations.add(materialization);
       }
     }
@@ -218,13 +218,12 @@ public abstract class RelOptMaterializations {
    * {@code usedTables}.
    */
   private static boolean usesTable(
-      RelOptTable table,
+      List<String> qualifiedName,
       Set<RelOptTable> usedTables,
       Graphs.FrozenGraph<List<String>, DefaultEdge> usesGraph) {
     for (RelOptTable queryTable : usedTables) {
-      if (usesGraph.getShortestPath(
-          queryTable.getQualifiedName(),
-          table.getQualifiedName()) != null) {
+      if (usesGraph.getShortestPath(queryTable.getQualifiedName(), qualifiedName)
+          != null) {
         return true;
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 72fa04b..17b6e3a 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -742,7 +742,8 @@ public class CalcitePrepareImpl implements CalcitePrepare {
           .setQuotedCasing(config.quotedCasing())
           .setUnquotedCasing(config.unquotedCasing())
           .setQuoting(config.quoting())
-          .setConformance(config.conformance());
+          .setConformance(config.conformance())
+          .setCaseSensitive(config.caseSensitive());
       final SqlParserImplFactory parserFactory =
           config.parserFactory(SqlParserImplFactory.class, null);
       if (parserFactory != null) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/main/java/org/apache/calcite/prepare/Prepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/Prepare.java b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
index 29af1ef..8cb09c3 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -143,10 +143,12 @@ public abstract class Prepare {
 
     final List<RelOptMaterialization> materializationList = new ArrayList<>();
     for (Materialization materialization : materializations) {
+      List<String> qualifiedTableName = materialization.materializedTable.path();
       materializationList.add(
           new RelOptMaterialization(materialization.tableRel,
               materialization.queryRel,
-              materialization.starRelOptTable));
+              materialization.starRelOptTable,
+              qualifiedTableName));
     }
 
     final List<RelOptLattice> latticeList = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 76a464a..490c7e5 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -925,8 +925,13 @@ public class CalciteAssert {
       return with("model", "inline:" + model);
     }
 
-    /** Adds materializations to the schema. */
     public final AssertThat withMaterializations(String model,
+         final String... materializations) {
+      return withMaterializations(model, false, materializations);
+    }
+
+    /** Adds materializations to the schema. */
+    public final AssertThat withMaterializations(String model, final boolean existing,
         final String... materializations) {
       return withMaterializations(model,
           new Function<JsonBuilder, List<Object>>() {
@@ -937,7 +942,9 @@ public class CalciteAssert {
                 String table = materializations[i++];
                 final Map<String, Object> map = builder.map();
                 map.put("table", table);
-                map.put("view", table + "v");
+                if (!existing) {
+                  map.put("view", table + "v");
+                }
                 String sql = materializations[i];
                 final String sql2 = sql.replaceAll("`", "\"");
                 map.put("sql", sql2);
@@ -957,15 +964,17 @@ public class CalciteAssert {
           "materializations: " + builder.toJsonString(list);
       final String model2;
       if (model.contains("defaultSchema: 'foodmart'")) {
-        model2 = model.replace("]",
-            ", { name: 'mat', "
+        int endIndex = model.lastIndexOf(']');
+        model2 = model.substring(0, endIndex)
+            + ", \n{ name: 'mat', "
             + buf
             + "}\n"
-            + "]");
+            + "]"
+            + model.substring(endIndex + 1);
       } else if (model.contains("type: ")) {
-        model2 = model.replace("type: ",
-            buf + ",\n"
-            + "type: ");
+        model2 = model.replaceFirst("type: ",
+            java.util.regex.Matcher.quoteReplacement(buf + ",\n"
+            + "type: "));
       } else {
         throw new AssertionError("do not know where to splice");
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
index fb96721..6a3bcc9 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -197,14 +197,22 @@ public class MaterializationTest {
         RuleSets.ofList(ImmutableList.<RelOptRule>of()));
   }
 
+
+  private void checkMaterialize(String materialize, String query, String model,
+      Function<ResultSet, Void> explainChecker, final RuleSet rules) {
+    checkThatMaterialize(materialize, query, "m0", false, model, explainChecker,
+        rules).sameResultWithMaterializationsDisabled();
+  }
+
   /** Checks that a given query can use a materialized view with a given
    * definition. */
-  private void checkMaterialize(String materialize, String query, String model,
+  private CalciteAssert.AssertQuery checkThatMaterialize(String materialize,
+      String query, String name, boolean existing, String model,
       Function<ResultSet, Void> explainChecker, final RuleSet rules) {
     try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.AssertQuery that = CalciteAssert.that()
-          .withMaterializations(model, "m0", materialize)
+          .withMaterializations(model, existing, name, materialize)
           .query(query)
           .enableMaterializations(true);
 
@@ -220,8 +228,7 @@ public class MaterializationTest {
         });
       }
 
-      that.explainMatches("", explainChecker)
-          .sameResultWithMaterializationsDisabled();
+      return that.explainMatches("", explainChecker);
     }
   }
 
@@ -1797,6 +1804,23 @@ public class MaterializationTest {
         HR_FKUK_MODEL);
   }
 
+  @Test public void testViewMaterialization() {
+    checkThatMaterialize(
+        "select \"depts\".\"name\"\n"
+            + "from \"emps\"\n"
+            + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
+        "select \"depts\".\"name\"\n"
+            + "from \"depts\"\n"
+            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
+        "matview",
+        true,
+        HR_FKUK_MODEL,
+        CalciteAssert.checkResultContains(
+            "EnumerableValues(tuples=[[{ 'noname' }]])"),
+        RuleSets.ofList(ImmutableList.<RelOptRule>of()))
+        .returnsValue("noname");
+  }
+
   @Test public void testSubQuery() {
     String q = "select \"empid\", \"deptno\", \"salary\" from \"emps\" e1\n"
         + "where \"empid\" = (\n"
@@ -2131,6 +2155,10 @@ public class MaterializationTest {
     public TranslatableTable view(String s) {
       return Smalls.view(s);
     }
+
+    public TranslatableTable matview() {
+      return Smalls.strView("noname");
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/6d2fc4ec/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index b3e64ca..39f0263 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -349,6 +349,17 @@ public class Smalls {
         }, "values (1), (3), " + s, ImmutableList.<String>of(), Arrays.asList("view"));
   }
 
+  public static TranslatableTable strView(String s) {
+    return new ViewTable(Object.class,
+        new RelProtoDataType() {
+          public RelDataType apply(RelDataTypeFactory typeFactory) {
+            return typeFactory.builder().add("c", SqlTypeName.VARCHAR, 100)
+                    .build();
+          }
+        }, "values (" + SqlDialect.CALCITE.quoteStringLiteral(s) + ")",
+        ImmutableList.<String>of(), Arrays.asList("view"));
+  }
+
   public static TranslatableTable str(Object o, Object p) {
     assertThat(RexLiteral.validConstant(o, Litmus.THROW), is(true));
     assertThat(RexLiteral.validConstant(p, Litmus.THROW), is(true));


[10/16] calcite git commit: [CALCITE-1968] OpenGIS Simple Feature Access SQL 1.2.1: add GEOMETRY data type and first 35 functions

Posted by mm...@apache.org.
[CALCITE-1968] OpenGIS Simple Feature Access SQL 1.2.1: add GEOMETRY data type and first 35 functions

Add Spatial page, document GIS functions in SQL reference (indicating
which ones are implemented), and add "countries" data set for testing.

Functions: ST_AsText, ST_AsWKT, ST_Boundary, ST_Buffer, ST_Contains,
ST_ContainsProperly, ST_Crosses, ST_Disjoint, ST_Distance, ST_DWithin,
ST_Envelope, ST_EnvelopesIntersect, ST_Equals, ST_GeometryType,
ST_GeometryTypeCode, ST_GeomFromText, ST_Intersects, ST_Is3D,
ST_LineFromText, ST_MakeLine, ST_MakePoint, ST_MLineFromText,
ST_MPointFromText, ST_MPolyFromText, ST_Overlaps, ST_Point,
ST_PointFromText, ST_PolyFromText, ST_SetSRID, ST_Touches,
ST_Transform, ST_Union, ST_Within, ST_Z.


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

Branch: refs/heads/branch-1.14
Commit: cc20ca13db4d506d9d4d1b861dd1c7ac3944e56e
Parents: b2bf1ca
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Aug 24 03:27:35 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 13:24:42 2017 -0700

----------------------------------------------------------------------
 core/pom.xml                                    |    4 +
 core/src/main/codegen/templates/Parser.jj       |   11 +-
 .../calcite/jdbc/JavaTypeFactoryImpl.java       |    3 +
 .../org/apache/calcite/model/ModelHandler.java  |   39 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |    6 +-
 .../apache/calcite/runtime/CalciteResource.java |    3 +
 .../apache/calcite/runtime/GeoFunctions.java    |  651 ++++++++++
 .../apache/calcite/sql/type/ExtraSqlTypes.java  |    3 +
 .../sql/type/JavaToSqlTypeConversionRules.java  |    3 +
 .../sql/type/SqlTypeAssignmentRules.java        |    3 +
 .../apache/calcite/sql/type/SqlTypeFamily.java  |    5 +-
 .../apache/calcite/sql/type/SqlTypeName.java    |    3 +-
 .../sql/validate/SqlAbstractConformance.java    |    4 +
 .../calcite/sql/validate/SqlConformance.java    |   12 +-
 .../sql/validate/SqlConformanceEnum.java        |   11 +
 .../calcite/runtime/CalciteResource.properties  |    1 +
 .../calcite/jdbc/CalciteRemoteDriverTest.java   |    2 +-
 .../calcite/sql/parser/SqlParserTest.java       |    7 +
 .../org/apache/calcite/test/CalciteAssert.java  |   23 +
 .../calcite/test/CountriesTableFunction.java    |  327 +++++
 .../org/apache/calcite/test/QuidemTest.java     |    4 +
 .../apache/calcite/test/SqlValidatorTest.java   |   18 +
 core/src/test/resources/sql/spatial.iq          | 1114 ++++++++++++++++++
 file/src/test/resources/geo/countries.csv       |  246 ++++
 pom.xml                                         |    6 +
 site/_data/docs.yml                             |    1 +
 site/_docs/reference.md                         |  342 +++++-
 site/_docs/spatial.md                           |   63 +
 src/main/config/checkstyle/suppressions.xml     |    3 +
 29 files changed, 2908 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 6e8428a..de6e24c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -60,6 +60,10 @@ limitations under the License.
       <artifactId>commons-lang3</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-core</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/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 fae4fd1..0aef85b 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -4363,7 +4363,7 @@ SqlIdentifier TypeName() :
     }
 }
 
-// Types used for for JDBC and ODBC scalar conversion function
+// Types used for JDBC and ODBC scalar conversion function
 SqlTypeName SqlTypeName(Span s) :
 {
 }
@@ -4383,6 +4383,13 @@ SqlTypeName SqlTypeName(Span s) :
 |
     <TIMESTAMP> { return SqlTypeName.TIMESTAMP; }
 |
+    <GEOMETRY> {
+        if (!this.conformance.allowGeometry()) {
+            throw new ParseException(RESOURCE.geometryDisabled().str());
+        }
+        return SqlTypeName.GEOMETRY;
+    }
+|
     (<DECIMAL> | <DEC> | <NUMERIC>) { return SqlTypeName.DECIMAL; }
 |
     <BOOLEAN> { return SqlTypeName.BOOLEAN; }
@@ -5564,6 +5571,7 @@ SqlPostfixOperator PostfixRowOperator() :
 |   < G: "G" >
 |   < GENERAL: "GENERAL" >
 |   < GENERATED: "GENERATED" >
+|   < GEOMETRY: "GEOMETRY" >
 |   < GET: "GET" >
 |   < GLOBAL: "GLOBAL" >
 |   < GO: "GO" >
@@ -6105,6 +6113,7 @@ String CommonNonReservedKeyWord() :
     |   <G>
     |   <GENERAL>
     |   <GENERATED>
+    |   <GEOMETRY>
     |   <GO>
     |   <GOTO>
     |   <GRANTED>

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
index 083d362..2e0ded2 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.runtime.GeoFunctions;
 import org.apache.calcite.runtime.Unit;
 import org.apache.calcite.sql.type.BasicSqlType;
 import org.apache.calcite.sql.type.IntervalSqlType;
@@ -209,6 +210,8 @@ public class JavaTypeFactoryImpl
       case BINARY:
       case VARBINARY:
         return ByteString.class;
+      case GEOMETRY:
+        return GeoFunctions.Geom.class;
       case ANY:
         return Object.class;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/model/ModelHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
index ee6373e..6af4395 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -54,6 +54,7 @@ import java.util.ArrayDeque;
 import java.util.Collections;
 import java.util.Deque;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import javax.sql.DataSource;
 
@@ -86,9 +87,29 @@ public class ModelHandler {
     visit(root);
   }
 
-  /** Creates and validates a ScalarFunctionImpl. */
+  /** @deprecated Use {@link #addFunctions}. */
+  @Deprecated
   public static void create(SchemaPlus schema, String functionName,
       List<String> path, String className, String methodName) {
+    addFunctions(schema, functionName, path, className, methodName, false);
+  }
+
+  /** Creates and validates a {@link ScalarFunctionImpl}, and adds it to a
+   * schema. If {@code methodName} is "*", may add more than one function.
+   *
+   * @param schema Schema to add to
+   * @param functionName Name of function; null to derived from method name
+   * @param path Path to look for functions
+   * @param className Class to inspect for methods that may be user-defined
+   *                  functions
+   * @param methodName Method name;
+   *                  null means use the class as a UDF;
+   *                  "*" means add all methods
+   * @param upCase Whether to convert method names to upper case, so that they
+   *               can be called without using quotes
+   */
+  public static void addFunctions(SchemaPlus schema, String functionName,
+      List<String> path, String className, String methodName, boolean upCase) {
     final Class<?> clazz;
     try {
       clazz = Class.forName(className);
@@ -112,14 +133,26 @@ public class ModelHandler {
     if (methodName != null && methodName.equals("*")) {
       for (Map.Entry<String, ScalarFunction> entry
           : ScalarFunctionImpl.createAll(clazz).entries()) {
-        schema.add(entry.getKey(), entry.getValue());
+        String name = entry.getKey();
+        if (upCase) {
+          name = name.toUpperCase(Locale.ROOT);
+        }
+        schema.add(name, entry.getValue());
       }
       return;
     } else {
       final ScalarFunction function =
           ScalarFunctionImpl.create(clazz, Util.first(methodName, "eval"));
       if (function != null) {
-        schema.add(Util.first(functionName, methodName), function);
+        final String name;
+        if (functionName != null) {
+          name = functionName;
+        } else if (upCase) {
+          name = methodName.toUpperCase(Locale.ROOT);
+        } else {
+          name = methodName;
+        }
+        schema.add(name, function);
         return;
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 17b6e3a..ec57a8d 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -124,6 +124,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.type.ExtraSqlTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlConformance;
@@ -922,7 +923,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
       assert rep != null;
       return ColumnMetaData.array(componentType, typeName, rep);
     } else {
-      final int typeOrdinal = getTypeOrdinal(type);
+      int typeOrdinal = getTypeOrdinal(type);
       switch (typeOrdinal) {
       case Types.STRUCT:
         final List<ColumnMetaData> columns = new ArrayList<>();
@@ -932,6 +933,9 @@ public class CalcitePrepareImpl implements CalcitePrepare {
                   field.getType(), null, null));
         }
         return ColumnMetaData.struct(columns);
+      case ExtraSqlTypes.GEOMETRY:
+        typeOrdinal = Types.VARCHAR;
+        // fall through
       default:
         final Type clazz =
             typeFactory.getJavaClass(Util.first(fieldType, type));

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 61956cf..b42b1a0 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -55,6 +55,9 @@ public interface CalciteResource {
   @BaseMessage("BETWEEN operator has no terminating AND")
   ExInst<SqlValidatorException> betweenWithoutAnd();
 
+  @BaseMessage("Geo-spatial extensions and the GEOMETRY data type are not enabled")
+  ExInst<SqlValidatorException> geometryDisabled();
+
   @BaseMessage("Illegal INTERVAL literal {0}; at {1}")
   @Property(name = "SQLSTATE", value = "42000")
   ExInst<CalciteException> illegalIntervalLiteral(String a0, String a1);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java b/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
new file mode 100644
index 0000000..6ddcc8a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
@@ -0,0 +1,651 @@
+/*
+ * 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.runtime;
+
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.Experimental;
+import org.apache.calcite.linq4j.function.SemiStrict;
+import org.apache.calcite.linq4j.function.Strict;
+import org.apache.calcite.util.Util;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryEngine;
+import com.esri.core.geometry.Line;
+import com.esri.core.geometry.MapGeometry;
+import com.esri.core.geometry.Operator;
+import com.esri.core.geometry.OperatorBoundary;
+import com.esri.core.geometry.OperatorFactoryLocal;
+import com.esri.core.geometry.OperatorIntersects;
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.Polygon;
+import com.esri.core.geometry.Polyline;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktExportFlags;
+import com.esri.core.geometry.WktImportFlags;
+
+import com.google.common.base.Preconditions;
+
+import java.math.BigDecimal;
+
+/**
+ * Helper methods to implement Geo-spatial functions in generated code.
+ *
+ * <p>Remaining tasks:
+ *
+ * <ul>
+ *   <li>Determine type code for
+ *   {@link org.apache.calcite.sql.type.ExtraSqlTypes#GEOMETRY}
+ *   <li>Should we create aliases for functions in upper-case?
+ *   Without ST_ prefix?
+ *   <li>Consider adding spatial literals, e.g. `GEOMETRY 'POINT (30 10)'`
+ *   <li>Integer arguments, e.g. SELECT ST_MakePoint(1, 2, 1.5),
+ *     ST_MakePoint(1, 2)
+ *   <li>Are GEOMETRY values comparable? If so add ORDER BY test
+ *   <li>We have to add 'Z' to create 3D objects. This is inconsistent with
+ *   PostGIS. Who is right? At least document the difference.
+ *   <li>Should add GeometryEngine.intersects; similar to disjoint etc.
+ *   <li>Make {@link #ST_MakeLine(Geom, Geom)} varargs</li>
+ * </ul>
+ */
+@SuppressWarnings({"UnnecessaryUnboxing", "WeakerAccess", "unused"})
+@Deterministic
+@Strict
+@Experimental
+public class GeoFunctions {
+  private static final int NO_SRID = 0;
+  private static final SpatialReference SPATIAL_REFERENCE =
+      SpatialReference.create(4326);
+
+  private GeoFunctions() {}
+
+  private static UnsupportedOperationException todo() {
+    return new UnsupportedOperationException();
+  }
+
+  protected static Geom bind(Geometry geometry, int srid) {
+    if (geometry == null) {
+      return null;
+    }
+    if (srid == NO_SRID) {
+      return new SimpleGeom(geometry);
+    }
+    return bind(geometry, SpatialReference.create(srid));
+  }
+
+  private static MapGeom bind(Geometry geometry, SpatialReference sr) {
+    return new MapGeom(new MapGeometry(geometry, sr));
+  }
+
+  // Geometry conversion functions (2D and 3D) ================================
+
+  public static String ST_AsText(Geom g) {
+    return ST_AsWKT(g);
+  }
+
+  public static String ST_AsWKT(Geom g) {
+    return GeometryEngine.geometryToWkt(g.g(),
+        WktExportFlags.wktExportDefaults);
+  }
+
+  public static Geom ST_GeomFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_GeomFromText(String s, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(s,
+        WktImportFlags.wktImportDefaults, Geometry.Type.Unknown);
+    return bind(g, srid);
+  }
+
+  public static Geom ST_LineFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_LineFromText(String wkt, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(wkt,
+        WktImportFlags.wktImportDefaults,
+        Geometry.Type.Line);
+    return bind(g, srid);
+  }
+
+  public static Geom ST_MPointFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_MPointFromText(String wkt, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(wkt,
+        WktImportFlags.wktImportDefaults,
+        Geometry.Type.MultiPoint);
+    return bind(g, srid);
+  }
+
+  public static Geom ST_PointFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_PointFromText(String wkt, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(wkt,
+        WktImportFlags.wktImportDefaults,
+        Geometry.Type.Point);
+    return bind(g, srid);
+  }
+
+  public static Geom ST_PolyFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_PolyFromText(String wkt, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(wkt,
+        WktImportFlags.wktImportDefaults,
+        Geometry.Type.Polygon);
+    return bind(g, srid);
+  }
+
+  public static Geom ST_MLineFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_MLineFromText(String wkt, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(wkt,
+        WktImportFlags.wktImportDefaults,
+        Geometry.Type.Unknown); // NOTE: there is no Geometry.Type.MultiLine
+    return bind(g, srid);
+  }
+
+  public static Geom ST_MPolyFromText(String s) {
+    return ST_GeomFromText(s, NO_SRID);
+  }
+
+  public static Geom ST_MPolyFromText(String wkt, int srid) {
+    final Geometry g = GeometryEngine.geometryFromWkt(wkt,
+        WktImportFlags.wktImportDefaults,
+        Geometry.Type.Unknown); // NOTE: there is no Geometry.Type.MultiPolygon
+    return bind(g, srid);
+  }
+
+  // Geometry creation functions ==============================================
+
+  /**  Creates a line-string from the given POINTs (or MULTIPOINTs). */
+  public static Geom ST_MakeLine(Geom geom1, Geom geom2) {
+    return makeLine(geom1, geom2);
+  }
+
+  public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3) {
+    return makeLine(geom1, geom2, geom3);
+  }
+
+  public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3,
+      Geom geom4) {
+    return makeLine(geom1, geom2, geom3, geom4);
+  }
+
+  public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3,
+      Geom geom4, Geom geom5) {
+    return makeLine(geom1, geom2, geom3, geom4, geom5);
+  }
+
+  public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3,
+      Geom geom4, Geom geom5, Geom geom6) {
+    return makeLine(geom1, geom2, geom3, geom4, geom5, geom6);
+  }
+
+  private static Geom makeLine(Geom... geoms) {
+    final Polyline g = new Polyline();
+    Point p = null;
+    for (Geom geom : geoms) {
+      if (geom.g() instanceof Point) {
+        final Point prev = p;
+        p = (Point) geom.g();
+        if (prev != null) {
+          final Line line = new Line();
+          line.setStart(prev);
+          line.setEnd(p);
+          g.addSegment(line, false);
+        }
+      }
+    }
+    return new SimpleGeom(g);
+  }
+
+  /**  Alias for {@link #ST_Point(BigDecimal, BigDecimal)}. */
+  public static Geom ST_MakePoint(BigDecimal x, BigDecimal y) {
+    return ST_Point(x, y);
+  }
+
+  /**  Alias for {@link #ST_Point(BigDecimal, BigDecimal, BigDecimal)}. */
+  public static Geom ST_MakePoint(BigDecimal x, BigDecimal y, BigDecimal z) {
+    return ST_Point(x, y, z);
+  }
+
+  /**  Constructs a 2D point from coordinates. */
+  public static Geom ST_Point(BigDecimal x, BigDecimal y) {
+    // NOTE: Combine the double and BigDecimal variants of this function
+    return point(x.doubleValue(), y.doubleValue());
+  }
+
+  /**  Constructs a 3D point from coordinates. */
+  public static Geom ST_Point(BigDecimal x, BigDecimal y, BigDecimal z) {
+    final Geometry g = new Point(x.doubleValue(), y.doubleValue(),
+        z.doubleValue());
+    return new SimpleGeom(g);
+  }
+
+  private static Geom point(double x, double y) {
+    final Geometry g = new Point(x, y);
+    return new SimpleGeom(g);
+  }
+
+  // Geometry properties (2D and 3D) ==========================================
+
+  /** Returns whether {@code geom} has at least one z-coordinate. */
+  public static boolean ST_Is3D(Geom geom) {
+    return geom.g().hasZ();
+  }
+
+  /** Returns the z-value of the first coordinate of {@code geom}. */
+  public static Double ST_Z(Geom geom) {
+    return geom.g().getDescription().hasZ() && geom.g() instanceof Point
+        ? ((Point) geom.g()).getZ() : null;
+  }
+
+  /** Returns the boundary of {@code geom}. */
+  public static Geom ST_Boundary(Geom geom) {
+    OperatorBoundary op = OperatorBoundary.local();
+    Geometry result = op.execute(geom.g(), null);
+    return geom.wrap(result);
+  }
+
+  /** Returns the distance between {@code geom1} and {@code geom2}. */
+  public static double ST_Distance(Geom geom1, Geom geom2) {
+    return GeometryEngine.distance(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns the type of {@code geom}. */
+  public static String ST_GeometryType(Geom geom) {
+    return type(geom.g()).name();
+  }
+
+  /** Returns the OGC SFS type code of {@code geom}. */
+  public static int ST_GeometryTypeCode(Geom geom) {
+    return type(geom.g()).code;
+  }
+
+  /** Returns the OGC type of a geometry. */
+  private static Type type(Geometry g) {
+    switch (g.getType()) {
+    case Point:
+      return Type.POINT;
+    case Polyline:
+      return Type.LINESTRING;
+    case Polygon:
+      return Type.POLYGON;
+    case MultiPoint:
+      return Type.MULTIPOINT;
+    case Envelope:
+      return Type.POLYGON;
+    case Line:
+      return Type.LINESTRING;
+    case Unknown:
+      return Type.Geometry;
+    default:
+      throw new AssertionError(g);
+    }
+  }
+
+  /** Returns the minimum bounding box of {@code geom} (which may be a
+   *  GEOMETRYCOLLECTION). */
+  public static Geom ST_Envelope(Geom geom) {
+    final Envelope env = envelope(geom.g());
+    return geom.wrap(env);
+  }
+
+  private static Envelope envelope(Geometry g) {
+    final Envelope env = new Envelope();
+    g.queryEnvelope(env);
+    return env;
+  }
+
+  // Geometry predicates ======================================================
+
+  /** Returns whether {@code geom1} contains {@code geom2}. */
+  public static boolean ST_Contains(Geom geom1, Geom geom2) {
+    return GeometryEngine.contains(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} contains {@code geom2} but does not
+   * intersect its boundary. */
+  public static boolean ST_ContainsProperly(Geom geom1, Geom geom2) {
+    return GeometryEngine.contains(geom1.g(), geom2.g(), geom1.sr())
+        && !GeometryEngine.crosses(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether no point in {@code geom2} is outside {@code geom1}. */
+  private static boolean ST_Covers(Geom geom1, Geom geom2)  {
+    throw todo();
+  }
+
+  /** Returns whether {@code geom1} crosses {@code geom2}. */
+  public static boolean ST_Crosses(Geom geom1, Geom geom2)  {
+    return GeometryEngine.crosses(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} and {@code geom2} are disjoint. */
+  public static boolean ST_Disjoint(Geom geom1, Geom geom2)  {
+    return GeometryEngine.disjoint(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether the envelope of {@code geom1} intersects the envelope of
+   *  {@code geom2}. */
+  public static boolean ST_EnvelopesIntersect(Geom geom1, Geom geom2)  {
+    final Geometry e1 = envelope(geom1.g());
+    final Geometry e2 = envelope(geom2.g());
+    return intersects(e1, e2, geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} equals {@code geom2}. */
+  public static boolean ST_Equals(Geom geom1, Geom geom2)  {
+    return GeometryEngine.equals(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} intersects {@code geom2}. */
+  public static boolean ST_Intersects(Geom geom1, Geom geom2)  {
+    final Geometry g1 = geom1.g();
+    final Geometry g2 = geom2.g();
+    final SpatialReference sr = geom1.sr();
+    return intersects(g1, g2, sr);
+  }
+
+  private static boolean intersects(Geometry g1, Geometry g2,
+      SpatialReference sr) {
+    final OperatorIntersects op = (OperatorIntersects) OperatorFactoryLocal
+        .getInstance().getOperator(Operator.Type.Intersects);
+    return op.execute(g1, g2, sr, null);
+  }
+
+  /** Returns whether {@code geom1} equals {@code geom2} and their coordinates
+   * and component Geometries are listed in the same order. */
+  public static boolean ST_OrderingEquals(Geom geom1, Geom geom2)  {
+    return GeometryEngine.equals(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns {@code geom1} overlaps {@code geom2}. */
+  public static boolean ST_Overlaps(Geom geom1, Geom geom2)  {
+    return GeometryEngine.overlaps(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} touches {@code geom2}. */
+  public static boolean ST_Touches(Geom geom1, Geom geom2)  {
+    return GeometryEngine.touches(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} is within {@code geom2}. */
+  public static boolean ST_Within(Geom geom1, Geom geom2)  {
+    return GeometryEngine.within(geom1.g(), geom2.g(), geom1.sr());
+  }
+
+  /** Returns whether {@code geom1} and {@code geom2} are within
+   * {@code distance} of each other. */
+  public static boolean ST_DWithin(Geom geom1, Geom geom2, double distance) {
+    final double distance1 =
+        GeometryEngine.distance(geom1.g(), geom2.g(), geom1.sr());
+    return distance1 <= distance;
+  }
+
+  // Geometry operators (2D and 3D) ===========================================
+
+  /** Computes a buffer around {@code geom}. */
+  public static Geom ST_Buffer(Geom geom, double distance) {
+    final Polygon g = GeometryEngine.buffer(geom.g(), geom.sr(), distance);
+    return geom.wrap(g);
+  }
+
+  /** Computes a buffer around {@code geom} with . */
+  public static Geom ST_Buffer(Geom geom, double distance, int quadSegs) {
+    throw todo();
+  }
+
+  /** Computes a buffer around {@code geom}. */
+  public static Geom ST_Buffer(Geom geom, double bufferSize, String style) {
+    int quadSegCount = 8;
+    CapStyle endCapStyle = CapStyle.ROUND;
+    JoinStyle joinStyle = JoinStyle.ROUND;
+    float mitreLimit = 5f;
+    int i = 0;
+    parse:
+    for (;;) {
+      int equals = style.indexOf('=', i);
+      if (equals < 0) {
+        break;
+      }
+      int space = style.indexOf(' ', equals);
+      if (space < 0) {
+        space = style.length();
+      }
+      String name = style.substring(i, equals);
+      String value = style.substring(equals + 1, space);
+      switch (name) {
+      case "quad_segs":
+        quadSegCount = Integer.valueOf(value);
+        break;
+      case "endcap":
+        endCapStyle = CapStyle.of(value);
+        break;
+      case "join":
+        joinStyle = JoinStyle.of(value);
+        break;
+      case "mitre_limit":
+      case "miter_limit":
+        mitreLimit = Float.parseFloat(value);
+        break;
+      default:
+        // ignore the value
+      }
+      i = space;
+      for (;;) {
+        if (i >= style.length()) {
+          break parse;
+        }
+        if (style.charAt(i) != ' ') {
+          break;
+        }
+        ++i;
+      }
+    }
+    return buffer(geom, bufferSize, quadSegCount, endCapStyle, joinStyle,
+        mitreLimit);
+  }
+
+  private static Geom buffer(Geom geom, double bufferSize,
+      int quadSegCount, CapStyle endCapStyle, JoinStyle joinStyle,
+      float mitreLimit) {
+    Util.discard(endCapStyle + ":" + joinStyle + ":" + mitreLimit
+        + ":" + quadSegCount);
+    throw todo();
+  }
+
+  /** Computes the union of {@code geom1} and {@code geom2}. */
+  public static Geom ST_Union(Geom geom1, Geom geom2) {
+    SpatialReference sr = geom1.sr();
+    final Geometry g =
+        GeometryEngine.union(new Geometry[]{geom1.g(), geom2.g()}, sr);
+    return bind(g, sr);
+  }
+
+  /** Computes the union of the geometries in {@code geomCollection}. */
+  @SemiStrict public static Geom ST_Union(Geom geomCollection) {
+    SpatialReference sr = geomCollection.sr();
+    final Geometry g =
+        GeometryEngine.union(new Geometry[] {geomCollection.g()}, sr);
+    return bind(g, sr);
+  }
+
+  // Geometry projection functions ============================================
+
+  /** Transforms {@code geom} from one coordinate reference
+   * system (CRS) to the CRS specified by {@code srid}. */
+  public static Geom ST_Transform(Geom geom, int srid) {
+    return geom.transform(srid);
+  }
+
+  /** Returns a copy of {@code geom} with a new SRID. */
+  public static Geom ST_SetSRID(Geom geom, int srid) {
+    return geom.transform(srid);
+  }
+
+  // Inner classes ============================================================
+
+  /** How the "buffer" command terminates the end of a line. */
+  enum CapStyle {
+    ROUND, FLAT, SQUARE;
+
+    static CapStyle of(String value) {
+      switch (value) {
+      case "round":
+        return ROUND;
+      case "flat":
+      case "butt":
+        return FLAT;
+      case "square":
+        return SQUARE;
+      default:
+        throw new IllegalArgumentException("unknown endcap value: " + value);
+      }
+    }
+  }
+
+  /** How the "buffer" command decorates junctions between line segments. */
+  enum JoinStyle {
+    ROUND, MITRE, BEVEL;
+
+    static JoinStyle of(String value) {
+      switch (value) {
+      case "round":
+        return ROUND;
+      case "mitre":
+      case "miter":
+        return MITRE;
+      case "bevel":
+        return BEVEL;
+      default:
+        throw new IllegalArgumentException("unknown join value: " + value);
+      }
+    }
+  }
+
+  /** Geometry. It may or may not have a spatial reference
+   * associated with it. */
+  public interface Geom {
+    Geometry g();
+
+    SpatialReference sr();
+
+    Geom transform(int srid);
+
+    Geom wrap(Geometry g);
+  }
+
+  /** Sub-class of geometry that has no spatial reference. */
+  static class SimpleGeom implements Geom {
+    final Geometry g;
+
+    SimpleGeom(Geometry g) {
+      this.g = Preconditions.checkNotNull(g);
+    }
+
+    @Override public String toString() {
+      return g.toString();
+    }
+
+    public Geometry g() {
+      return g;
+    }
+
+    public SpatialReference sr() {
+      return SPATIAL_REFERENCE;
+    }
+
+    public Geom transform(int srid) {
+      if (srid == SPATIAL_REFERENCE.getID()) {
+        return this;
+      }
+      return bind(g, srid);
+    }
+
+    public Geom wrap(Geometry g) {
+      return new SimpleGeom(g);
+    }
+  }
+
+  /** Sub-class of geometry that has a spatial reference. */
+  static class MapGeom implements Geom {
+    final MapGeometry mg;
+
+    MapGeom(MapGeometry mg) {
+      this.mg = Preconditions.checkNotNull(mg);
+    }
+
+    @Override public String toString() {
+      return mg.toString();
+    }
+
+    public Geometry g() {
+      return mg.getGeometry();
+    }
+
+    public SpatialReference sr() {
+      return mg.getSpatialReference();
+    }
+
+    public Geom transform(int srid) {
+      if (srid == NO_SRID) {
+        return new SimpleGeom(mg.getGeometry());
+      }
+      if (srid == mg.getSpatialReference().getID()) {
+        return this;
+      }
+      return bind(mg.getGeometry(), srid);
+    }
+
+    public Geom wrap(Geometry g) {
+      return bind(g, this.mg.getSpatialReference());
+    }
+  }
+
+  /** Geometry types, with the names and codes assigned by OGC. */
+  enum Type {
+    Geometry(0),
+    POINT(1),
+    LINESTRING(2),
+    POLYGON(3),
+    MULTIPOINT(4),
+    MULTILINESTRING(5),
+    MULTIPOLYGON(6),
+    GEOMCOLLECTION(7),
+    CURVE(13),
+    SURFACE(14),
+    POLYHEDRALSURFACE(15);
+
+    final int code;
+
+    Type(int code) {
+      this.code = code;
+    }
+  }
+}
+
+// End GeoFunctions.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
index c9d775d..e883115 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
@@ -42,6 +42,9 @@ public interface ExtraSqlTypes {
   int REF_CURSOR = 2012;
   int TIME_WITH_TIMEZONE = 2013;
   int TIMESTAMP_WITH_TIMEZONE = 2014;
+
+  // From OpenGIS
+  int GEOMETRY = 2015; // TODO: confirm
 }
 
 // End ExtraSqlTypes.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java b/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
index c096dae..f2f3c80 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql.type;
 
 import org.apache.calcite.avatica.util.ArrayImpl;
+import org.apache.calcite.runtime.GeoFunctions;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -70,6 +71,8 @@ public class JavaToSqlTypeConversionRules {
           .put(Time.class, SqlTypeName.TIME)
           .put(BigDecimal.class, SqlTypeName.DECIMAL)
 
+          .put(GeoFunctions.Geom.class, SqlTypeName.GEOMETRY)
+
           .put(ResultSet.class, SqlTypeName.CURSOR)
           .put(ColumnList.class, SqlTypeName.COLUMN_LIST)
           .put(ArrayImpl.class, SqlTypeName.ARRAY)

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
index 9753f46..f77d639 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
@@ -161,6 +161,9 @@ public class SqlTypeAssignmentRules {
     // Timestamp is assignable from ...
     rules.put(SqlTypeName.TIMESTAMP, EnumSet.of(SqlTypeName.TIMESTAMP));
 
+    // Geometry is assignable from ...
+    rules.put(SqlTypeName.GEOMETRY, EnumSet.of(SqlTypeName.GEOMETRY));
+
     // Array is assignable from ...
     rules.put(SqlTypeName.ARRAY, EnumSet.of(SqlTypeName.ARRAY));
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
index 4543b00..e88f96a 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
@@ -66,7 +66,8 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
   NULL,
   ANY,
   CURSOR,
-  COLUMN_LIST;
+  COLUMN_LIST,
+  GEO;
 
   private static final Map<Integer, SqlTypeFamily> JDBC_TYPE_TO_FAMILY =
       ImmutableMap.<Integer, SqlTypeFamily>builder()
@@ -150,6 +151,8 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
       return SqlTypeName.DATETIME_TYPES;
     case DATETIME_INTERVAL:
       return SqlTypeName.INTERVAL_TYPES;
+    case GEO:
+      return ImmutableList.of(SqlTypeName.GEOMETRY);
     case MULTISET:
       return ImmutableList.of(SqlTypeName.MULTISET);
     case ARRAY:

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index f1b4e6a..fcae1ec 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -115,7 +115,8 @@ public enum SqlTypeName {
   COLUMN_LIST(PrecScale.NO_NO, false, Types.OTHER + 2,
       SqlTypeFamily.COLUMN_LIST),
   DYNAMIC_STAR(PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES, true,
-      Types.JAVA_OBJECT, SqlTypeFamily.ANY);
+      Types.JAVA_OBJECT, SqlTypeFamily.ANY),
+  GEOMETRY(PrecScale.NO_NO, true, ExtraSqlTypes.GEOMETRY, SqlTypeFamily.GEO);
 
   public static final int MAX_DATETIME_PRECISION = 3;
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
index 6fa2483..dbf8b6b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
@@ -82,6 +82,10 @@ public abstract class SqlAbstractConformance implements SqlConformance {
   public boolean isPercentRemainderAllowed() {
     return SqlConformanceEnum.DEFAULT.isPercentRemainderAllowed();
   }
+
+  public boolean allowGeometry() {
+    return SqlConformanceEnum.DEFAULT.allowGeometry();
+  }
 }
 
 // End SqlAbstractConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index 2b37026..76b6da9 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -294,8 +294,18 @@ public interface SqlConformance {
    * {@link SqlConformanceEnum#MYSQL_5};
    * false otherwise.
    */
-
   boolean isLimitStartCountAllowed();
+
+  /**
+   * Whether to allow geo-spatial extensions, including the GEOMETRY type.
+   *
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#LENIENT},
+   * {@link SqlConformanceEnum#MYSQL_5},
+   * {@link SqlConformanceEnum#SQL_SERVER_2008};
+   * false otherwise.
+   */
+  boolean allowGeometry();
 }
 
 // End SqlConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
index 9f06c18..1a5752f 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
@@ -228,6 +228,17 @@ public enum SqlConformanceEnum implements SqlConformance {
       return false;
     }
   }
+
+  public boolean allowGeometry() {
+    switch (this) {
+    case LENIENT:
+    case MYSQL_5:
+    case SQL_SERVER_2008:
+      return true;
+    default:
+      return false;
+    }
+  }
 }
 
 // End SqlConformanceEnum.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 52168b0..1f52475 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -25,6 +25,7 @@ IllegalLiteral=Illegal {0} literal {1}: {2}
 IdentifierTooLong=Length of identifier ''{0}'' must be less than or equal to {1,number,#} characters
 BadFormat=not in format ''{0}''
 BetweenWithoutAnd=BETWEEN operator has no terminating AND
+GeometryDisabled=Geo-spatial extensions and the GEOMETRY data type are not enabled
 IllegalIntervalLiteral=Illegal INTERVAL literal {0}; at {1}
 IllegalMinusDate=Illegal expression. Was expecting "(DATETIME - DATETIME) INTERVALQUALIFIER"
 IllegalOverlaps=Illegal overlaps expression. Was expecting expression on the form "(DATETIME, EXPRESSION) OVERLAPS (DATETIME, EXPRESSION)"

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
index ab68285..9dd3019 100644
--- a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
+++ b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
@@ -274,7 +274,7 @@ public class CalciteRemoteDriverTest {
   @Test public void testRemoteTypeInfo() throws Exception {
     CalciteAssert.hr().with(REMOTE_CONNECTION_FACTORY)
         .metaData(GET_TYPEINFO)
-        .returns(CalciteAssert.checkResultCount(is(42)));
+        .returns(CalciteAssert.checkResultCount(is(43)));
   }
 
   @Test public void testRemoteTableTypes() throws Exception {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/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 435e906..e91a76c 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
@@ -6710,6 +6710,13 @@ public class SqlParserTest {
         "(?s)Encountered \"to\".*");
   }
 
+  @Test public void testGeometry() {
+    checkExpFails("cast(null as geometry)",
+        "Geo-spatial extensions and the GEOMETRY data type are not enabled");
+    conformance = SqlConformanceEnum.LENIENT;
+    checkExp("cast(null as geometry)", "CAST(NULL AS GEOMETRY)");
+  }
+
   @Test public void testIntervalArithmetics() {
     checkExp(
         "TIME '23:59:59' - interval '1' hour ",

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 490c7e5..6db9435 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -28,15 +28,19 @@ import org.apache.calcite.jdbc.CalciteMetaImpl;
 import org.apache.calcite.jdbc.CalcitePrepare;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.model.ModelHandler;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.GeoFunctions;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.schema.impl.ViewTableMacro;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.RelBuilder;
@@ -753,6 +757,17 @@ public class CalciteAssert {
             CalciteAssert.addSchema(rootSchema, SchemaSpec.JDBC_FOODMART);
       }
       return rootSchema.add("foodmart2", new CloneSchema(foodmart));
+    case GEO:
+      ModelHandler.addFunctions(rootSchema, null, ImmutableList.<String>of(),
+          GeoFunctions.class.getName(), "*", true);
+      final SchemaPlus s = rootSchema.add("GEO", new AbstractSchema());
+      ModelHandler.addFunctions(s, "countries", ImmutableList.<String>of(),
+          CountriesTableFunction.class.getName(), null, false);
+      final String sql = "select * from table(\"countries\"(true))";
+      final ViewTableMacro viewMacro = ViewTable.viewMacro(rootSchema, sql,
+          ImmutableList.of("GEO"), ImmutableList.<String>of(), false);
+      s.add("countries", viewMacro);
+      return s;
     case HR:
       return rootSchema.add("hr",
           new ReflectiveSchema(new JdbcTest.HrSchema()));
@@ -863,6 +878,10 @@ public class CalciteAssert {
             SchemaSpec.POST);
       case REGULAR_PLUS_METADATA:
         return with(SchemaSpec.HR, SchemaSpec.REFLECTIVE_FOODMART);
+      case GEO:
+        return with(SchemaSpec.GEO)
+            .with(CalciteConnectionProperty.CONFORMANCE.camelName(),
+                SqlConformanceEnum.LENIENT);
       case LINGUAL:
         return with(SchemaSpec.LINGUAL);
       case JDBC_FOODMART:
@@ -1636,6 +1655,9 @@ public class CalciteAssert {
      * database. */
     FOODMART_CLONE,
 
+    /** Configuration that contains geo-spatial functions. */
+    GEO,
+
     /** Configuration that contains an in-memory clone of the FoodMart
      * database, plus a lattice to enable on-the-fly materializations. */
     JDBC_FOODMART_WITH_LATTICE,
@@ -1761,6 +1783,7 @@ public class CalciteAssert {
     JDBC_FOODMART,
     CLONE_FOODMART,
     JDBC_FOODMART_WITH_LATTICE,
+    GEO,
     HR,
     JDBC_SCOTT,
     SCOTT,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/test/java/org/apache/calcite/test/CountriesTableFunction.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CountriesTableFunction.java b/core/src/test/java/org/apache/calcite/test/CountriesTableFunction.java
new file mode 100644
index 0000000..b909241
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/CountriesTableFunction.java
@@ -0,0 +1,327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableList;
+
+/** A table function that returns all countries in the world.
+ *
+ * <p>Has same content as
+ * <code>file/src/test/resources/geo/countries.csv</code>. */
+public class CountriesTableFunction {
+  private CountriesTableFunction() {}
+
+  private static final Object[][] ROWS = {
+      {"AD", 42.546245, 1.601554, "Andorra"},
+      {"AE", 23.424076, 53.847818, "United Arab Emirates"},
+      {"AF", 33.93911, 67.709953, "Afghanistan"},
+      {"AG", 17.060816, -61.796428, "Antigua and Barbuda"},
+      {"AI", 18.220554, -63.068615, "Anguilla"},
+      {"AL", 41.153332, 20.168331, "Albania"},
+      {"AM", 40.069099, 45.038189, "Armenia"},
+      {"AN", 12.226079, -69.060087, "Netherlands Antilles"},
+      {"AO", -11.202692, 17.873887, "Angola"},
+      {"AQ", -75.250973, -0.071389, "Antarctica"},
+      {"AR", -38.416097, -63.616672, "Argentina"},
+      {"AS", -14.270972, -170.132217, "American Samoa"},
+      {"AT", 47.516231, 14.550072, "Austria"},
+      {"AU", -25.274398, 133.775136, "Australia"},
+      {"AW", 12.52111, -69.968338, "Aruba"},
+      {"AZ", 40.143105, 47.576927, "Azerbaijan"},
+      {"BA", 43.915886, 17.679076, "Bosnia and Herzegovina"},
+      {"BB", 13.193887, -59.543198, "Barbados"},
+      {"BD", 23.684994, 90.356331, "Bangladesh"},
+      {"BE", 50.503887, 4.469936, "Belgium"},
+      {"BF", 12.238333, -1.561593, "Burkina Faso"},
+      {"BG", 42.733883, 25.48583, "Bulgaria"},
+      {"BH", 25.930414, 50.637772, "Bahrain"},
+      {"BI", -3.373056, 29.918886, "Burundi"},
+      {"BJ", 9.30769, 2.315834, "Benin"},
+      {"BM", 32.321384, -64.75737, "Bermuda"},
+      {"BN", 4.535277, 114.727669, "Brunei"},
+      {"BO", -16.290154, -63.588653, "Bolivia"},
+      {"BR", -14.235004, -51.92528, "Brazil"},
+      {"BS", 25.03428, -77.39628, "Bahamas"},
+      {"BT", 27.514162, 90.433601, "Bhutan"},
+      {"BV", -54.423199, 3.413194, "Bouvet Island"},
+      {"BW", -22.328474, 24.684866, "Botswana"},
+      {"BY", 53.709807, 27.953389, "Belarus"},
+      {"BZ", 17.189877, -88.49765, "Belize"},
+      {"CA", 56.130366, -106.346771, "Canada"},
+      {"CC", -12.164165, 96.870956, "Cocos [Keeling] Islands"},
+      {"CD", -4.038333, 21.758664, "Congo [DRC]"},
+      {"CF", 6.611111, 20.939444, "Central African Republic"},
+      {"CG", -0.228021, 15.827659, "Congo [Republic]"},
+      {"CH", 46.818188, 8.227512, "Switzerland"},
+      {"CI", 7.539989, -5.54708, "Côte d'Ivoire"},
+      {"CK", -21.236736, -159.777671, "Cook Islands"},
+      {"CL", -35.675147, -71.542969, "Chile"},
+      {"CM", 7.369722, 12.354722, "Cameroon"},
+      {"CN", 35.86166, 104.195397, "China"},
+      {"CO", 4.570868, -74.297333, "Colombia"},
+      {"CR", 9.748917, -83.753428, "Costa Rica"},
+      {"CU", 21.521757, -77.781167, "Cuba"},
+      {"CV", 16.002082, -24.013197, "Cape Verde"},
+      {"CX", -10.447525, 105.690449, "Christmas Island"},
+      {"CY", 35.126413, 33.429859, "Cyprus"},
+      {"CZ", 49.817492, 15.472962, "Czech Republic"},
+      {"DE", 51.165691, 10.451526, "Germany"},
+      {"DJ", 11.825138, 42.590275, "Djibouti"},
+      {"DK", 56.26392, 9.501785, "Denmark"},
+      {"DM", 15.414999, -61.370976, "Dominica"},
+      {"DO", 18.735693, -70.162651, "Dominican Republic"},
+      {"DZ", 28.033886, 1.659626, "Algeria"},
+      {"EC", -1.831239, -78.183406, "Ecuador"},
+      {"EE", 58.595272, 25.013607, "Estonia"},
+      {"EG", 26.820553, 30.802498, "Egypt"},
+      {"EH", 24.215527, -12.885834, "Western Sahara"},
+      {"ER", 15.179384, 39.782334, "Eritrea"},
+      {"ES", 40.463667, -3.74922, "Spain"},
+      {"ET", 9.145, 40.489673, "Ethiopia"},
+      {"FI", 61.92411, 25.748151, "Finland"},
+      {"FJ", -16.578193, 179.414413, "Fiji"},
+      {"FK", -51.796253, -59.523613, "Falkland Islands [Islas Malvinas]"},
+      {"FM", 7.425554, 150.550812, "Micronesia"},
+      {"FO", 61.892635, -6.911806, "Faroe Islands"},
+      {"FR", 46.227638, 2.213749, "France"},
+      {"GA", -0.803689, 11.609444, "Gabon"},
+      {"GB", 55.378051, -3.435973, "United Kingdom"},
+      {"GD", 12.262776, -61.604171, "Grenada"},
+      {"GE", 42.315407, 43.356892, "Georgia"},
+      {"GF", 3.933889, -53.125782, "French Guiana"},
+      {"GG", 49.465691, -2.585278, "Guernsey"},
+      {"GH", 7.946527, -1.023194, "Ghana"},
+      {"GI", 36.137741, -5.345374, "Gibraltar"},
+      {"GL", 71.706936, -42.604303, "Greenland"},
+      {"GM", 13.443182, -15.310139, "Gambia"},
+      {"GN", 9.945587, -9.696645, "Guinea"},
+      {"GP", 16.995971, -62.067641, "Guadeloupe"},
+      {"GQ", 1.650801, 10.267895, "Equatorial Guinea"},
+      {"GR", 39.074208, 21.824312, "Greece"},
+      {"GS", -54.429579, -36.587909, "South Georgia and the South Sandwich Islands"},
+      {"GT", 15.783471, -90.230759, "Guatemala"},
+      {"GU", 13.444304, 144.793731, "Guam"},
+      {"GW", 11.803749, -15.180413, "Guinea-Bissau"},
+      {"GY", 4.860416, -58.93018, "Guyana"},
+      {"GZ", 31.354676, 34.308825, "Gaza Strip"},
+      {"HK", 22.396428, 114.109497, "Hong Kong"},
+      {"HM", -53.08181, 73.504158, "Heard Island and McDonald Islands"},
+      {"HN", 15.199999, -86.241905, "Honduras"},
+      {"HR", 45.1, 15.2, "Croatia"},
+      {"HT", 18.971187, -72.285215, "Haiti"},
+      {"HU", 47.162494, 19.503304, "Hungary"},
+      {"ID", -0.789275, 113.921327, "Indonesia"},
+      {"IE", 53.41291, -8.24389, "Ireland"},
+      {"IL", 31.046051, 34.851612, "Israel"},
+      {"IM", 54.236107, -4.548056, "Isle of Man"},
+      {"IN", 20.593684, 78.96288, "India"},
+      {"IO", -6.343194, 71.876519, "British Indian Ocean Territory"},
+      {"IQ", 33.223191, 43.679291, "Iraq"},
+      {"IR", 32.427908, 53.688046, "Iran"},
+      {"IS", 64.963051, -19.020835, "Iceland"},
+      {"IT", 41.87194, 12.56738, "Italy"},
+      {"JE", 49.214439, -2.13125, "Jersey"},
+      {"JM", 18.109581, -77.297508, "Jamaica"},
+      {"JO", 30.585164, 36.238414, "Jordan"},
+      {"JP", 36.204824, 138.252924, "Japan"},
+      {"KE", -0.023559, 37.906193, "Kenya"},
+      {"KG", 41.20438, 74.766098, "Kyrgyzstan"},
+      {"KH", 12.565679, 104.990963, "Cambodia"},
+      {"KI", -3.370417, -168.734039, "Kiribati"},
+      {"KM", -11.875001, 43.872219, "Comoros"},
+      {"KN", 17.357822, -62.782998, "Saint Kitts and Nevis"},
+      {"KP", 40.339852, 127.510093, "North Korea"},
+      {"KR", 35.907757, 127.766922, "South Korea"},
+      {"KW", 29.31166, 47.481766, "Kuwait"},
+      {"KY", 19.513469, -80.566956, "Cayman Islands"},
+      {"KZ", 48.019573, 66.923684, "Kazakhstan"},
+      {"LA", 19.85627, 102.495496, "Laos"},
+      {"LB", 33.854721, 35.862285, "Lebanon"},
+      {"LC", 13.909444, -60.978893, "Saint Lucia"},
+      {"LI", 47.166, 9.555373, "Liechtenstein"},
+      {"LK", 7.873054, 80.771797, "Sri Lanka"},
+      {"LR", 6.428055, -9.429499, "Liberia"},
+      {"LS", -29.609988, 28.233608, "Lesotho"},
+      {"LT", 55.169438, 23.881275, "Lithuania"},
+      {"LU", 49.815273, 6.129583, "Luxembourg"},
+      {"LV", 56.879635, 24.603189, "Latvia"},
+      {"LY", 26.3351, 17.228331, "Libya"},
+      {"MA", 31.791702, -7.09262, "Morocco"},
+      {"MC", 43.750298, 7.412841, "Monaco"},
+      {"MD", 47.411631, 28.369885, "Moldova"},
+      {"ME", 42.708678, 19.37439, "Montenegro"},
+      {"MG", -18.766947, 46.869107, "Madagascar"},
+      {"MH", 7.131474, 171.184478, "Marshall Islands"},
+      {"MK", 41.608635, 21.745275, "Macedonia [FYROM]"},
+      {"ML", 17.570692, -3.996166, "Mali"},
+      {"MM", 21.913965, 95.956223, "Myanmar [Burma]"},
+      {"MN", 46.862496, 103.846656, "Mongolia"},
+      {"MO", 22.198745, 113.543873, "Macau"},
+      {"MP", 17.33083, 145.38469, "Northern Mariana Islands"},
+      {"MQ", 14.641528, -61.024174, "Martinique"},
+      {"MR", 21.00789, -10.940835, "Mauritania"},
+      {"MS", 16.742498, -62.187366, "Montserrat"},
+      {"MT", 35.937496, 14.375416, "Malta"},
+      {"MU", -20.348404, 57.552152, "Mauritius"},
+      {"MV", 3.202778, 73.22068, "Maldives"},
+      {"MW", -13.254308, 34.301525, "Malawi"},
+      {"MX", 23.634501, -102.552784, "Mexico"},
+      {"MY", 4.210484, 101.975766, "Malaysia"},
+      {"MZ", -18.665695, 35.529562, "Mozambique"},
+      {"NA", -22.95764, 18.49041, "Namibia"},
+      {"NC", -20.904305, 165.618042, "New Caledonia"},
+      {"NE", 17.607789, 8.081666, "Niger"},
+      {"NF", -29.040835, 167.954712, "Norfolk Island"},
+      {"NG", 9.081999, 8.675277, "Nigeria"},
+      {"NI", 12.865416, -85.207229, "Nicaragua"},
+      {"NL", 52.132633, 5.291266, "Netherlands"},
+      {"NO", 60.472024, 8.468946, "Norway"},
+      {"NP", 28.394857, 84.124008, "Nepal"},
+      {"NR", -0.522778, 166.931503, "Nauru"},
+      {"NU", -19.054445, -169.867233, "Niue"},
+      {"NZ", -40.900557, 174.885971, "New Zealand"},
+      {"OM", 21.512583, 55.923255, "Oman"},
+      {"PA", 8.537981, -80.782127, "Panama"},
+      {"PE", -9.189967, -75.015152, "Peru"},
+      {"PF", -17.679742, -149.406843, "French Polynesia"},
+      {"PG", -6.314993, 143.95555, "Papua New Guinea"},
+      {"PH", 12.879721, 121.774017, "Philippines"},
+      {"PK", 30.375321, 69.345116, "Pakistan"},
+      {"PL", 51.919438, 19.145136, "Poland"},
+      {"PM", 46.941936, -56.27111, "Saint Pierre and Miquelon"},
+      {"PN", -24.703615, -127.439308, "Pitcairn Islands"},
+      {"PR", 18.220833, -66.590149, "Puerto Rico"},
+      {"PS", 31.952162, 35.233154, "Palestinian Territories"},
+      {"PT", 39.399872, -8.224454, "Portugal"},
+      {"PW", 7.51498, 134.58252, "Palau"},
+      {"PY", -23.442503, -58.443832, "Paraguay"},
+      {"QA", 25.354826, 51.183884, "Qatar"},
+      {"RE", -21.115141, 55.536384, "Réunion"},
+      {"RO", 45.943161, 24.96676, "Romania"},
+      {"RS", 44.016521, 21.005859, "Serbia"},
+      {"RU", 61.52401, 105.318756, "Russia"},
+      {"RW", -1.940278, 29.873888, "Rwanda"},
+      {"SA", 23.885942, 45.079162, "Saudi Arabia"},
+      {"SB", -9.64571, 160.156194, "Solomon Islands"},
+      {"SC", -4.679574, 55.491977, "Seychelles"},
+      {"SD", 12.862807, 30.217636, "Sudan"},
+      {"SE", 60.128161, 18.643501, "Sweden"},
+      {"SG", 1.352083, 103.819836, "Singapore"},
+      {"SH", -24.143474, -10.030696, "Saint Helena"},
+      {"SI", 46.151241, 14.995463, "Slovenia"},
+      {"SJ", 77.553604, 23.670272, "Svalbard and Jan Mayen"},
+      {"SK", 48.669026, 19.699024, "Slovakia"},
+      {"SL", 8.460555, -11.779889, "Sierra Leone"},
+      {"SM", 43.94236, 12.457777, "San Marino"},
+      {"SN", 14.497401, -14.452362, "Senegal"},
+      {"SO", 5.152149, 46.199616, "Somalia"},
+      {"SR", 3.919305, -56.027783, "Suriname"},
+      {"ST", 0.18636, 6.613081, "São Tomé and Príncipe"},
+      {"SV", 13.794185, -88.89653, "El Salvador"},
+      {"SY", 34.802075, 38.996815, "Syria"},
+      {"SZ", -26.522503, 31.465866, "Swaziland"},
+      {"TC", 21.694025, -71.797928, "Turks and Caicos Islands"},
+      {"TD", 15.454166, 18.732207, "Chad"},
+      {"TF", -49.280366, 69.348557, "French Southern Territories"},
+      {"TG", 8.619543, 0.824782, "Togo"},
+      {"TH", 15.870032, 100.992541, "Thailand"},
+      {"TJ", 38.861034, 71.276093, "Tajikistan"},
+      {"TK", -8.967363, -171.855881, "Tokelau"},
+      {"TL", -8.874217, 125.727539, "Timor-Leste"},
+      {"TM", 38.969719, 59.556278, "Turkmenistan"},
+      {"TN", 33.886917, 9.537499, "Tunisia"},
+      {"TO", -21.178986, -175.198242, "Tonga"},
+      {"TR", 38.963745, 35.243322, "Turkey"},
+      {"TT", 10.691803, -61.222503, "Trinidad and Tobago"},
+      {"TV", -7.109535, 177.64933, "Tuvalu"},
+      {"TW", 23.69781, 120.960515, "Taiwan"},
+      {"TZ", -6.369028, 34.888822, "Tanzania"},
+      {"UA", 48.379433, 31.16558, "Ukraine"},
+      {"UG", 1.373333, 32.290275, "Uganda"},
+      {"UM", null, null, "U.S.Minor Outlying Islands"},
+      {"US", 37.09024, -95.712891, "United States"},
+      {"UY", -32.522779, -55.765835, "Uruguay"},
+      {"UZ", 41.377491, 64.585262, "Uzbekistan"},
+      {"VA", 41.902916, 12.453389, "Vatican City"},
+      {"VC", 12.984305, -61.287228, "Saint Vincent and the Grenadines"},
+      {"VE", 6.42375, -66.58973, "Venezuela"},
+      {"VG", 18.420695, -64.639968, "British Virgin Islands"},
+      {"VI", 18.335765, -64.896335, "U.S. Virgin Islands"},
+      {"VN", 14.058324, 108.277199, "Vietnam"},
+      {"VU", -15.376706, 166.959158, "Vanuatu"},
+      {"WF", -13.768752, -177.156097, "Wallis and Futuna"},
+      {"WS", -13.759029, -172.104629, "Samoa"},
+      {"XK", 42.602636, 20.902977, "Kosovo"},
+      {"YE", 15.552727, 48.516388, "Yemen"},
+      {"YT", -12.8275, 45.166244, "Mayotte"},
+      {"ZA", -30.559482, 22.937506, "South Africa"},
+      {"ZM", -13.133897, 27.849332, "Zambia"},
+      {"ZW", -19.015438, 29.154857, "Zimbabwe"},
+  };
+
+  public static ScannableTable eval(boolean b) {
+    return new ScannableTable() {
+      public Enumerable<Object[]> scan(DataContext root) {
+        return Linq4j.asEnumerable(ROWS);
+      };
+
+      public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+        return typeFactory.builder()
+            .add("country", SqlTypeName.VARCHAR)
+            .add("latitude", SqlTypeName.DECIMAL).nullable(true)
+            .add("longitude", SqlTypeName.DECIMAL).nullable(true)
+            .add("name", SqlTypeName.VARCHAR)
+            .build();
+      }
+
+      public Statistic getStatistic() {
+        return Statistics.of(246D,
+            ImmutableList.of(ImmutableBitSet.of(0), ImmutableBitSet.of(3)));
+      }
+
+      public Schema.TableType getJdbcTableType() {
+        return Schema.TableType.TABLE;
+      }
+
+      public boolean isRolledUp(String column) {
+        return false;
+      }
+
+      public boolean rolledUpColumnValidInsideAgg(String column, SqlCall call,
+          SqlNode parent, CalciteConnectionConfig config) {
+        return false;
+      }
+    };
+  }
+}
+
+// End CountriesTableFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
index 28994a2..f30ebc8 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -279,6 +279,10 @@ public class QuidemTest {
         return CalciteAssert.that()
             .with(CalciteAssert.Config.FOODMART_CLONE)
             .connect();
+      case "geo":
+        return CalciteAssert.that()
+            .with(CalciteAssert.Config.GEO)
+            .connect();
       case "scott":
         return CalciteAssert.that()
             .with(CalciteAssert.Config.SCOTT)

http://git-wip-us.apache.org/repos/asf/calcite/blob/cc20ca13/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 f8c644b..44e510e 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -992,6 +992,24 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Binary literal string must contain an even number of hexits");
   }
 
+  /**
+   * Tests whether the GEOMETRY data type is allowed.
+   *
+   * @see SqlConformance#allowGeometry()
+   */
+  @Test public void testGeometry() {
+    final SqlTester lenient =
+        tester.withConformance(SqlConformanceEnum.LENIENT);
+    final SqlTester strict =
+        tester.withConformance(SqlConformanceEnum.STRICT_2003);
+
+    final String err =
+        "Geo-spatial extensions and the GEOMETRY data type are not enabled";
+    sql("select cast(null as geometry) as g from emp")
+        .tester(strict).fails(err)
+        .tester(lenient).sansCarets().ok();
+  }
+
   @Test public void testDateTime() {
     // LOCAL_TIME
     checkExp("LOCALTIME(3)");


[13/16] calcite git commit: [CALCITE-1069] In Aggregate, deprecate indicators, and allow GROUPING to be used as an aggregate function

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 42986b8..235dfb4 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -34,7 +34,6 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.SingleRel;
-import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Collect;
 import org.apache.calcite.rel.core.CorrelationId;
@@ -238,6 +237,7 @@ public class SqlToRelConverter {
   private final SqlNodeToRexConverter exprConverter;
   private int explainParamCount;
   public final SqlToRelConverter.Config config;
+  private final RelBuilder relBuilder;
 
   /**
    * Fields used in name resolution for correlated sub-queries.
@@ -318,6 +318,7 @@ public class SqlToRelConverter {
     this.exprConverter = new SqlNodeToRexConverterImpl(convertletTable);
     this.explainParamCount = 0;
     this.config = new ConfigBuilder().withConfig(config).build();
+    this.relBuilder = RelFactories.LOGICAL_BUILDER.create(cluster, null);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -527,8 +528,6 @@ public class SqlToRelConverter {
    * @return Field trimmer
    */
   protected RelFieldTrimmer newFieldTrimmer() {
-    final RelBuilder relBuilder =
-        RelFactories.LOGICAL_BUILDER.create(cluster, null);
     return new RelFieldTrimmer(validator, relBuilder);
   }
 
@@ -757,9 +756,8 @@ public class SqlToRelConverter {
     // different.
     final ImmutableBitSet groupSet =
         ImmutableBitSet.range(rel.getRowType().getFieldCount());
-    rel =
-        createAggregate(bb, false, groupSet, ImmutableList.of(groupSet),
-            ImmutableList.<AggregateCall>of());
+    rel = createAggregate(bb, groupSet, ImmutableList.of(groupSet),
+        ImmutableList.<AggregateCall>of());
 
     bb.setRoot(
         rel,
@@ -1099,7 +1097,7 @@ public class SqlToRelConverter {
         final int keyCount = leftKeys.size();
         final List<Integer> args = ImmutableIntList.range(0, keyCount);
         LogicalAggregate aggregate =
-            LogicalAggregate.create(seek, false, ImmutableBitSet.of(), null,
+            LogicalAggregate.create(seek, ImmutableBitSet.of(), null,
                 ImmutableList.of(
                     AggregateCall.create(SqlStdOperatorTable.COUNT, false,
                         ImmutableList.<Integer>of(), -1, longType, null),
@@ -2706,7 +2704,7 @@ public class SqlToRelConverter {
       aggConverter.addGroupExpr(groupExpr);
     }
 
-    RexNode havingExpr = null;
+    final RexNode havingExpr;
     final List<Pair<RexNode, String>> projects = Lists.newArrayList();
 
     try {
@@ -2763,42 +2761,10 @@ public class SqlToRelConverter {
 
       // Add the aggregator
       bb.setRoot(
-          createAggregate(bb, r.indicator, r.groupSet, r.groupSets,
+          createAggregate(bb, r.groupSet, r.groupSets,
               aggConverter.getAggCalls()),
           false);
 
-      // Generate NULL values for rolled-up not-null fields.
-      final Aggregate aggregate = (Aggregate) bb.root;
-      if (aggregate.getGroupType() != Aggregate.Group.SIMPLE) {
-        assert aggregate.indicator;
-        List<Pair<RexNode, String>> projects2 = Lists.newArrayList();
-        int converted = 0;
-        final int groupCount = aggregate.getGroupSet().cardinality();
-        for (RelDataTypeField field : aggregate.getRowType().getFieldList()) {
-          final int i = field.getIndex();
-          final RexNode rex;
-          if (i < groupCount && r.isNullable(i)) {
-            ++converted;
-
-            rex = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-                rexBuilder.makeInputRef(aggregate, groupCount + i),
-                rexBuilder.makeCast(
-                    typeFactory.createTypeWithNullability(
-                        field.getType(), true),
-                    rexBuilder.constantNull()),
-                rexBuilder.makeInputRef(aggregate, i));
-          } else {
-            rex = rexBuilder.makeInputRef(aggregate, i);
-          }
-          projects2.add(Pair.of(rex, field.getName()));
-        }
-        if (converted > 0) {
-          bb.setRoot(
-              RelOptUtil.createProject(bb.root, projects2, true),
-              false);
-        }
-      }
-
       bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection);
 
       // Replace sub-queries in having here and modify having to use
@@ -2807,9 +2773,8 @@ public class SqlToRelConverter {
         SqlNode newHaving = pushDownNotForIn(bb.scope, having);
         replaceSubQueries(bb, newHaving, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
         havingExpr = bb.convertExpression(newHaving);
-        if (havingExpr.isAlwaysTrue()) {
-          havingExpr = null;
-        }
+      } else {
+        havingExpr = relBuilder.literal(true);
       }
 
       // Now convert the other sub-queries in the select list.
@@ -2855,19 +2820,15 @@ public class SqlToRelConverter {
     }
 
     // implement HAVING (we have already checked that it is non-trivial)
+    relBuilder.push(bb.root);
     if (havingExpr != null) {
-      final RelFactories.FilterFactory factory =
-          RelFactories.DEFAULT_FILTER_FACTORY;
-      bb.setRoot(factory.createFilter(bb.root, havingExpr), false);
+      relBuilder.filter(havingExpr);
     }
 
     // implement the SELECT list
-    bb.setRoot(
-        RelOptUtil.createProject(
-            bb.root,
-            projects,
-            true),
-        false);
+    relBuilder.project(Pair.left(projects), Pair.right(projects))
+        .rename(Pair.right(projects));
+    bb.setRoot(relBuilder.build(), false);
 
     // Tell bb which of group columns are sorted.
     bb.columnMonotonicities.clear();
@@ -2890,17 +2851,14 @@ public class SqlToRelConverter {
    * parameter.
    *
    * @param bb       Blackboard
-   * @param indicator Whether to output fields indicating grouping sets
    * @param groupSet Bit set of ordinals of grouping columns
    * @param groupSets Grouping sets
    * @param aggCalls Array of calls to aggregate functions
    * @return LogicalAggregate
    */
-  protected RelNode createAggregate(Blackboard bb, boolean indicator,
-      ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
-      List<AggregateCall> aggCalls) {
-    return LogicalAggregate.create(
-        bb.root, indicator, groupSet, groupSets, aggCalls);
+  protected RelNode createAggregate(Blackboard bb, ImmutableBitSet groupSet,
+      ImmutableList<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
+    return LogicalAggregate.create(bb.root, groupSet, groupSets, aggCalls);
   }
 
   public RexDynamicParam convertDynamicParam(
@@ -3537,7 +3495,7 @@ public class SqlToRelConverter {
     }
     final Pair<RexNode, Map<String, Integer>> e0 = bb.lookupExp(qualified);
     RexNode e = e0.left;
-    for (String name : qualified.suffixTranslated()) {
+    for (String name : qualified.suffix()) {
       if (e == e0.left && e0.right != null) {
         int i = e0.right.get(name);
         e = rexBuilder.makeFieldAccess(e, i);
@@ -4199,7 +4157,14 @@ public class SqlToRelConverter {
         if (node == null) {
           return null;
         } else {
-          return Pair.of(node, null);
+          final Map<String, Integer> fieldOffsets = new HashMap<>();
+          for (RelDataTypeField f : resolve.rowType().getFieldList()) {
+            if (!fieldOffsets.containsKey(f.getName())) {
+              fieldOffsets.put(f.getName(), f.getIndex());
+            }
+          }
+          final Map<String, Integer> map = ImmutableMap.copyOf(fieldOffsets);
+          return Pair.of(node, map);
         }
       } else {
         // We're referencing a relational expression which has not been
@@ -4223,8 +4188,7 @@ public class SqlToRelConverter {
             builder.addAll(c.getRowType().getFieldList());
             if (i == resolve.path.steps().get(0).i) {
               for (RelDataTypeField field : c.getRowType().getFieldList()) {
-                fields.put(c.translate(field.getName()),
-                    field.getIndex() + offset);
+                fields.put(field.getName(), field.getIndex() + offset);
               }
             }
             ++i;
@@ -4924,7 +4888,7 @@ public class SqlToRelConverter {
           rexBuilder.addAggCall(
               aggCall,
               groupExprs.size(),
-              r.indicator,
+              false,
               aggCalls,
               aggCallMapping,
               argTypes);
@@ -4964,36 +4928,6 @@ public class SqlToRelConverter {
       // assert call.getOperator().isAggregator();
       assert bb.agg == this;
 
-      switch (call.getKind()) {
-      case GROUPING:
-      case GROUP_ID:
-        final RelDataType type = validator.getValidatedNodeType(call);
-        if (!aggregatingSelectScope.resolved.get().indicator) {
-          return rexBuilder.makeExactLiteral(
-              TWO.pow(effectiveArgCount(call)).subtract(BigDecimal.ONE), type);
-        } else {
-          final List<Integer> operands;
-          switch (call.getKind()) {
-          case GROUP_ID:
-            operands = ImmutableIntList.range(0, groupExprs.size());
-            break;
-          default:
-            operands = Lists.newArrayList();
-            for (SqlNode operand : call.getOperandList()) {
-              final int x = lookupGroupExpr(operand);
-              assert x >= 0;
-              operands.add(x);
-            }
-          }
-          RexNode node = null;
-          int shift = operands.size();
-          for (int operand : operands) {
-            node = bitValue(node, type, operand, --shift);
-          }
-          return node;
-        }
-      }
-
       for (Map.Entry<SqlNode, Ord<AuxiliaryConverter>> e
           : auxiliaryGroupExprs.entrySet()) {
         if (call.equalsDeep(e.getKey(), Litmus.IGNORE)) {
@@ -5008,35 +4942,6 @@ public class SqlToRelConverter {
       return aggMapping.get(call);
     }
 
-    private int effectiveArgCount(SqlCall call) {
-      switch (call.getKind()) {
-      case GROUPING:
-        return call.operandCount();
-      case GROUP_ID:
-        return groupExprs.size();
-      default:
-        throw new AssertionError(call.getKind());
-      }
-    }
-
-    private RexNode bitValue(RexNode previous, RelDataType type, int x,
-        int shift) {
-      final AggregatingSelectScope.Resolved r =
-          aggregatingSelectScope.resolved.get();
-      RexNode node = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-          rexBuilder.makeInputRef(bb.root, r.groupExprList.size() + x),
-          rexBuilder.makeExactLiteral(BigDecimal.ONE, type),
-          rexBuilder.makeExactLiteral(BigDecimal.ZERO, type));
-      if (shift > 0) {
-        node = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, node,
-            rexBuilder.makeExactLiteral(TWO.pow(shift), type));
-      }
-      if (previous != null) {
-        node = rexBuilder.makeCall(SqlStdOperatorTable.PLUS, previous, node);
-      }
-      return node;
-    }
-
     public List<Pair<RexNode, String>> getPreExprs() {
       return convertedInputExprs;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 4ef40df..fe822f0 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -575,11 +575,16 @@ public class RelBuilder {
     return call(SqlStdOperatorTable.NOT, operand);
   }
 
-  /** Creates an =. */
+  /** Creates an {@code =}. */
   public RexNode equals(RexNode operand0, RexNode operand1) {
     return call(SqlStdOperatorTable.EQUALS, operand0, operand1);
   }
 
+  /** Creates a {@code <>}. */
+  public RexNode notEquals(RexNode operand0, RexNode operand1) {
+    return call(SqlStdOperatorTable.NOT_EQUALS, operand0, operand1);
+  }
+
   /** Creates a IS NULL. */
   public RexNode isNull(RexNode operand) {
     return call(SqlStdOperatorTable.IS_NULL, operand);
@@ -681,8 +686,20 @@ public class RelBuilder {
    * <p>This method of creating a group key does not allow you to group on new
    * expressions, only column projections, but is efficient, especially when you
    * are coming from an existing {@link Aggregate}. */
+  public GroupKey groupKey(ImmutableBitSet groupSet,
+      ImmutableList<ImmutableBitSet> groupSets) {
+    return groupKey_(groupSet, false, groupSets);
+  }
+
+  /** @deprecated Use {@link #groupKey(ImmutableBitSet, ImmutableList)}. */
+  @Deprecated // to be removed before 2.0
   public GroupKey groupKey(ImmutableBitSet groupSet, boolean indicator,
       ImmutableList<ImmutableBitSet> groupSets) {
+    return groupKey_(groupSet, indicator, groupSets);
+  }
+
+  private GroupKey groupKey_(ImmutableBitSet groupSet, boolean indicator,
+      ImmutableList<ImmutableBitSet> groupSets) {
     if (groupSet.length() > peek().getRowType().getFieldCount()) {
       throw new IllegalArgumentException("out of bounds: " + groupSet);
     }
@@ -949,6 +966,46 @@ public class RelBuilder {
     return project(ImmutableList.copyOf(nodes));
   }
 
+  /** Ensures that the field names match those given.
+   *
+   * <p>If all fields have the same name, adds nothing;
+   * if any fields do not have the same name, adds a {@link Project}.
+   *
+   * <p>Note that the names can be short-lived. Other {@code RelBuilder}
+   * operations make no guarantees about the field names of the rows they
+   * produce.
+   *
+   * @param fieldNames List of desired field names; may contain null values or
+   * have fewer fields than the current row type
+   */
+  public RelBuilder rename(List<String> fieldNames) {
+    final List<String> oldFieldNames = peek().getRowType().getFieldNames();
+    Preconditions.checkArgument(fieldNames.size() <= oldFieldNames.size(),
+        "More names than fields");
+    final List<String> newFieldNames = new ArrayList<>(oldFieldNames);
+    for (int i = 0; i < fieldNames.size(); i++) {
+      final String s = fieldNames.get(i);
+      if (s != null) {
+        newFieldNames.set(i, s);
+      }
+    }
+    if (oldFieldNames.equals(newFieldNames)) {
+      return this;
+    }
+    project(fields(), newFieldNames, true);
+
+    // If, after de-duplication, the field names are unchanged, discard the
+    // identity project we just created.
+    if (peek().getRowType().getFieldNames().equals(oldFieldNames)) {
+      final RelNode r = peek();
+      if (r instanceof Project) {
+        stack.pop();
+        push(((Project) r).getInput());
+      }
+    }
+    return this;
+  }
+
   /** Infers the alias of an expression.
    *
    * <p>If the expression was created by {@link #alias}, replaces the expression
@@ -992,8 +1049,7 @@ public class RelBuilder {
   /** Creates an {@link org.apache.calcite.rel.core.Aggregate} with a list of
    * calls. */
   public RelBuilder aggregate(GroupKey groupKey, Iterable<AggCall> aggCalls) {
-    final RelDataType inputRowType = peek().getRowType();
-    final List<RexNode> extraNodes = projects(inputRowType);
+    final List<RexNode> extraNodes = new ArrayList<>(fields());
     final GroupKeyImpl groupKey_ = (GroupKeyImpl) groupKey;
     final ImmutableBitSet groupSet =
         ImmutableBitSet.of(registerExpressions(extraNodes, groupKey_.nodes));
@@ -1051,9 +1107,7 @@ public class RelBuilder {
         }
       }
     }
-    if (extraNodes.size() > inputRowType.getFieldCount()) {
-      project(extraNodes);
-    }
+    project(extraNodes);
     final Frame frame = stack.pop();
     final RelNode r = frame.rel;
     final List<AggregateCall> aggregateCalls = new ArrayList<>();
@@ -1064,6 +1118,12 @@ public class RelBuilder {
         final List<Integer> args = registerExpressions(extraNodes, aggCall1.operands);
         final int filterArg = aggCall1.filter == null ? -1
             : registerExpression(extraNodes, aggCall1.filter);
+        if (aggCall1.distinct && !aggCall1.aggFunction.isQuantifierAllowed()) {
+          throw new IllegalArgumentException("DISTINCT not allowed");
+        }
+        if (aggCall1.filter != null && !aggCall1.aggFunction.allowsFilter()) {
+          throw new IllegalArgumentException("FILTER not allowed");
+        }
         aggregateCall =
             AggregateCall.create(aggCall1.aggFunction, aggCall1.distinct, args,
                 filterArg, groupSet.cardinality(), r, null, aggCall1.alias);
@@ -1124,15 +1184,6 @@ public class RelBuilder {
     return this;
   }
 
-  private List<RexNode> projects(RelDataType inputRowType) {
-    final List<RexNode> exprList = new ArrayList<>();
-    for (RelDataTypeField field : inputRowType.getFieldList()) {
-      final RexBuilder rexBuilder = cluster.getRexBuilder();
-      exprList.add(rexBuilder.makeInputRef(field.getType(), field.getIndex()));
-    }
-    return exprList;
-  }
-
   private static int registerExpression(List<RexNode> exprList, RexNode node) {
     int i = exprList.indexOf(node);
     if (i < 0) {
@@ -1545,9 +1596,8 @@ public class RelBuilder {
   public RelBuilder sortLimit(int offset, int fetch,
       Iterable<? extends RexNode> nodes) {
     final List<RelFieldCollation> fieldCollations = new ArrayList<>();
-    final RelDataType inputRowType = peek().getRowType();
-    final List<RexNode> extraNodes = projects(inputRowType);
-    final List<RexNode> originalExtraNodes = ImmutableList.copyOf(extraNodes);
+    final List<RexNode> originalExtraNodes = fields();
+    final List<RexNode> extraNodes = new ArrayList<>(originalExtraNodes);
     for (RexNode node : nodes) {
       fieldCollations.add(
           collation(node, RelFieldCollation.Direction.ASCENDING, null,
@@ -1705,6 +1755,7 @@ public class RelBuilder {
     GroupKeyImpl(ImmutableList<RexNode> nodes, boolean indicator,
         ImmutableList<ImmutableList<RexNode>> nodeLists, String alias) {
       this.nodes = Preconditions.checkNotNull(nodes);
+      assert !indicator;
       this.indicator = indicator;
       this.nodeLists = nodeLists;
       this.alias = alias;

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 1f52475..74d0c0d 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -107,7 +107,6 @@ GroupingInWrongClause={0} operator may only occur in SELECT, HAVING or ORDER BY
 NotSelectDistinctExpr=Expression ''{0}'' is not in the select clause
 AggregateIllegalInClause=Aggregate expression is illegal in {0} clause
 WindowedAggregateIllegalInClause=Windowed aggregate expression is illegal in {0} clause
-AggregateIllegalInGroupBy=Aggregate expression is illegal in GROUP BY clause
 NestedAggIllegal=Aggregate expressions cannot be nested
 AggregateInFilterIllegal=FILTER must not contain aggregate expression
 AggregateIllegalInOrderBy=Aggregate expression is illegal in ORDER BY clause of non-aggregating SELECT

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
index 5869024..9749b6d 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
@@ -138,8 +138,7 @@ public class RelWriterTest {
                 final RelDataType bigIntType =
                     cluster.getTypeFactory().createSqlType(SqlTypeName.BIGINT);
                 LogicalAggregate aggregate =
-                    LogicalAggregate.create(filter, false,
-                        ImmutableBitSet.of(0), null,
+                    LogicalAggregate.create(filter, ImmutableBitSet.of(0), null,
                         ImmutableList.of(
                             AggregateCall.create(SqlStdOperatorTable.COUNT,
                                 true, ImmutableList.of(1), -1, bigIntType,

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index 4eafeae..8a337cc 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -487,6 +487,64 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testRename() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+
+    // No rename necessary (null name is ignored)
+    RelNode root =
+        builder.scan("DEPT")
+            .rename(Arrays.asList("DEPTNO", null))
+            .build();
+    final String expected = "LogicalTableScan(table=[[scott, DEPT]])\n";
+    assertThat(str(root), is(expected));
+
+    // No rename necessary (prefix matches)
+    root =
+        builder.scan("DEPT")
+            .rename(ImmutableList.of("DEPTNO"))
+            .build();
+    assertThat(str(root), is(expected));
+
+    // Add project to rename fields
+    root =
+        builder.scan("DEPT")
+            .rename(Arrays.asList("NAME", null, "DEPTNO"))
+            .build();
+    final String expected2 = ""
+        + "LogicalProject(NAME=[$0], DNAME=[$1], DEPTNO=[$2])\n"
+        + "  LogicalTableScan(table=[[scott, DEPT]])\n";
+    assertThat(str(root), is(expected2));
+
+    // If our requested list has non-unique names, we might get the same field
+    // names we started with. Don't add a useless project.
+    root =
+        builder.scan("DEPT")
+            .rename(Arrays.asList("DEPTNO", null, "DEPTNO"))
+            .build();
+    final String expected3 = ""
+        + "LogicalProject(DEPTNO=[$0], DNAME=[$1], DEPTNO0=[$2])\n"
+        + "  LogicalTableScan(table=[[scott, DEPT]])\n";
+    assertThat(str(root), is(expected3));
+    root =
+        builder.scan("DEPT")
+            .rename(Arrays.asList("DEPTNO", null, "DEPTNO"))
+            .rename(Arrays.asList("DEPTNO", null, "DEPTNO"))
+            .build();
+    // No extra Project
+    assertThat(str(root), is(expected3));
+
+    // Name list too long
+    try {
+      root =
+          builder.scan("DEPT")
+              .rename(ImmutableList.of("NAME", "DEPTNO", "Y", "Z"))
+              .build();
+      fail("expected error, got " + root);
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("More names than fields"));
+    }
+  }
+
   @Test public void testPermute() {
     final RelBuilder builder = RelBuilder.create(config().build());
     RelNode root =
@@ -587,7 +645,7 @@ public class RelBuilderTest {
     RelNode root =
         builder.scan("EMP")
             .aggregate(
-                builder.groupKey(ImmutableBitSet.of(7), true,
+                builder.groupKey(ImmutableBitSet.of(7),
                     ImmutableList.of(ImmutableBitSet.of(7),
                         ImmutableBitSet.of())),
                 builder.aggregateCall(SqlStdOperatorTable.COUNT, false,
@@ -595,7 +653,7 @@ public class RelBuilderTest {
                         builder.field("EMPNO"), builder.literal(100)), "C"))
             .build();
     final String expected = ""
-        + "LogicalAggregate(group=[{7}], groups=[[{7}, {}]], indicator=[true], C=[COUNT() FILTER $8])\n"
+        + "LogicalAggregate(group=[{7}], groups=[[{7}, {}]], C=[COUNT() FILTER $8])\n"
         + "  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], $f8=[>($0, 100)])\n"
         + "    LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(str(root), is(expected));
@@ -649,7 +707,7 @@ public class RelBuilderTest {
     try {
       RelNode root =
           builder.scan("EMP")
-              .aggregate(builder.groupKey(ImmutableBitSet.of(17), false, null))
+              .aggregate(builder.groupKey(ImmutableBitSet.of(17), null))
               .build();
       fail("expected error, got " + root);
     } catch (IllegalArgumentException e) {
@@ -663,7 +721,7 @@ public class RelBuilderTest {
       RelNode root =
           builder.scan("EMP")
               .aggregate(
-                  builder.groupKey(ImmutableBitSet.of(7), true,
+                  builder.groupKey(ImmutableBitSet.of(7),
                       ImmutableList.of(ImmutableBitSet.of(4),
                           ImmutableBitSet.of())))
               .build();
@@ -679,17 +737,61 @@ public class RelBuilderTest {
     RelNode root =
         builder.scan("EMP")
             .aggregate(
-                builder.groupKey(ImmutableBitSet.of(7, 6), true,
+                builder.groupKey(ImmutableBitSet.of(7, 6),
                     ImmutableList.of(ImmutableBitSet.of(7),
                         ImmutableBitSet.of(6),
                         ImmutableBitSet.of(7))))
             .build();
     final String expected = ""
-        + "LogicalAggregate(group=[{6, 7}], groups=[[{6}, {7}]], indicator=[true])\n"
+        + "LogicalAggregate(group=[{6, 7}], groups=[[{6}, {7}]])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testAggregateGrouping() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("EMP")
+            .aggregate(builder.groupKey(6, 7),
+                builder.aggregateCall(SqlStdOperatorTable.GROUPING, false, null,
+                    "g", builder.field("DEPTNO")))
+            .build();
+    final String expected = ""
+        + "LogicalAggregate(group=[{6, 7}], g=[GROUPING($7)])\n"
+        + "  LogicalTableScan(table=[[scott, EMP]])\n";
+    assertThat(str(root), is(expected));
+  }
+
+  @Test public void testAggregateGroupingWithDistinctFails() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    try {
+      RelNode root =
+          builder.scan("EMP")
+              .aggregate(builder.groupKey(6, 7),
+                  builder.aggregateCall(SqlStdOperatorTable.GROUPING, true, null,
+                      "g", builder.field("DEPTNO")))
+              .build();
+      fail("expected error, got " + root);
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("DISTINCT not allowed"));
+    }
+  }
+
+  @Test public void testAggregateGroupingWithFilterFails() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    try {
+      RelNode root =
+          builder.scan("EMP")
+              .aggregate(builder.groupKey(6, 7),
+                  builder.aggregateCall(SqlStdOperatorTable.GROUPING, false,
+                      builder.literal(true), "g", builder.field("DEPTNO")))
+              .build();
+      fail("expected error, got " + root);
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("FILTER not allowed"));
+    }
+  }
+
   @Test public void testDistinct() {
     // Equivalent SQL:
     //   SELECT DISTINCT deptno

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index cf0c0e8..115da8e 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -1280,7 +1280,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
     // Aggregate
     final LogicalAggregate aggregate =
-        LogicalAggregate.create(join, false, ImmutableBitSet.of(2, 0),
+        LogicalAggregate.create(join, ImmutableBitSet.of(2, 0),
             ImmutableList.<ImmutableBitSet>of(),
             ImmutableList.of(
                 AggregateCall.create(
@@ -2162,7 +2162,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
         + "group by grouping sets ((deptno), (ename, deptno))";
     final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
     expected.put(TableScan.class, 1);
-    expected.put(Project.class, 3);
+    expected.put(Project.class, 2);
     expected.put(Aggregate.class, 1);
     checkNodeTypeCount(sql, expected);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/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 44e510e..95c47fc 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -5243,34 +5243,55 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testGroupId() {
+    final String groupIdOnlyInAggregate =
+        "GROUP_ID operator may only occur in an aggregate query";
+    final String groupIdWrongClause =
+        "GROUP_ID operator may only occur in SELECT, HAVING or ORDER BY clause";
+
     sql("select deptno, group_id() from emp group by deptno").ok();
     sql("select deptno, ^group_id^ as x from emp group by deptno")
         .fails("Column 'GROUP_ID' not found in any table");
     sql("select deptno, ^group_id(deptno)^ from emp group by deptno")
         .fails("Invalid number of arguments to function 'GROUP_ID'\\. "
             + "Was expecting 0 arguments");
+    // Oracle throws "GROUPING function only supported with GROUP BY CUBE or
+    // ROLLUP"
+    sql("select ^group_id()^ from emp")
+        .fails(groupIdOnlyInAggregate);
     sql("select deptno from emp order by ^group_id(deptno)^")
-        .fails("GROUP_ID operator may only occur in an aggregate query");
+        .fails(groupIdOnlyInAggregate);
+    // Oracle throws "GROUPING function only supported with GROUP BY CUBE or
+    // ROLLUP"
+    sql("select 1 from emp order by ^group_id()^")
+        .fails(groupIdOnlyInAggregate);
+    sql("select 1 from emp order by ^grouping(deptno)^")
+        .fails("GROUPING operator may only occur in an aggregate query");
+    // Oracle throws "group function is not allowed here"
     sql("select deptno from emp where ^group_id()^ = 1")
-        .fails("GROUP_ID operator may only occur in an aggregate query");
+        .fails(groupIdOnlyInAggregate);
+    // Oracle throws "group function is not allowed here"
+    sql("select deptno from emp group by ^group_id()^")
+        .fails(groupIdWrongClause);
     sql("select deptno from emp where ^group_id()^ = 1 group by deptno")
-        .fails(
-            "GROUP_ID operator may only occur in SELECT, HAVING or ORDER BY clause");
+        .fails(groupIdWrongClause);
     sql("select deptno from emp group by deptno, ^group_id()^")
-        .fails(
-            "GROUP_ID operator may only occur in SELECT, HAVING or ORDER BY clause");
+        .fails(groupIdWrongClause);
     sql("select deptno from emp\n"
         + "group by grouping sets(deptno, ^group_id()^)")
-        .fails(
-            "GROUP_ID operator may only occur in SELECT, HAVING or ORDER BY clause");
+        .fails(groupIdWrongClause);
     sql("select deptno from emp\n"
         + "group by cube(empno, ^group_id()^)")
-        .fails(
-            "GROUP_ID operator may only occur in SELECT, HAVING or ORDER BY clause");
+        .fails(groupIdWrongClause);
     sql("select deptno from emp\n"
         + "group by rollup(empno, ^group_id()^)")
-        .fails(
-            "GROUP_ID operator may only occur in SELECT, HAVING or ORDER BY clause");
+        .fails(groupIdWrongClause);
+    sql("select grouping(^group_id()^) from emp")
+        .fails(groupIdOnlyInAggregate);
+    // Oracle throws "not a GROUP BY expression"
+    sql("select grouping(^group_id()^) from emp group by deptno")
+        .fails(groupIdWrongClause);
+    sql("select ^grouping(sum(empno))^ from emp group by deptno")
+        .fails("Aggregate expressions cannot be nested");
   }
 
   @Test public void testCubeGrouping() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index 2da30be..f575271 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -600,10 +600,12 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$2])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EXPR$0=[$1], EXPR$1=[$2])
-  LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)], EXPR$1=[MIN($1)])
-    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
-      LogicalProject(DEPTNO=[$7], SAL=[$5])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(DEPTNO=[$0], EXPR$0=[$1], EXPR$1=[CAST($2):INTEGER NOT NULL])
+    LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0) FILTER $2], EXPR$1=[MIN($1) FILTER $2])
+      LogicalProject(DEPTNO=[$0], EXPR$1=[$1], $g_0=[=($2, 0)])
+        LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], $g=[GROUPING($0)])
+          LogicalProject(DEPTNO=[$7], SAL=[$5])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1089,9 +1091,9 @@ LogicalProject(DNAME=[$0], C=[$1])
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(DDEPTNO=[$0], DNAME=[$1], C=[$2])
-  LogicalProject(DDEPTNO=[CASE($2, null, $0)], DNAME=[CASE($3, null, $1)], C=[$4])
-    LogicalFilter(condition=[=(CASE($3, null, $1), 'Charlie')])
-      LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], C=[COUNT()])
+  LogicalProject(DDEPTNO=[$0], DNAME=[$1], C=[$2])
+    LogicalFilter(condition=[=($1, 'Charlie')])
+      LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], C=[COUNT()])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
@@ -1115,21 +1117,19 @@ LogicalProject(DDEPTNO=[$0], DNAME=[$1], C=[$2])
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(DNAME=[$0], DDEPTNO=[$1], C=[$2])
-  LogicalProject(DNAME=[$0], DDEPTNO=[CASE($3, null, $1)], C=[$4])
-    LogicalFilter(condition=[=($0, 'Charlie')])
-      LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], indicator=[true], C=[COUNT()])
-        LogicalProject(DNAME=[$1], DDEPTNO=[$0])
-          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+  LogicalFilter(condition=[=($0, 'Charlie')])
+    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], C=[COUNT()])
+      LogicalProject(DNAME=[$1], DDEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DNAME=[$0], DDEPTNO=[$1], C=[$2])
-  LogicalProject(DNAME=[$0], DDEPTNO=[CASE($3, null, $1)], C=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], indicator=[true], C=[COUNT()])
-      LogicalFilter(condition=[=($0, 'Charlie')])
-        LogicalProject(DNAME=[$1], DDEPTNO=[$0])
-          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], C=[COUNT()])
+    LogicalFilter(condition=[=($0, 'Charlie')])
+      LogicalProject(DNAME=[$1], DDEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -3110,28 +3110,25 @@ group by rollup(deptno,job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], U=[$9])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[2])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[3])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], U=[$9])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[2])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], U=[2])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], U=[3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], U=[2])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], U=[3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3144,28 +3141,25 @@ group by rollup(deptno,job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], U=[$9])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], U=[$9])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], U=[null])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], U=[null])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3179,28 +3173,25 @@ group by rollup(deptno, job)
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3213,28 +3204,25 @@ group by rollup(deptno, job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[COUNT()])
-      LogicalProject(DEPTNO=[$7], JOB=[$2])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[COUNT()])
+  LogicalProject(DEPTNO=[$7], JOB=[$2])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[$SUM0($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
-        LogicalProject(DEPTNO=[$7], JOB=[$2])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
-        LogicalProject(DEPTNO=[$7], JOB=[$2])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[$SUM0($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+      LogicalProject(DEPTNO=[$7], JOB=[$2])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+      LogicalProject(DEPTNO=[$7], JOB=[$2])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3247,28 +3235,25 @@ group by rollup(deptno, job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[COUNT($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[COUNT($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[$SUM0($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[$SUM0($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3281,28 +3266,25 @@ group by rollup(deptno, job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[MAX($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[MAX($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[MAX($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[MAX($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3315,28 +3297,25 @@ LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[MIN($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[MIN($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[MIN($2)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[MIN($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[MIN($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[MIN($2)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[MIN($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[MIN($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3349,26 +3328,23 @@ group by rollup(deptno, job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[AVG($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[AVG($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[AVG($2)])
-    LogicalUnion(all=[true])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[AVG($2)])
+  LogicalUnion(all=[true])
+    LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3381,28 +3357,25 @@ group by rollup(deptno,job)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], EXPR$5=[$7])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], EXPR$5=[$7])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)], EXPR$3=[COUNT()], EXPR$4=[MIN($0)], EXPR$5=[MAX($2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-        LogicalUnion(all=[true])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-          LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)], EXPR$3=[COUNT()], EXPR$4=[MIN($0)], EXPR$5=[MAX($2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], EXPR$5=[$7])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[SUM($2)], EXPR$3=[$SUM0($3)], EXPR$4=[MIN($4)], EXPR$5=[MAX($5)])
-    LogicalUnion(all=[true])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)], EXPR$3=[COUNT()], EXPR$4=[MIN($0)], EXPR$5=[MAX($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)], EXPR$3=[COUNT()], EXPR$4=[MIN($0)], EXPR$5=[MAX($2)])
-        LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[SUM($2)], EXPR$3=[$SUM0($3)], EXPR$4=[MIN($4)], EXPR$5=[MAX($5)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)], EXPR$3=[COUNT()], EXPR$4=[MIN($0)], EXPR$5=[MAX($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)], EXPR$3=[COUNT()], EXPR$4=[MIN($0)], EXPR$5=[MAX($2)])
+      LogicalProject(DEPTNO=[$7], JOB=[$2], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -3470,21 +3443,19 @@ LogicalFilter(condition=[>($1, 5000)])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(ENAME=[CASE($3, null, $0)], SAL=[CASE($4, null, $1)], DEPTNO=[CASE($5, null, $2)])
-  LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], indicator=[true])
-    LogicalFilter(condition=[>($1, 5000)])
-      LogicalProject(ENAME=[$1], SAL=[$5], DEPTNO=[$7])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]])
+  LogicalFilter(condition=[>($1, 5000)])
+    LogicalProject(ENAME=[$1], SAL=[$5], DEPTNO=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(ENAME=[CASE($3, null, $0)], SAL=[CASE($4, null, $1)], DEPTNO=[CASE($5, null, $2)])
-  LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], indicator=[true])
-    LogicalFilter(condition=[>($1, 5000)])
-      LogicalAggregate(group=[{0, 1, 2}])
-        LogicalProject(ENAME=[$1], SAL=[$5], DEPTNO=[$7])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]])
+  LogicalFilter(condition=[>($1, 5000)])
+    LogicalAggregate(group=[{0, 1, 2}])
+      LogicalProject(ENAME=[$1], SAL=[$5], DEPTNO=[$7])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -4911,21 +4882,19 @@ group by rollup(x, y)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(X=[$0], EXPR$1=[$4], Y=[$1])
-  LogicalProject(X=[CASE($2, null, $0)], Y=[CASE($3, null, $1)], i$X=[$2], i$Y=[$3], EXPR$1=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$1=[SUM($2)])
-      LogicalProject(X=[$0], Y=[$1], Z=[$2])
-        LogicalProject(X=[$7], Y=[$0], Z=[$5], ZZ=[*($5, 2)])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalProject(X=[$0], EXPR$1=[$2], Y=[$1])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$1=[SUM($2)])
+    LogicalProject(X=[$0], Y=[$1], Z=[$2])
+      LogicalProject(X=[$7], Y=[$0], Z=[$5], ZZ=[*($5, 2)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(X=[$0], EXPR$1=[$4], Y=[$1])
-  LogicalProject(X=[CASE($2, null, $0)], Y=[CASE($3, null, $1)], i$X=[$2], i$Y=[$3], EXPR$1=[$4])
-    LogicalProject(DEPTNO=[$1], EMPNO=[$0], i$DEPTNO=[$3], i$EMPNO=[$2], EXPR$1=[$4])
-      LogicalAggregate(group=[{0, 7}], groups=[[{0, 7}, {7}, {}]], indicator=[true], EXPR$1=[SUM($5)])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalProject(X=[$0], EXPR$1=[$2], Y=[$1])
+  LogicalProject(DEPTNO=[$1], EMPNO=[$0], EXPR$1=[$2])
+    LogicalAggregate(group=[{0, 7}], groups=[[{0, 7}, {7}, {}]], EXPR$1=[SUM($5)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -5084,10 +5053,10 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], EXPR$1=[$1], EXPR$2=[CAST($2):INTEGER NOT NULL])
-  LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $5], EXPR$2=[MIN($4) FILTER $6])
-    LogicalProject(DEPTNO=[$0], ENAME=[$1], i$DEPTNO=[$2], i$ENAME=[$3], EXPR$2=[$4], $i0_1=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 3)], $i0=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 1)])
-      LogicalProject(DEPTNO=[$1], ENAME=[$0], i$DEPTNO=[$3], i$ENAME=[$2], EXPR$2=[$4])
-        LogicalAggregate(group=[{1, 7}], groups=[[{1, 7}, {7}]], indicator=[true], EXPR$2=[SUM($5)])
+  LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4])
+    LogicalProject(DEPTNO=[$0], ENAME=[$1], EXPR$2=[$2], $g_0=[=($3, 0)], $g_1=[=($3, 1)])
+      LogicalProject(DEPTNO=[$1], ENAME=[$0], EXPR$2=[$2], $g=[$3])
+        LogicalAggregate(group=[{1, 7}], groups=[[{1, 7}, {7}]], EXPR$2=[SUM($5)], $g=[GROUPING($7, $1)])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5098,18 +5067,16 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[$1], EXPR$2=[CAST($2):INTEGER NOT NULL])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[COUNT(DISTINCT $2)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], ENAME=[$1])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[COUNT(DISTINCT $2)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], ENAME=[$1])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4])
-  LogicalAggregate(group=[{0, 1}], indicator=[true], EXPR$2=[COUNT($2)])
-    LogicalAggregate(group=[{0, 1, 2}])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[COUNT($2) FILTER $3])
+  LogicalProject(DEPTNO=[$0], JOB=[$1], ENAME=[$2], $g_0=[=($3, 0)])
+    LogicalAggregate(group=[{0, 1, 2}], $g=[GROUPING($0, $1, $2)])
       LogicalProject(DEPTNO=[$7], JOB=[$2], ENAME=[$1])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -5121,19 +5088,17 @@ LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$4], EXPR$3=[$5])
-  LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], i$DEPTNO=[$2], i$JOB=[$3], EXPR$2=[$4], EXPR$3=[$5])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($3)])
-      LogicalProject(DEPTNO=[$7], JOB=[$2], ENAME=[$1], SAL=[$5])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($3)])
+  LogicalProject(DEPTNO=[$7], JOB=[$2], ENAME=[$1], SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[CASE($2, null, $0)], JOB=[CASE($3, null, $1)], EXPR$2=[$4], EXPR$3=[CAST($5):INTEGER NOT NULL])
-  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[COUNT($2) FILTER $7], EXPR$3=[MIN($6) FILTER $8])
-    LogicalProject(DEPTNO=[$0], JOB=[$1], ENAME=[$2], i$DEPTNO=[$3], i$JOB=[$4], i$ENAME=[$5], EXPR$3=[$6], $i0_1_2=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 7)], $i0_1=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 3)])
-      LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}]], indicator=[true], EXPR$3=[SUM($3)])
+LogicalProject(DEPTNO=[$0], JOB=[$1], EXPR$2=[$2], EXPR$3=[CAST($3):INTEGER NOT NULL])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[COUNT($2) FILTER $4], EXPR$3=[MIN($3) FILTER $5])
+    LogicalProject(DEPTNO=[$0], JOB=[$1], ENAME=[$2], EXPR$3=[$3], $g_0=[=($4, 0)], $g_1=[=($4, 1)])
+      LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}]], EXPR$3=[SUM($3)], $g=[GROUPING($0, $1, $2)])
         LogicalProject(DEPTNO=[$7], JOB=[$2], ENAME=[$1], SAL=[$5])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -5597,19 +5562,21 @@ group by e.deptno, d.deptno]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalAggregate(group=[{7, 9}])
-  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
-    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(DEPTNO=[$0], DEPTNO0=[$1])
+  LogicalAggregate(group=[{7, 9}])
+    LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], DEPTNO0=[$1])
-  LogicalJoin(condition=[=($0, $1)], joinType=[inner])
-    LogicalAggregate(group=[{7}])
-      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+  LogicalProject(DEPTNO=[$0], DEPTNO0=[$1])
+    LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      LogicalAggregate(group=[{7}])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -5819,10 +5786,10 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[COUNT(DISTINC
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $6], EXPR$2=[COUNT($2) FILTER $7])
-  LogicalProject(DEPTNO=[$0], ENAME=[$1], JOB=[$2], i$DEPTNO=[$3], i$ENAME=[$4], i$JOB=[$5], $i0_1=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 3)], $i0_2=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 5)], $i0=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 1)])
-    LogicalProject(DEPTNO=[$2], ENAME=[$0], JOB=[$1], i$DEPTNO=[$5], i$ENAME=[$3], i$JOB=[$4])
-      LogicalAggregate(group=[{1, 2, 7}], groups=[[{1, 7}, {2, 7}, {7}]], indicator=[true])
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[COUNT($2) FILTER $4])
+  LogicalProject(DEPTNO=[$0], ENAME=[$1], JOB=[$2], $g_1=[=($3, 1)], $g_2=[=($3, 2)])
+    LogicalProject(DEPTNO=[$2], ENAME=[$0], JOB=[$1], $g=[$3])
+      LogicalAggregate(group=[{1, 2, 7}], groups=[[{1, 7}, {2, 7}]], $g=[GROUPING($7, $1, $2)])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5841,9 +5808,9 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[COUNT(DISTINCT
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $4], EXPR$1=[COUNT($1) FILTER $5])
-  LogicalProject(ENAME=[$0], JOB=[$1], i$ENAME=[$2], i$JOB=[$3], $i0=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 1)], $i1=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 2)], $=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 0)])
-    LogicalAggregate(group=[{1, 2}], groups=[[{1}, {2}, {}]], indicator=[true])
+LogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $2], EXPR$1=[COUNT($1) FILTER $3])
+  LogicalProject(ENAME=[$0], JOB=[$1], $g_1=[=($2, 1)], $g_2=[=($2, 2)])
+    LogicalAggregate(group=[{1, 2}], groups=[[{1}, {2}]], $g=[GROUPING($1, $2)])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5863,9 +5830,9 @@ LogicalAggregate(group=[{0}], CDDJ=[COUNT(DISTINCT $0, $1)], S=[SUM($2)])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], CDDJ=[$1], S=[CAST($2):INTEGER NOT NULL])
-  LogicalAggregate(group=[{0}], CDDJ=[COUNT($0, $1) FILTER $5], S=[MIN($4) FILTER $6])
-    LogicalProject(DEPTNO=[$0], JOB=[$1], i$DEPTNO=[$2], i$JOB=[$3], S=[$4], $i0_1=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 3)], $i0=[=(+(CASE($2, 0, 1), CASE($3, 0, 2)), 1)])
-      LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], indicator=[true], S=[SUM($2)])
+  LogicalAggregate(group=[{0}], CDDJ=[COUNT($0, $1) FILTER $3], S=[MIN($2) FILTER $4])
+    LogicalProject(DEPTNO=[$0], JOB=[$1], S=[$2], $g_0=[=($3, 0)], $g_1=[=($3, 1)])
+      LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], S=[SUM($2)], $g=[GROUPING($0, $1)])
         LogicalProject(DEPTNO=[$7], JOB=[$2], SAL=[$5])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -5922,9 +5889,9 @@ LogicalAggregate(group=[{0}], CDE=[COUNT(DISTINCT $1)], CDJE=[COUNT(DISTINCT $2,
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], CDE=[$1], CDJE=[$2], CDDJ=[$3], S=[CAST($4):INTEGER NOT NULL])
-  LogicalAggregate(group=[{0}], CDE=[COUNT($1) FILTER $8], CDJE=[COUNT($2, $1) FILTER $7], CDDJ=[COUNT($0, $2) FILTER $9], S=[MIN($6) FILTER $10])
-    LogicalProject(DEPTNO=[$2], ENAME=[$0], JOB=[$1], i$DEPTNO=[$5], i$ENAME=[$3], i$JOB=[$4], S=[$6], $i0_1_2=[=(+(+(CASE($5, 0, 1), CASE($3, 0, 2)), CASE($4, 0, 4)), 7)], $i0_1=[=(+(+(CASE($5, 0, 1), CASE($3, 0, 2)), CASE($4, 0, 4)), 3)], $i0_2=[=(+(+(CASE($5, 0, 1), CASE($3, 0, 2)), CASE($4, 0, 4)), 5)], $i0=[=(+(+(CASE($5, 0, 1), CASE($3, 0, 2)), CASE($4, 0, 4)), 1)])
-      LogicalAggregate(group=[{1, 2, 7}], groups=[[{1, 2, 7}, {1, 7}, {2, 7}, {7}]], indicator=[true], S=[SUM($5)])
+  LogicalAggregate(group=[{0}], CDE=[COUNT($1) FILTER $5], CDJE=[COUNT($2, $1) FILTER $4], CDDJ=[COUNT($0, $2) FILTER $6], S=[MIN($3) FILTER $7])
+    LogicalProject(DEPTNO=[$2], ENAME=[$0], JOB=[$1], S=[$3], $g_0=[=($4, 0)], $g_1=[=($4, 1)], $g_2=[=($4, 2)], $g_3=[=($4, 3)])
+      LogicalAggregate(group=[{1, 2, 7}], groups=[[{1, 2, 7}, {1, 7}, {2, 7}, {7}]], S=[SUM($5)], $g=[GROUPING($7, $1, $2)])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -7488,9 +7455,9 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[SUM(DISTINCT $2
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(NAME=[$0], EXPR$1=[CAST($1):BIGINT NOT NULL], EXPR$2=[CAST($2):INTEGER NOT NULL])
-  LogicalAggregate(group=[{0}], EXPR$1=[SUM($1) FILTER $6], EXPR$2=[SUM($2) FILTER $7])
-    LogicalProject(NAME=[$0], CN=[$1], SM=[$2], i$NAME=[$3], i$CN=[$4], i$SM=[$5], $i0_1=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 3)], $i0_2=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 5)], $i0=[=(+(+(CASE($3, 0, 1), CASE($4, 0, 2)), CASE($5, 0, 4)), 1)])
-      LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}, {0}]], indicator=[true])
+  LogicalAggregate(group=[{0}], EXPR$1=[SUM($1) FILTER $3], EXPR$2=[SUM($2) FILTER $4])
+    LogicalProject(NAME=[$0], CN=[$1], SM=[$2], $g_1=[=($3, 1)], $g_2=[=($3, 2)])
+      LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], $g=[GROUPING($0, $1, $2)])
         LogicalAggregate(group=[{0}], CN=[COUNT()], SM=[SUM($1)])
           LogicalProject(NAME=[$1], DEPTNO=[$0])
             LogicalTableScan(table=[[CATALOG, SALES, DEPT]])

http://git-wip-us.apache.org/repos/asf/calcite/blob/1e7ae1c3/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 2e8f904..41fce01 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2174,11 +2174,9 @@ order by 2]]>
         <Resource name="plan">
             <![CDATA[
 LogicalSort(sort0=[$1], dir0=[ASC])
-  LogicalProject(DEPTNO=[$0], ENAME=[$1], EXPR$2=[$4])
-    LogicalProject(DEPTNO=[$0], ENAME=[CASE($3, null, $1)], i$DEPTNO=[$2], i$ENAME=[$3], EXPR$2=[$4])
-      LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], indicator=[true], EXPR$2=[SUM($2)])
-        LogicalProject(DEPTNO=[$7], ENAME=[$1], SAL=[$5])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], EXPR$2=[SUM($2)])
+    LogicalProject(DEPTNO=[$7], ENAME=[$1], SAL=[$5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -2193,11 +2191,10 @@ group by sal,
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[$6])
-  LogicalProject(SAL=[$0], DEPTNO=[CASE($4, null, $1)], ENAME=[CASE($5, null, $2)], i$SAL=[$3], i$DEPTNO=[$4], i$ENAME=[$5], EXPR$0=[$6])
-    LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}]], indicator=[true], EXPR$0=[SUM($0)])
-      LogicalProject(SAL=[$5], DEPTNO=[$7], ENAME=[$1])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$3])
+  LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}]], EXPR$0=[SUM($0)])
+    LogicalProject(SAL=[$5], DEPTNO=[$7], ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -2210,9 +2207,8 @@ group by grouping sets (a, b), grouping sets (c, d)]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$0=[CASE($4, null, $0)], EXPR$1=[CASE($5, null, $1)], EXPR$2=[CASE($6, null, $2)], EXPR$3=[CASE($7, null, $3)], i$EXPR$0=[$4], i$EXPR$1=[$5], i$EXPR$2=[$6], i$EXPR$3=[$7])
-    LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2}, {0, 3}, {1, 2}, {1, 3}]], indicator=[true])
-      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2}, {0, 3}, {1, 2}, {1, 3}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2225,9 +2221,8 @@ group by grouping sets (a, (a, b)), grouping sets (c), d]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$0=[$0], EXPR$1=[CASE($5, null, $1)], EXPR$2=[$2], EXPR$3=[$3], i$EXPR$0=[$4], i$EXPR$1=[$5], i$EXPR$2=[$6], i$EXPR$3=[$7])
-    LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}]], indicator=[true])
-      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2240,9 +2235,8 @@ group by rollup(a, b), rollup(c, d)]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$0=[CASE($4, null, $0)], EXPR$1=[CASE($5, null, $1)], EXPR$2=[CASE($6, null, $2)], EXPR$3=[CASE($7, null, $3)], i$EXPR$0=[$4], i$EXPR$1=[$5], i$EXPR$2=[$6], i$EXPR$3=[$7])
-    LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1}, {0, 2, 3}, {0, 2}, {0}, {2, 3}, {2}, {}]], indicator=[true])
-      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1}, {0, 2, 3}, {0, 2}, {0}, {2, 3}, {2}, {}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2255,10 +2249,9 @@ group by cube(a, b)]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$0=[CASE($2, null, $0)], EXPR$1=[CASE($3, null, $1)], i$EXPR$0=[$2], i$EXPR$1=[$3])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], indicator=[true])
-      LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
-        LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]])
+    LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2271,10 +2264,9 @@ group by rollup(b, (a, d))]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$1=[CASE($3, null, $0)], EXPR$0=[CASE($4, null, $1)], EXPR$3=[CASE($5, null, $2)], i$EXPR$1=[$3], i$EXPR$0=[$4], i$EXPR$3=[$5])
-    LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0}, {}]], indicator=[true])
-      LogicalProject(EXPR$1=[$1], EXPR$0=[$0], EXPR$3=[$3])
-        LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0}, {}]])
+    LogicalProject(EXPR$1=[$1], EXPR$0=[$0], EXPR$3=[$3])
+      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2287,9 +2279,8 @@ group by rollup(a, b), rollup(c, d)]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$0=[CASE($4, null, $0)], EXPR$1=[CASE($5, null, $1)], EXPR$2=[CASE($6, null, $2)], EXPR$3=[CASE($7, null, $3)], i$EXPR$0=[$4], i$EXPR$1=[$5], i$EXPR$2=[$6], i$EXPR$3=[$7])
-    LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1}, {0, 2, 3}, {0, 2}, {0}, {2, 3}, {2}, {}]], indicator=[true])
-      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1}, {0, 2, 3}, {0, 2}, {0}, {2, 3}, {2}, {}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2301,11 +2292,9 @@ group by rollup(a, b)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(A=[$0], B=[$1], C=[$4])
-  LogicalProject(A=[CASE($2, null, $0)], B=[CASE($3, null, $1)], i$A=[$2], i$B=[$3], C=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], C=[COUNT()])
-      LogicalProject(A=[null], B=[2])
-        LogicalValues(tuples=[[{ 0 }]])
+LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], C=[COUNT()])
+  LogicalProject(A=[null], B=[2])
+    LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2333,9 +2322,8 @@ group by grouping sets ((a, b), c), grouping sets ((x, y), ())]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[1])
-  LogicalProject(EXPR$0=[CASE($5, null, $0)], EXPR$1=[CASE($6, null, $1)], EXPR$2=[CASE($7, null, $2)], EXPR$3=[CASE($8, null, $3)], EXPR$4=[CASE($9, null, $4)], i$EXPR$0=[$5], i$EXPR$1=[$6], i$EXPR$2=[$7], i$EXPR$3=[$8], i$EXPR$4=[$9])
-    LogicalAggregate(group=[{0, 1, 2, 3, 4}], groups=[[{0, 1, 3, 4}, {0, 1}, {2, 3, 4}, {2}]], indicator=[true])
-      LogicalValues(tuples=[[{ 0, 1, 2, 3, 4 }]])
+  LogicalAggregate(group=[{0, 1, 2, 3, 4}], groups=[[{0, 1, 3, 4}, {0, 1}, {2, 3, 4}, {2}]])
+    LogicalValues(tuples=[[{ 0, 1, 2, 3, 4 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -2350,8 +2338,8 @@ order by 2]]>
         <Resource name="plan">
             <![CDATA[
 LogicalSort(sort0=[$1], dir0=[ASC])
-  LogicalProject(DEPTNO=[$1], EXPR$1=[1], EXPR$2=[$2], EXPR$3=[1])
-    LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+  LogicalProject(DEPTNO=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4])
+    LogicalAggregate(group=[{0, 1}], EXPR$1=[GROUPING($1)], EXPR$2=[COUNT()], EXPR$3=[GROUPING($0)])
       LogicalProject(EMPNO=[$0], DEPTNO=[$7])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -2366,11 +2354,10 @@ group by rollup(empno, deptno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(DEPTNO=[$1], EXPR$1=[CASE($3, 1, 0)], EXPR$2=[$4], EXPR$3=[CASE($2, 1, 0)])
-  LogicalProject(EMPNO=[CASE($2, null, $0)], DEPTNO=[CASE($3, null, $1)], i$EMPNO=[$2], i$DEPTNO=[$3], EXPR$2=[$4])
-    LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], indicator=[true], EXPR$2=[COUNT()])
-      LogicalProject(EMPNO=[$0], DEPTNO=[$7])
-        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$1=[GROUPING($1)], EXPR$2=[COUNT()], EXPR$3=[GROUPING($0)])
+    LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -2726,9 +2713,10 @@ GROUP BY empno, EXPR$2]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
-  LogicalProject(EMPNO=[$0], EXPR$2=[$7])
-    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], EXPR$2=[$1], EXPR$20=[$2])
+  LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+    LogicalProject(EMPNO=[$0], EXPR$2=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -2848,9 +2836,10 @@ group by hop(rowtime, interval '1' hour, interval '3' hour)]]>
         <Resource name="plan">
             <![CDATA[
 LogicalDelta
-  LogicalAggregate(group=[{0}], C=[COUNT()])
-    LogicalProject($f0=[HOP($0, 3600000, 10800000)])
-      LogicalTableScan(table=[[CATALOG, SALES, ORDERS]])
+  LogicalProject(ROWTIME=[$0], C=[$1])
+    LogicalAggregate(group=[{0}], C=[COUNT()])
+      LogicalProject($f0=[HOP($0, 3600000, 10800000)])
+        LogicalTableScan(table=[[CATALOG, SALES, ORDERS]])
 ]]>
         </Resource>
     </TestCase>


[04/16] calcite git commit: [CALCITE-1967] Elasticsearch 5 adapter (Christian Beikov)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
deleted file mode 100644
index 660e268..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Implementation of {@link org.apache.calcite.rel.core.Project}
- * relational expression in Elasticsearch.
- */
-public class ElasticsearchProject extends Project implements ElasticsearchRel {
-  public ElasticsearchProject(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
-      List<? extends RexNode> projects, RelDataType rowType) {
-    super(cluster, traitSet, input, projects, rowType);
-    assert getConvention() == ElasticsearchRel.CONVENTION;
-    assert getConvention() == input.getConvention();
-  }
-
-  @Override public Project copy(RelTraitSet relTraitSet, RelNode input, List<RexNode> projects,
-      RelDataType relDataType) {
-    return new ElasticsearchProject(getCluster(), traitSet, input, projects, relDataType);
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return super.computeSelfCost(planner, mq).multiplyBy(0.1);
-  }
-
-  @Override public void implement(Implementor implementor) {
-    implementor.visitChild(0, getInput());
-
-    final ElasticsearchRules.RexToElasticsearchTranslator translator =
-        new ElasticsearchRules.RexToElasticsearchTranslator(
-            (JavaTypeFactory) getCluster().getTypeFactory(),
-            ElasticsearchRules.elasticsearchFieldNames(getInput().getRowType()));
-
-    final List<String> findItems = new ArrayList<>();
-    final List<String> scriptFieldItems = new ArrayList<>();
-    for (Pair<RexNode, String> pair: getNamedProjects()) {
-      final String name = pair.right;
-      final String expr = pair.left.accept(translator);
-
-      if (expr.equals("\"" + name + "\"")) {
-        findItems.add(ElasticsearchRules.quote(name));
-      } else if (expr.matches("\"literal\":.+")) {
-        scriptFieldItems.add(ElasticsearchRules.quote(name) + ":{\"script\": "
-            + expr.split(":")[1] + "}");
-      } else {
-        scriptFieldItems.add(ElasticsearchRules.quote(name) + ":{\"script\":\"_source."
-            + expr.replaceAll("\"", "") + "\"}");
-      }
-    }
-    final String findString = Util.toString(findItems, "", ", ", "");
-    final String scriptFieldString = "\"script_fields\": {"
-        + Util.toString(scriptFieldItems, "", ", ", "") + "}";
-    final String fieldString = "\"fields\" : [" + findString + "]"
-        + ", " + scriptFieldString;
-
-    for (String opfield : implementor.list) {
-      if (opfield.startsWith("\"fields\"")) {
-        implementor.list.remove(opfield);
-      }
-    }
-    implementor.add(fieldString);
-  }
-}
-
-// End ElasticsearchProject.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java
deleted file mode 100644
index e24cb0d..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRel.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.rel.RelNode;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Relational expression that uses Elasticsearch calling convention.
- */
-public interface ElasticsearchRel extends RelNode {
-  void implement(Implementor implementor);
-
-  /**
-   * Calling convention for relational operations that occur in Elasticsearch.
-   */
-  Convention CONVENTION = new Convention.Impl("ELASTICSEARCH", ElasticsearchRel.class);
-
-  /**
-   * Callback for the implementation process that converts a tree of
-   * {@link ElasticsearchRel} nodes into an Elasticsearch query.
-   */
-  class Implementor {
-    final List<String> list = new ArrayList<>();
-
-    RelOptTable table;
-    ElasticsearchTable elasticsearchTable;
-
-    public void add(String findOp) {
-      list.add(findOp);
-    }
-
-    public void visitChild(int ordinal, RelNode input) {
-      assert ordinal == 0;
-      ((ElasticsearchRel) input).implement(this);
-    }
-  }
-}
-
-// End ElasticsearchRel.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java
deleted file mode 100644
index ba2978d..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchRules.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.adapter.enumerable.RexImpTable;
-import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlValidatorUtil;
-
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Rules and relational operators for
- * {@link ElasticsearchRel#CONVENTION ELASTICSEARCH}
- * calling convention.
- */
-class ElasticsearchRules {
-  static final RelOptRule[] RULES = {
-      ElasticsearchSortRule.INSTANCE,
-      ElasticsearchFilterRule.INSTANCE,
-      ElasticsearchProjectRule.INSTANCE
-  };
-
-  private ElasticsearchRules() {}
-
-  /**
-   * Returns 'string' if it is a call to item['string'], null otherwise.
-   */
-  static String isItem(RexCall call) {
-    if (call.getOperator() != SqlStdOperatorTable.ITEM) {
-      return null;
-    }
-    final RexNode op0 = call.getOperands().get(0);
-    final RexNode op1 = call.getOperands().get(1);
-
-    if (op0 instanceof RexInputRef
-        && ((RexInputRef) op0).getIndex() == 0
-        && op1 instanceof RexLiteral
-        && ((RexLiteral) op1).getValue2() instanceof String) {
-      return (String) ((RexLiteral) op1).getValue2();
-    }
-    return null;
-  }
-
-  static List<String> elasticsearchFieldNames(final RelDataType rowType) {
-    return SqlValidatorUtil.uniquify(
-        new AbstractList<String>() {
-          @Override public String get(int index) {
-            final String name = rowType.getFieldList().get(index).getName();
-            return name.startsWith("$") ? "_" + name.substring(2) : name;
-          }
-
-          @Override public int size() {
-            return rowType.getFieldCount();
-          }
-        },
-        SqlValidatorUtil.EXPR_SUGGESTER, true);
-  }
-
-  static String quote(String s) {
-    return "\"" + s + "\"";
-  }
-
-  /**
-   * Translator from {@link RexNode} to strings in Elasticsearch's expression
-   * language.
-   */
-  static class RexToElasticsearchTranslator extends RexVisitorImpl<String> {
-    private final JavaTypeFactory typeFactory;
-    private final List<String> inFields;
-
-    RexToElasticsearchTranslator(JavaTypeFactory typeFactory, List<String> inFields) {
-      super(true);
-      this.typeFactory = typeFactory;
-      this.inFields = inFields;
-    }
-
-    @Override public String visitLiteral(RexLiteral literal) {
-      if (literal.getValue() == null) {
-        return "null";
-      }
-      return "\"literal\":\""
-        + RexToLixTranslator.translateLiteral(literal, literal.getType(),
-          typeFactory, RexImpTable.NullAs.NOT_POSSIBLE)
-        + "\"";
-    }
-
-    @Override public String visitInputRef(RexInputRef inputRef) {
-      return quote(inFields.get(inputRef.getIndex()));
-    }
-
-    @Override public String visitCall(RexCall call) {
-      final String name = isItem(call);
-      if (name != null) {
-        return "\"" + name + "\"";
-      }
-
-      final List<String> strings = visitList(call.operands);
-      if (call.getKind() == SqlKind.CAST) {
-        return strings.get(0).startsWith("$") ? strings.get(0).substring(1) : strings.get(0);
-      }
-      if (call.getOperator() == SqlStdOperatorTable.ITEM) {
-        final RexNode op1 = call.getOperands().get(1);
-        if (op1 instanceof RexLiteral && op1.getType().getSqlTypeName() == SqlTypeName.INTEGER) {
-          return stripQuotes(strings.get(0)) + "[" + ((RexLiteral) op1).getValue2() + "]";
-        }
-      }
-      throw new IllegalArgumentException("Translation of " + call.toString()
-        + "is not supported by ElasticsearchProject");
-    }
-
-    private String stripQuotes(String s) {
-      return s.startsWith("'") && s.endsWith("'") ? s.substring(1, s.length() - 1) : s;
-    }
-
-    List<String> visitList(List<RexNode> list) {
-      final List<String> strings = new ArrayList<>();
-      for (RexNode node: list) {
-        strings.add(node.accept(this));
-      }
-      return strings;
-    }
-  }
-
-  /**
-   * Base class for planner rules that convert a relational expression to
-   * Elasticsearch calling convention.
-   */
-  abstract static class ElasticsearchConverterRule extends ConverterRule {
-    final Convention out;
-
-    ElasticsearchConverterRule(Class<? extends RelNode> clazz, RelTrait in, Convention out,
-        String description) {
-      super(clazz, in, out, description);
-      this.out = out;
-    }
-  }
-
-  /**
-   * Rule to convert a {@link org.apache.calcite.rel.core.Sort} to an
-   * {@link ElasticsearchSort}.
-   */
-  private static class ElasticsearchSortRule extends ElasticsearchConverterRule {
-    private static final ElasticsearchSortRule INSTANCE = new ElasticsearchSortRule();
-
-    private ElasticsearchSortRule() {
-      super(Sort.class, Convention.NONE, ElasticsearchRel.CONVENTION, "ElasticsearchSortRule");
-    }
-
-    @Override public RelNode convert(RelNode relNode) {
-      final Sort sort = (Sort) relNode;
-      final RelTraitSet traitSet = sort.getTraitSet().replace(out).replace(sort.getCollation());
-      return new ElasticsearchSort(relNode.getCluster(), traitSet,
-        convert(sort.getInput(), traitSet.replace(RelCollations.EMPTY)), sort.getCollation(),
-        sort.offset, sort.fetch);
-    }
-  }
-
-  /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to an
-   * {@link ElasticsearchFilter}.
-   */
-  private static class ElasticsearchFilterRule extends ElasticsearchConverterRule {
-    private static final ElasticsearchFilterRule INSTANCE = new ElasticsearchFilterRule();
-
-    private ElasticsearchFilterRule() {
-      super(LogicalFilter.class, Convention.NONE, ElasticsearchRel.CONVENTION,
-        "ElasticsearchFilterRule");
-    }
-
-    @Override public RelNode convert(RelNode relNode) {
-      final LogicalFilter filter = (LogicalFilter) relNode;
-      final RelTraitSet traitSet = filter.getTraitSet().replace(out);
-      return new ElasticsearchFilter(relNode.getCluster(), traitSet,
-        convert(filter.getInput(), out),
-        filter.getCondition());
-    }
-  }
-
-  /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject}
-   * to an {@link ElasticsearchProject}.
-   */
-  private static class ElasticsearchProjectRule extends ElasticsearchConverterRule {
-    private static final ElasticsearchProjectRule INSTANCE = new ElasticsearchProjectRule();
-
-    private ElasticsearchProjectRule() {
-      super(LogicalProject.class, Convention.NONE, ElasticsearchRel.CONVENTION,
-        "ElasticsearchProjectRule");
-    }
-
-    @Override public RelNode convert(RelNode relNode) {
-      final LogicalProject project = (LogicalProject) relNode;
-      final RelTraitSet traitSet = project.getTraitSet().replace(out);
-      return new ElasticsearchProject(project.getCluster(), traitSet,
-        convert(project.getInput(), out), project.getProjects(), project.getRowType());
-    }
-  }
-}
-
-// End ElasticsearchRules.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java
deleted file mode 100644
index 960b387..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchema.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AbstractSchema;
-
-import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
-import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
-import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.cluster.metadata.MappingMetaData;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.transport.TransportAddress;
-
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Schema mapped onto an index of ELASTICSEARCH types.
- *
- * <p>Each table in the schema is an ELASTICSEARCH type in that index.
- */
-public class ElasticsearchSchema extends AbstractSchema {
-  final String index;
-
-  private transient Client client;
-
-  /**
-   * Creates an Elasticsearch schema.
-   *
-   * @param coordinates Map of Elasticsearch node locations (host, port)
-   * @param userConfig Map of user-specified configurations
-   * @param indexName Elasticsearch database name, e.g. "usa".
-   */
-  ElasticsearchSchema(Map<String, Integer> coordinates,
-      Map<String, String> userConfig, String indexName) {
-    super();
-
-    final List<InetSocketAddress> transportAddresses = new ArrayList<>();
-    for (Map.Entry<String, Integer> coordinate: coordinates.entrySet()) {
-      transportAddresses.add(new InetSocketAddress(coordinate.getKey(), coordinate.getValue()));
-    }
-
-    open(transportAddresses, userConfig);
-
-    if (client != null) {
-      final String[] indices = client.admin().indices()
-          .getIndex(new GetIndexRequest().indices(indexName))
-          .actionGet().getIndices();
-      if (indices.length == 1) {
-        index = indices[0];
-      } else {
-        index = null;
-      }
-    } else {
-      index = null;
-    }
-  }
-
-  @Override protected Map<String, Table> getTableMap() {
-    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
-
-    try {
-      GetMappingsResponse response = client.admin().indices()
-          .getMappings(new GetMappingsRequest().indices(index))
-          .get();
-      ImmutableOpenMap<String, MappingMetaData> mapping = response.getMappings().get(index);
-      for (ObjectObjectCursor<String, MappingMetaData> c: mapping) {
-        builder.put(c.key, new ElasticsearchTable(client, index, c.key));
-      }
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return builder.build();
-  }
-
-  private void open(List<InetSocketAddress> transportAddresses, Map<String, String> userConfig) {
-    final List<TransportAddress> transportNodes = new ArrayList<>(transportAddresses.size());
-    for (InetSocketAddress address : transportAddresses) {
-      transportNodes.add(new InetSocketTransportAddress(address));
-    }
-
-    Settings settings = Settings.settingsBuilder().put(userConfig).build();
-
-    final TransportClient transportClient = TransportClient.builder().settings(settings).build();
-    for (TransportAddress transport : transportNodes) {
-      transportClient.addTransportAddress(transport);
-    }
-
-    final List<DiscoveryNode> nodes = ImmutableList.copyOf(transportClient.connectedNodes());
-    if (nodes.isEmpty()) {
-      throw new RuntimeException("Cannot connect to any elasticsearch nodes");
-    }
-
-    client = transportClient;
-  }
-}
-
-// End ElasticsearchSchema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchemaFactory.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchemaFactory.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchemaFactory.java
deleted file mode 100644
index 41ffc10..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSchemaFactory.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaFactory;
-import org.apache.calcite.schema.SchemaPlus;
-
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Factory that creates a {@link ElasticsearchSchema}.
- *
- * <p>Allows a custom schema to be included in a model.json file.
- */
-@SuppressWarnings("UnusedDeclaration")
-public class ElasticsearchSchemaFactory implements SchemaFactory {
-
-  public ElasticsearchSchemaFactory() {
-  }
-
-  @Override public Schema create(SchemaPlus parentSchema, String name,
-      Map<String, Object> operand) {
-    final Map map = (Map) operand;
-
-    final ObjectMapper mapper = new ObjectMapper();
-    mapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true);
-
-    try {
-      final Map<String, Integer> coordinates =
-          mapper.readValue((String) map.get("coordinates"),
-              new TypeReference<Map<String, Integer>>() { });
-      final Map<String, String> userConfig =
-          mapper.readValue((String) map.get("userConfig"),
-              new TypeReference<Map<String, String>>() { });
-      final String index = (String) map.get("index");
-      return new ElasticsearchSchema(coordinates, userConfig, index);
-    } catch (IOException e) {
-      throw new RuntimeException("Cannot parse values from json", e);
-    }
-  }
-}
-
-// End ElasticsearchSchemaFactory.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java
deleted file mode 100644
index 5f5dfe8..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchSort.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.Util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Implementation of {@link org.apache.calcite.rel.core.Sort}
- * relational expression in Elasticsearch.
- */
-public class ElasticsearchSort extends Sort implements ElasticsearchRel {
-  public ElasticsearchSort(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
-      RelCollation collation, RexNode offset, RexNode fetch) {
-    super(cluster, traitSet, child, collation, offset, fetch);
-    assert getConvention() == ElasticsearchRel.CONVENTION;
-    assert getConvention() == child.getConvention();
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return super.computeSelfCost(planner, mq).multiplyBy(0.05);
-  }
-
-  @Override public Sort copy(RelTraitSet traitSet, RelNode relNode, RelCollation relCollation,
-      RexNode offset, RexNode fetch) {
-    return new ElasticsearchSort(getCluster(), traitSet, relNode, collation, offset, fetch);
-  }
-
-  @Override public void implement(Implementor implementor) {
-    implementor.visitChild(0, getInput());
-    if (!collation.getFieldCollations().isEmpty()) {
-      final List<String> keys = new ArrayList<>();
-      final List<RelDataTypeField> fields = getRowType().getFieldList();
-
-      for (RelFieldCollation fieldCollation: collation.getFieldCollations()) {
-        final String name = fields.get(fieldCollation.getFieldIndex()).getName();
-        keys.add(ElasticsearchRules.quote(name) + ": " + direction(fieldCollation));
-      }
-
-      implementor.add("\"sort\": [ " + Util.toString(keys, "{", "}, {", "}") + "]");
-    }
-
-    if (offset != null) {
-      implementor.add("\"from\": " + ((RexLiteral) offset).getValue());
-    }
-
-    if (fetch != null) {
-      implementor.add("\"size\": " + ((RexLiteral) fetch).getValue());
-    }
-  }
-
-  private String direction(RelFieldCollation fieldCollation) {
-    switch (fieldCollation.getDirection()) {
-    case DESCENDING:
-    case STRICTLY_DESCENDING:
-      return "\"desc\"";
-    case ASCENDING:
-    case STRICTLY_ASCENDING:
-    default:
-      return "\"asc\"";
-    }
-  }
-}
-
-// End ElasticsearchSort.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTable.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTable.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTable.java
deleted file mode 100644
index 5e9043c..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTable.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.adapter.java.AbstractQueryableTable;
-import org.apache.calcite.linq4j.AbstractEnumerable;
-import org.apache.calcite.linq4j.Enumerable;
-import org.apache.calcite.linq4j.Enumerator;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.linq4j.Queryable;
-import org.apache.calcite.linq4j.function.Function1;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.impl.AbstractTableQueryable;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-import org.apache.calcite.util.Util;
-
-import org.elasticsearch.client.Client;
-import org.elasticsearch.search.SearchHit;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Table based on an Elasticsearch type.
- */
-public class ElasticsearchTable extends AbstractQueryableTable implements TranslatableTable {
-  private final Client client;
-  private final String indexName;
-  private final String typeName;
-
-  /**
-   * Creates an ElasticsearchTable.
-   */
-  public ElasticsearchTable(Client client, String indexName,
-      String typeName) {
-    super(Object[].class);
-    this.client = client;
-    this.indexName = indexName;
-    this.typeName = typeName;
-  }
-
-  @Override public String toString() {
-    return "ElasticsearchTable{" + typeName + "}";
-  }
-
-  public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
-    final RelDataType mapType = relDataTypeFactory.createMapType(
-        relDataTypeFactory.createSqlType(SqlTypeName.VARCHAR),
-        relDataTypeFactory.createTypeWithNullability(
-            relDataTypeFactory.createSqlType(SqlTypeName.ANY),
-            true));
-    return relDataTypeFactory.builder().add("_MAP", mapType).build();
-  }
-
-  public <T> Queryable<T> asQueryable(QueryProvider queryProvider, SchemaPlus schema,
-      String tableName) {
-    return new ElasticsearchQueryable<>(queryProvider, schema, this, tableName);
-  }
-
-  public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
-    final RelOptCluster cluster = context.getCluster();
-    return new ElasticsearchTableScan(cluster, cluster.traitSetOf(ElasticsearchRel.CONVENTION),
-        relOptTable, this, null);
-  }
-
-  /** Executes a "find" operation on the underlying type.
-   *
-   * <p>For example,
-   * <code>client.prepareSearch(index).setTypes(type)
-   * .setSource("{\"fields\" : [\"state\"]}")</code></p>
-   *
-   * @param index Elasticsearch index
-   * @param ops List of operations represented as Json strings.
-   * @param fields List of fields to project; or null to return map
-   * @return Enumerator of results
-   */
-  private Enumerable<Object> find(String index, List<String> ops,
-      List<Map.Entry<String, Class>> fields) {
-    final String dbName = index;
-
-    final String queryString = "{" + Util.toString(ops, "", ", ", "") + "}";
-
-    final Function1<SearchHit, Object> getter = ElasticsearchEnumerator.getter(fields);
-
-    return new AbstractEnumerable<Object>() {
-      public Enumerator<Object> enumerator() {
-        final Iterator<SearchHit> cursor = client.prepareSearch(dbName).setTypes(typeName)
-            .setSource(queryString).execute().actionGet().getHits().iterator();
-        return new ElasticsearchEnumerator(cursor, getter);
-      }
-    };
-  }
-
-  /**
-   * Implementation of {@link org.apache.calcite.linq4j.Queryable} based on
-   * a {@link org.apache.calcite.adapter.elasticsearch.ElasticsearchTable}.
-   *
-   * @param <T> element type
-   */
-  public static class ElasticsearchQueryable<T> extends AbstractTableQueryable<T> {
-    public ElasticsearchQueryable(QueryProvider queryProvider, SchemaPlus schema,
-        ElasticsearchTable table, String tableName) {
-      super(queryProvider, schema, table, tableName);
-    }
-
-    public Enumerator<T> enumerator() {
-      return null;
-    }
-
-    private String getIndex() {
-      return schema.unwrap(ElasticsearchSchema.class).index;
-    }
-
-    private ElasticsearchTable getTable() {
-      return (ElasticsearchTable) table;
-    }
-
-    /** Called via code-generation.
-     *
-     * @see org.apache.calcite.adapter.elasticsearch.ElasticsearchMethod#ELASTICSEARCH_QUERYABLE_FIND
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    public Enumerable<Object> find(List<String> ops,
-        List<Map.Entry<String, Class>> fields) {
-      return getTable().find(getIndex(), ops, fields);
-    }
-  }
-}
-
-// End ElasticsearchTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java
deleted file mode 100644
index 636a629..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchTableScan.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataType;
-
-import java.util.List;
-
-/**
- * Relational expression representing a scan of an Elasticsearch type.
- *
- * <p> Additional operations might be applied,
- * using the "find" method.</p>
- */
-public class ElasticsearchTableScan extends TableScan implements ElasticsearchRel {
-  private final ElasticsearchTable elasticsearchTable;
-  private final RelDataType projectRowType;
-
-  /**
-   * Creates an ElasticsearchTableScan.
-   *
-   * @param cluster Cluster
-   * @param traitSet Trait set
-   * @param table Table
-   * @param elasticsearchTable Elasticsearch table
-   * @param projectRowType Fields and types to project; null to project raw row
-   */
-  protected ElasticsearchTableScan(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table,
-      ElasticsearchTable elasticsearchTable, RelDataType projectRowType) {
-    super(cluster, traitSet, table);
-    this.elasticsearchTable = elasticsearchTable;
-    this.projectRowType = projectRowType;
-
-    assert elasticsearchTable != null;
-    assert getConvention() == ElasticsearchRel.CONVENTION;
-  }
-
-  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    assert inputs.isEmpty();
-    return this;
-  }
-
-  @Override public RelDataType deriveRowType() {
-    return projectRowType != null ? projectRowType : super.deriveRowType();
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    final float f = projectRowType == null ? 1f : (float) projectRowType.getFieldCount() / 100f;
-    return super.computeSelfCost(planner, mq).multiplyBy(.1 * f);
-  }
-
-  @Override public void register(RelOptPlanner planner) {
-    planner.addRule(ElasticsearchToEnumerableConverterRule.INSTANCE);
-    for (RelOptRule rule: ElasticsearchRules.RULES) {
-      planner.addRule(rule);
-    }
-  }
-
-  @Override public void implement(Implementor implementor) {
-    implementor.elasticsearchTable = elasticsearchTable;
-    implementor.table = table;
-  }
-}
-
-// End ElasticsearchTableScan.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java
deleted file mode 100644
index adb88f7..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverter.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.adapter.enumerable.EnumerableRel;
-import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
-import org.apache.calcite.adapter.enumerable.JavaRowFormat;
-import org.apache.calcite.adapter.enumerable.PhysType;
-import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
-
-import org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.linq4j.tree.MethodCallExpression;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalcitePrepareImpl;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterImpl;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.runtime.Hook;
-import org.apache.calcite.util.BuiltInMethod;
-import org.apache.calcite.util.Pair;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-import java.util.AbstractList;
-import java.util.List;
-import javax.annotation.Nullable;
-
-/**
- * Relational expression representing a scan of a table in an Elasticsearch data source.
- */
-public class ElasticsearchToEnumerableConverter extends ConverterImpl implements EnumerableRel {
-  protected ElasticsearchToEnumerableConverter(RelOptCluster cluster, RelTraitSet traits,
-      RelNode input) {
-    super(cluster, ConventionTraitDef.INSTANCE, traits, input);
-  }
-
-  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new ElasticsearchToEnumerableConverter(getCluster(), traitSet, sole(inputs));
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return super.computeSelfCost(planner, mq).multiplyBy(.1);
-  }
-
-  @Override public Result implement(EnumerableRelImplementor implementor, Prefer prefer) {
-    final BlockBuilder list = new BlockBuilder();
-    final ElasticsearchRel.Implementor elasticsearchImplementor =
-        new ElasticsearchRel.Implementor();
-    elasticsearchImplementor.visitChild(0, getInput());
-    final RelDataType rowType = getRowType();
-    final PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), rowType,
-        prefer.prefer(JavaRowFormat.ARRAY));
-    final Expression fields = list.append("fields",
-        constantArrayList(
-            Pair.zip(ElasticsearchRules.elasticsearchFieldNames(rowType),
-                new AbstractList<Class>() {
-                  @Override public Class get(int index) {
-                    return physType.fieldClass(index);
-                  }
-
-                  @Override public int size() {
-                    return rowType.getFieldCount();
-                  }
-                }),
-            Pair.class));
-    final Expression table = list.append("table",
-        elasticsearchImplementor.table
-            .getExpression(ElasticsearchTable.ElasticsearchQueryable.class));
-    List<String> opList = elasticsearchImplementor.list;
-    final Expression ops = list.append("ops", constantArrayList(opList, String.class));
-    Expression enumerable = list.append("enumerable",
-        Expressions.call(table, ElasticsearchMethod.ELASTICSEARCH_QUERYABLE_FIND.method, ops,
-            fields));
-    if (CalcitePrepareImpl.DEBUG) {
-      System.out.println("Elasticsearch: " + opList);
-    }
-    Hook.QUERY_PLAN.run(opList);
-    list.add(Expressions.return_(null, enumerable));
-    return implementor.result(physType, list.toBlock());
-  }
-
-  /** E.g. {@code constantArrayList("x", "y")} returns
-   * "Arrays.asList('x', 'y')". */
-  private static <T> MethodCallExpression constantArrayList(List<T> values, Class clazz) {
-    return Expressions.call(BuiltInMethod.ARRAYS_AS_LIST.method,
-        Expressions.newArrayInit(clazz, constantList(values)));
-  }
-
-  /** E.g. {@code constantList("x", "y")} returns
-   * {@code {ConstantExpression("x"), ConstantExpression("y")}}. */
-  private static <T> List<Expression> constantList(List<T> values) {
-    return Lists.transform(values,
-        new Function<T, Expression>() {
-          @Nullable
-          @Override public Expression apply(@Nullable T t) {
-            return Expressions.constant(t);
-          }
-        });
-  }
-}
-
-// End ElasticsearchToEnumerableConverter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java
deleted file mode 100644
index 1047757..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchToEnumerableConverterRule.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.adapter.enumerable.EnumerableConvention;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-
-/**
- * Rule to convert a relational expression from
- * {@link ElasticsearchRel#CONVENTION} to {@link EnumerableConvention}.
- */
-public class ElasticsearchToEnumerableConverterRule extends ConverterRule {
-  public static final ConverterRule INSTANCE = new ElasticsearchToEnumerableConverterRule();
-
-  private ElasticsearchToEnumerableConverterRule() {
-    super(RelNode.class, ElasticsearchRel.CONVENTION, EnumerableConvention.INSTANCE,
-        "ElasticsearchToEnumerableConverterRule");
-  }
-
-  @Override public RelNode convert(RelNode relNode) {
-    RelTraitSet newTraitSet = relNode.getTraitSet().replace(getOutConvention());
-    return new ElasticsearchToEnumerableConverter(relNode.getCluster(), newTraitSet, relNode);
-  }
-}
-
-// End ElasticsearchToEnumerableConverterRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java b/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java
deleted file mode 100644
index dad800a..0000000
--- a/elasticsearch/src/main/java/org/apache/calcite/adapter/elasticsearch/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Query provider based on an Elasticsearch DB.
- */
-@PackageMarker
-package org.apache.calcite.adapter.elasticsearch;
-
-import org.apache.calcite.avatica.util.PackageMarker;
-
-// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/test/java/org/apache/calcite/test/ElasticsearchAdapterIT.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/java/org/apache/calcite/test/ElasticsearchAdapterIT.java b/elasticsearch/src/test/java/org/apache/calcite/test/ElasticsearchAdapterIT.java
deleted file mode 100644
index 89660a5..0000000
--- a/elasticsearch/src/test/java/org/apache/calcite/test/ElasticsearchAdapterIT.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.test;
-
-import org.apache.calcite.util.Util;
-
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableMap;
-
-import org.junit.Test;
-
-import java.util.List;
-import javax.annotation.Nullable;
-
-/**
- * Tests for the {@code org.apache.calcite.adapter.elasticsearch} package.
- *
- * <p>Before calling this test, you need to populate Elasticsearch, as follows:
- *
- * <blockquote><code>
- * git clone https://github.com/vlsi/calcite-test-dataset<br>
- * cd calcite-test-dataset<br>
- * mvn install
- * </code></blockquote>
- *
- * <p>This will create a virtual machine with Elasticsearch and the "zips" test
- * dataset.
- */
-public class ElasticsearchAdapterIT {
-  /**
-   * Whether to run Elasticsearch tests. Enabled by default, however test is only
-   * included if "it" profile is activated ({@code -Pit}). To disable,
-   * specify {@code -Dcalcite.test.elasticsearch=false} on the Java command line.
-   */
-  private static final boolean ENABLED = Util.getBooleanProperty("calcite.test.elasticsearch",
-      true);
-
-  /** Connection factory based on the "zips-es" model. */
-  private static final ImmutableMap<String, String> ZIPS = ImmutableMap.of("model",
-      ElasticsearchAdapterIT.class.getResource("/elasticsearch-zips-model.json").getPath());
-
-  /** Whether to run this test. */
-  private boolean enabled() {
-    return ENABLED;
-  }
-
-  /** Returns a function that checks that a particular Elasticsearch pipeline is
-   * generated to implement a query. */
-  private static Function<List, Void> elasticsearchChecker(final String... strings) {
-    return new Function<List, Void>() {
-      @Nullable
-      @Override public Void apply(@Nullable List actual) {
-        Object[] actualArray = actual == null || actual.isEmpty() ? null
-            : ((List) actual.get(0)).toArray();
-        CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
-            actualArray);
-        return null;
-      }
-    };
-  }
-
-  @Test public void testSort() {
-    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
-        + "  ElasticsearchSort(sort0=[$4], dir0=[ASC])\n"
-        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips order by \"state\"")
-        .returnsCount(10)
-        .explainContains(explain);
-  }
-
-  @Test public void testSortLimit() {
-    final String sql = "select \"state\", \"id\" from zips\n"
-        + "order by \"state\", \"id\" offset 2 rows fetch next 3 rows only";
-    CalciteAssert.that()
-        .with(ZIPS)
-        .query(sql)
-        .returnsUnordered("state=AK; id=99503",
-            "state=AK; id=99504",
-            "state=AK; id=99505")
-        .queryContains(
-            elasticsearchChecker(
-                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}",
-                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]",
-                "\"from\": 2",
-                "\"size\": 3"));
-  }
-
-  @Test public void testOffsetLimit() {
-    final String sql = "select \"state\", \"id\" from zips\n"
-        + "offset 2 fetch next 3 rows only";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .runs()
-        .queryContains(
-            elasticsearchChecker(
-                "\"from\": 2",
-                "\"size\": 3",
-                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
-  }
-
-  @Test public void testLimit() {
-    final String sql = "select \"state\", \"id\" from zips\n"
-        + "fetch next 3 rows only";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .runs()
-        .queryContains(
-            elasticsearchChecker(
-                "\"size\": 3",
-                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
-  }
-
-  @Test public void testFilterSort() {
-    final String sql = "select * from zips\n"
-        + "where \"city\" = 'SPRINGFIELD' and \"id\" >= '70000'\n"
-        + "order by \"state\", \"id\"";
-    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
-        + "  ElasticsearchSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
-        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-        + "      ElasticsearchFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
-        + "        ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .returnsOrdered(
-            "city=SPRINGFIELD; longitude=-92.54567; latitude=35.274879; pop=752; state=AR; id=72157",
-            "city=SPRINGFIELD; longitude=-102.617322; latitude=37.406727; pop=1992; state=CO; id=81073",
-            "city=SPRINGFIELD; longitude=-90.577479; latitude=30.415738; pop=5597; state=LA; id=70462",
-            "city=SPRINGFIELD; longitude=-123.015259; latitude=44.06106; pop=32384; state=OR; id=97477",
-            "city=SPRINGFIELD; longitude=-122.917108; latitude=44.056056; pop=27521; state=OR; id=97478")
-        .queryContains(
-            elasticsearchChecker("\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
-                    + "{\"must\":[{\"term\":{\"city\":\"springfield\"}},{\"range\":{\"id\":{\"gte\":\"70000\"}}}]}}}}",
-                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}",
-                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]"))
-        .explainContains(explain);
-  }
-
-  @Test public void testFilterSortDesc() {
-    final String sql = "select * from zips\n"
-        + "where \"pop\" BETWEEN 20000 AND 20100\n"
-        + "order by \"state\" desc, \"pop\"";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .limit(4)
-        .returnsOrdered(
-            "city=SHERIDAN; longitude=-106.964795; latitude=44.78486; pop=20025; state=WY; id=82801",
-            "city=MOUNTLAKE TERRAC; longitude=-122.304036; latitude=47.793061; pop=20059; state=WA; id=98043",
-            "city=FALMOUTH; longitude=-77.404537; latitude=38.314557; pop=20039; state=VA; id=22405",
-            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104");
-  }
-
-  @Test public void testFilterRedundant() {
-    final String sql = "select * from zips\n"
-        + "where \"state\" > 'CA' and \"state\" < 'AZ' and \"state\" = 'OK'";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .runs()
-        .queryContains(
-            elasticsearchChecker(""
-                + "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
-                + "{\"must\":[{\"term\":{\"state\":\"ok\"}}]}}}}",
-                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"));
-  }
-
-  @Test public void testInPlan() {
-    final String[] searches = {
-        "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":{\"should\":"
-          + "[{\"bool\":{\"must\":[{\"term\":{\"pop\":20012}}]}},{\"bool\":{\"must\":[{\"term\":"
-          + "{\"pop\":15590}}]}}]}}}}",
-        "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"
-    };
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips where \"pop\" in (20012, 15590)")
-        .returnsUnordered(
-            "city=COVINA; longitude=-117.884285; latitude=34.08596; pop=15590; state=CA; id=91723",
-            "city=ARLINGTON; longitude=-97.091987; latitude=32.654752; pop=15590; state=TX; id=76018",
-            "city=CROFTON; longitude=-76.680166; latitude=39.011163; pop=15590; state=MD; id=21114",
-            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104",
-            "city=DINUBA; longitude=-119.39087; latitude=36.534931; pop=20012; state=CA; id=93618")
-        .queryContains(elasticsearchChecker(searches));
-  }
-
-  @Test public void testZips() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips")
-        .returnsCount(10);
-  }
-
-  @Test public void testProject() {
-    final String sql = "select \"state\", \"city\", 0 as \"zero\"\n"
-        + "from zips\n"
-        + "order by \"state\", \"city\"";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .limit(2)
-        .returnsUnordered("state=AK; city=ELMENDORF AFB; zero=0",
-            "state=AK; city=EIELSON AFB; zero=0")
-        .queryContains(
-            elasticsearchChecker("\"sort\": [ {\"state\": \"asc\"}, {\"city\": \"asc\"}]",
-                "\"fields\" : [\"state\", \"city\"], \"script_fields\": {\"zero\":{\"script\": \"0\"}}"));
-  }
-
-  @Test public void testFilter() {
-    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
-        + "  ElasticsearchProject(state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-        + "    ElasticsearchFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
-        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips where \"state\" = 'CA'")
-        .limit(2)
-        .returnsUnordered("state=CA; city=LOS ANGELES",
-            "state=CA; city=LOS ANGELES")
-        .explainContains(explain);
-  }
-
-  @Test public void testFilterReversed() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips where 'WI' < \"state\"")
-        .limit(2)
-        .returnsUnordered("state=WV; city=WELCH",
-            "state=WV; city=HANOVER");
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips where \"state\" > 'WI'")
-        .limit(2)
-        .returnsUnordered("state=WV; city=WELCH",
-            "state=WV; city=HANOVER");
-  }
-}
-
-// End ElasticsearchAdapterIT.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/test/resources/elasticsearch-zips-model.json
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/resources/elasticsearch-zips-model.json b/elasticsearch/src/test/resources/elasticsearch-zips-model.json
deleted file mode 100644
index dcbf2a4..0000000
--- a/elasticsearch/src/test/resources/elasticsearch-zips-model.json
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the License); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an AS IS BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-{
-  "version": "1.0",
-  "defaultSchema": "elasticsearch",
-  "schemas": [
-    {
-      "type": "custom",
-      "name": "elasticsearch_raw",
-      "factory": "org.apache.calcite.adapter.elasticsearch.ElasticsearchSchemaFactory",
-      "operand": {
-        "coordinates": "{'127.0.0.1': 9300}",
-        "userConfig": "{'bulk.flush.max.actions': 10, 'bulk.flush.max.size.mb': 1}",
-        "index": "usa"
-      }
-    },
-    {
-      "name": "elasticsearch",
-      "tables": [
-        {
-          "name": "ZIPS",
-          "type": "view",
-          "sql": [
-            "select cast(_MAP['city'] AS varchar(20)) AS \"city\",\n",
-            " cast(_MAP['loc'][0] AS float) AS \"longitude\",\n",
-            " cast(_MAP['loc'][1] AS float) AS \"latitude\",\n",
-            " cast(_MAP['pop'] AS integer) AS \"pop\",\n",
-            " cast(_MAP['state'] AS varchar(2)) AS \"state\",\n",
-            " cast(_MAP['id'] AS varchar(5)) AS \"id\"\n",
-            "from \"elasticsearch_raw\".\"zips\""
-          ]
-        }
-      ]
-    }
-  ]
-}

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/resources/log4j.properties b/elasticsearch/src/test/resources/log4j.properties
deleted file mode 100644
index 834e2db..0000000
--- a/elasticsearch/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,24 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to you under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Root logger is configured at INFO and is sent to A1
-log4j.rootLogger=INFO, A1
-
-# A1 goes to the console
-log4j.appender.A1=org.apache.log4j.ConsoleAppender
-
-# Set the pattern for each log message
-log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%d [%t] %-5p - %m%n

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/pom.xml
----------------------------------------------------------------------
diff --git a/elasticsearch2/pom.xml b/elasticsearch2/pom.xml
new file mode 100644
index 0000000..6a645ac
--- /dev/null
+++ b/elasticsearch2/pom.xml
@@ -0,0 +1,148 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.calcite</groupId>
+    <artifactId>calcite</artifactId>
+    <version>1.14.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>calcite-elasticsearch2</artifactId>
+  <packaging>jar</packaging>
+  <version>1.14.0-SNAPSHOT</version>
+  <name>Calcite Elasticsearch</name>
+  <description>Elasticsearch adapter for Calcite</description>
+
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <type>jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-linq4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.elasticsearch</groupId>
+      <artifactId>elasticsearch</artifactId>
+      <version>${elasticsearch-java-driver.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.carrotsearch</groupId>
+      <artifactId>hppc</artifactId>
+      <version>${hppc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>${maven-dependency-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>org.apache.calcite.avatica:avatica</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-api</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-log4j12</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-release-plugin</artifactId>
+      </plugin>
+      <!-- Parent module has the same plugin and does the work of
+          generating -sources.jar for each project. But without the
+          plugin declared here, IDEs don't know the sources are
+          available. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>attach-sources</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>jar-no-fork</goal>
+              <goal>test-jar-no-fork</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
new file mode 100644
index 0000000..84370ab
--- /dev/null
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
@@ -0,0 +1,152 @@
+/*
+ * 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.adapter.elasticsearch2;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.Primitive;
+
+import org.elasticsearch.search.SearchHit;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Enumerator that reads from an Elasticsearch type.
+ */
+public class Elasticsearch2Enumerator implements Enumerator<Object> {
+  private final Iterator<SearchHit> cursor;
+  private final Function1<SearchHit, Object> getter;
+  private Object current;
+
+  /**
+   * Creates an ElasticsearchEnumerator.
+   *
+   * @param cursor Iterator over Elasticsearch {@link SearchHit} objects
+   * @param getter Converts an object into a list of fields
+   */
+  public Elasticsearch2Enumerator(Iterator<SearchHit> cursor,
+      Function1<SearchHit, Object> getter) {
+    this.cursor = cursor;
+    this.getter = getter;
+  }
+
+  public Object current() {
+    return current;
+  }
+
+  public boolean moveNext() {
+    if (cursor.hasNext()) {
+      SearchHit map = cursor.next();
+      current = getter.apply(map);
+      return true;
+    } else {
+      current = null;
+      return false;
+    }
+  }
+
+  public void reset() {
+    throw new UnsupportedOperationException();
+  }
+
+  public void close() {
+    // nothing to do
+  }
+
+  private static Function1<SearchHit, Map> mapGetter() {
+    return new Function1<SearchHit, Map>() {
+      public Map apply(SearchHit searchHitFields) {
+        return (Map) searchHitFields.fields();
+      }
+    };
+  }
+
+  private static Function1<SearchHit, Object> singletonGetter(final String fieldName,
+      final Class fieldClass) {
+    return new Function1<SearchHit, Object>() {
+      public Object apply(SearchHit searchHitFields) {
+        if (searchHitFields.fields().isEmpty()) {
+          return convert(searchHitFields.getSource(), fieldClass);
+        } else {
+          return convert(searchHitFields.getFields(), fieldClass);
+        }
+      }
+    };
+  }
+
+  /**
+   * Function that extracts a given set of fields from {@link SearchHit}
+   * objects.
+   *
+   * @param fields List of fields to project
+   */
+  private static Function1<SearchHit, Object[]> listGetter(
+      final List<Map.Entry<String, Class>> fields) {
+    return new Function1<SearchHit, Object[]>() {
+      public Object[] apply(SearchHit searchHitFields) {
+        Object[] objects = new Object[fields.size()];
+        for (int i = 0; i < fields.size(); i++) {
+          final Map.Entry<String, Class> field = fields.get(i);
+          final String name = field.getKey();
+          if (searchHitFields.fields().isEmpty()) {
+            objects[i] = convert(searchHitFields.getSource().get(name), field.getValue());
+          } else {
+            objects[i] = convert(searchHitFields.field(name).getValue(), field.getValue());
+          }
+        }
+        return objects;
+      }
+    };
+  }
+
+  static Function1<SearchHit, Object> getter(List<Map.Entry<String, Class>> fields) {
+    //noinspection unchecked
+    return fields == null
+      ? (Function1) mapGetter()
+      : fields.size() == 1
+      ? singletonGetter(fields.get(0).getKey(), fields.get(0).getValue())
+      : (Function1) listGetter(fields);
+  }
+
+  private static Object convert(Object o, Class clazz) {
+    if (o == null) {
+      return null;
+    }
+    Primitive primitive = Primitive.of(clazz);
+    if (primitive != null) {
+      clazz = primitive.boxClass;
+    } else {
+      primitive = Primitive.ofBox(clazz);
+    }
+    if (clazz.isInstance(o)) {
+      return o;
+    }
+    if (o instanceof Date && primitive != null) {
+      o = ((Date) o).getTime() / DateTimeUtils.MILLIS_PER_DAY;
+    }
+    if (o instanceof Number && primitive != null) {
+      return primitive.number((Number) o);
+    }
+    return o;
+  }
+}
+
+// End Elasticsearch2Enumerator.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
new file mode 100644
index 0000000..668402b
--- /dev/null
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
@@ -0,0 +1,134 @@
+/*
+ * 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.adapter.elasticsearch2;
+
+import org.apache.calcite.adapter.elasticsearch.ElasticsearchSchema;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
+import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
+import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.cluster.metadata.MappingMetaData;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.collect.ImmutableOpenMap;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.elasticsearch.common.transport.TransportAddress;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Schema mapped onto an index of ELASTICSEARCH types.
+ *
+ * <p>Each table in the schema is an ELASTICSEARCH type in that index.
+ */
+public class Elasticsearch2Schema extends AbstractSchema
+    implements ElasticsearchSchema {
+  final String index;
+
+  private transient Client client;
+
+  /**
+   * Creates an Elasticsearch2 schema.
+   *
+   * @param coordinates Map of Elasticsearch node locations (host, port)
+   * @param userConfig Map of user-specified configurations
+   * @param indexName Elasticsearch database name, e.g. "usa".
+   */
+  Elasticsearch2Schema(Map<String, Integer> coordinates,
+      Map<String, String> userConfig, String indexName) {
+    super();
+
+    final List<InetSocketAddress> transportAddresses = new ArrayList<>();
+    for (Map.Entry<String, Integer> coordinate: coordinates.entrySet()) {
+      transportAddresses.add(
+          new InetSocketAddress(coordinate.getKey(), coordinate.getValue()));
+    }
+
+    open(transportAddresses, userConfig);
+
+    if (client != null) {
+      final String[] indices = client.admin().indices()
+          .getIndex(new GetIndexRequest().indices(indexName))
+          .actionGet().getIndices();
+      if (indices.length == 1) {
+        index = indices[0];
+      } else {
+        index = null;
+      }
+    } else {
+      index = null;
+    }
+  }
+
+  @Override protected Map<String, Table> getTableMap() {
+    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+
+    try {
+      GetMappingsResponse response = client.admin().indices()
+          .getMappings(new GetMappingsRequest().indices(index))
+          .get();
+      ImmutableOpenMap<String, MappingMetaData> mapping = response.getMappings().get(index);
+      for (ObjectObjectCursor<String, MappingMetaData> c: mapping) {
+        builder.put(c.key, new Elasticsearch2Table(client, index, c.key));
+      }
+    } catch (RuntimeException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return builder.build();
+  }
+
+  private void open(List<InetSocketAddress> transportAddresses, Map<String, String> userConfig) {
+    final List<TransportAddress> transportNodes = new ArrayList<>(transportAddresses.size());
+    for (InetSocketAddress address : transportAddresses) {
+      transportNodes.add(new InetSocketTransportAddress(address));
+    }
+
+    Settings settings = Settings.settingsBuilder().put(userConfig).build();
+
+    final TransportClient transportClient = TransportClient.builder().settings(settings).build();
+    for (TransportAddress transport : transportNodes) {
+      transportClient.addTransportAddress(transport);
+    }
+
+    final List<DiscoveryNode> nodes = ImmutableList.copyOf(transportClient.connectedNodes());
+    if (nodes.isEmpty()) {
+      throw new RuntimeException("Cannot connect to any elasticsearch nodes");
+    }
+
+    client = transportClient;
+  }
+
+  @Override public String getIndex() {
+    return index;
+  }
+}
+
+// End Elasticsearch2Schema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e1525926/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2SchemaFactory.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2SchemaFactory.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2SchemaFactory.java
new file mode 100644
index 0000000..19378a0
--- /dev/null
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2SchemaFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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.adapter.elasticsearch2;
+
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Factory that creates an {@link Elasticsearch2Schema}.
+ *
+ * <p>Allows a custom schema to be included in a model.json file.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public class Elasticsearch2SchemaFactory implements SchemaFactory {
+
+  public Elasticsearch2SchemaFactory() {
+  }
+
+  @Override public Schema create(SchemaPlus parentSchema, String name,
+      Map<String, Object> operand) {
+    final Map map = (Map) operand;
+
+    final ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true);
+
+    try {
+      final Map<String, Integer> coordinates =
+          mapper.readValue((String) map.get("coordinates"),
+              new TypeReference<Map<String, Integer>>() { });
+      final Map<String, String> userConfig =
+          mapper.readValue((String) map.get("userConfig"),
+              new TypeReference<Map<String, String>>() { });
+      final String index = (String) map.get("index");
+      return new Elasticsearch2Schema(coordinates, userConfig, index);
+    } catch (IOException e) {
+      throw new RuntimeException("Cannot parse values from json", e);
+    }
+  }
+}
+
+// End Elasticsearch2SchemaFactory.java


[16/16] calcite git commit: Update version numbers for 1.14 release

Posted by mm...@apache.org.
Update version numbers for 1.14 release


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

Branch: refs/heads/branch-1.14
Commit: 5410e79aaf788f6fb59c9c72c5f672e7dd02e5b3
Parents: 3520913
Author: Michael Mior <mm...@uwaterloo.ca>
Authored: Mon Aug 28 17:03:37 2017 -0400
Committer: Michael Mior <mm...@uwaterloo.ca>
Committed: Thu Aug 31 04:51:50 2017 -0400

----------------------------------------------------------------------
 README              | 2 +-
 pom.xml             | 2 +-
 site/_docs/howto.md | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/5410e79a/README
----------------------------------------------------------------------
diff --git a/README b/README
index 48b3d4a..b2cd7e3 100644
--- a/README
+++ b/README
@@ -1,4 +1,4 @@
-Apache Calcite release 1.13.0
+Apache Calcite release 1.14.0
 
 This is a source or binary distribution of Apache Calcite.
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5410e79a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cb83c87..e888a11 100644
--- a/pom.xml
+++ b/pom.xml
@@ -49,7 +49,7 @@ limitations under the License.
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <top.dir>${project.basedir}</top.dir>
     <version.major>1</version.major>
-    <version.minor>13</version.minor>
+    <version.minor>14</version.minor>
 
     <!-- This list is in alphabetical order. -->
     <airlift-tpch.version>0.1</airlift-tpch.version>

http://git-wip-us.apache.org/repos/asf/calcite/blob/5410e79a/site/_docs/howto.md
----------------------------------------------------------------------
diff --git a/site/_docs/howto.md b/site/_docs/howto.md
index 8a5617d..b570725 100644
--- a/site/_docs/howto.md
+++ b/site/_docs/howto.md
@@ -39,8 +39,8 @@ Unpack the source distribution `.tar.gz` or `.zip` file,
 then build using maven:
 
 {% highlight bash %}
-$ tar xvfz calcite-1.13.0-source.tar.gz
-$ cd calcite-1.13.0
+$ tar xvfz calcite-1.14.0-source.tar.gz
+$ cd calcite-1.14.0
 $ mvn install
 {% endhighlight %}